diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java index 90f6a90..d0fbc9e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java @@ -89,7 +89,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; @@ -624,7 +623,7 @@ public class HConnectionManager { * @return */ public String toString(){ - return "hconnection 0x" + Integer.toHexString( hashCode() ); + return "hconnection_0x" + Integer.toHexString(hashCode()); } private String clusterId = null; @@ -980,13 +979,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 root 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 root 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) { @@ -1787,26 +1789,17 @@ public class HConnectionManager { } @Deprecated - private Callable createCallable( - final HRegionLocation loc, final MultiAction multi, - final byte [] tableName) { - // TODO: This does not belong in here!!! St.Ack HConnections should + private Callable createCallable(final HRegionLocation loc, + final MultiAction 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() { - public MultiResponse call() throws IOException { + @Override + public MultiResponse call() throws Exception { ServerCallable callable = - new ServerCallable(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(connection, tableName, loc, multi); return callable.withoutRetries(); } }; @@ -2128,8 +2121,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); } } } @@ -2188,8 +2180,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. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java index bbf5e78..382add5 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java @@ -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; @@ -731,9 +732,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); } @@ -755,8 +758,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); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java new file mode 100644 index 0000000..572013a --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java @@ -0,0 +1,111 @@ +/** + * 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 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 multi method calls against regionservers. + * @param + */ +class MultiServerCallable extends ServerCallable { + private final MultiAction multi; + private final HRegionLocation loc; + + MultiServerCallable(final HConnection connection, final byte [] tableName, + final HRegionLocation loc, final MultiAction multi) { + super(connection, tableName, null); + this.multi = multi; + this.loc = loc; + } + + @Override + public MultiResponse call() throws IOException { + MultiResponse response = new MultiResponse(); + try { + // The multi object is a list of Actions by region. + for (Map.Entry>> e: this.multi.actions.entrySet()) { + byte[] regionName = e.getKey(); + int rowMutations = 0; + List> actions = e.getValue(); + for (Action 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) { + RowMutations rms = (RowMutations)row; + // Stick all Cells for all RowMutations in here into 'cells'. Populated when we call + // buildNoDataMultiRequest in the below. + List cells = new ArrayList(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); + rowMutations++; + } + } + // Are there any non-RowMutation actions to send for this region? + if (actions.size() > rowMutations) { + // Stick all Cells for the multiRequest in here into 'cells'. Gets filled in when we + // call buildNoDataMultiRequest + List cells = new ArrayList(actions.size() - rowMutations); + // 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); + List results = + ResponseConverter.getResults(responseProto, controller.cellScanner()); + for (int i = 0, n = results.size(); i < n; i++) { + int originalIndex = actions.get(i).getOriginalIndex(); + response.add(regionName, originalIndex, results.get(i)); + } + } + } + return response; + } catch (ServiceException se) { + throw ProtobufUtil.getRemoteException(se); + } + } + + @Override + public void connect(boolean reload) throws IOException { + server = connection.getClient(loc.getServerName()); + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FatalConnectionException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FatalConnectionException.java new file mode 100644 index 0000000..d6193e1 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FatalConnectionException.java @@ -0,0 +1,40 @@ +/** + * 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.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") +public class FatalConnectionException extends DoNotRetryIOException { + public FatalConnectionException() { + super(); + } + + public FatalConnectionException(String msg) { + super(msg); + } + + public FatalConnectionException(String msg, Throwable t) { + super(msg, t); + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java index 717259d..f9f74c2 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java @@ -19,23 +19,24 @@ package org.apache.hadoop.hbase.ipc; -import com.google.protobuf.CodedOutputStream; import com.google.protobuf.Message; import com.google.protobuf.Message.Builder; +import com.google.protobuf.TextFormat; + 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.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; @@ -51,6 +52,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; @@ -67,7 +69,6 @@ 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; @@ -79,6 +80,7 @@ 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; @@ -86,29 +88,23 @@ import java.util.LinkedList; import java.util.Map; import java.util.Map.Entry; import java.util.Random; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; -/** A client for an IPC service. IPC calls take a single Protobuf message as a - * parameter, and return a single Protobuf message as their value. A service runs on +/** + * A client for an IPC service. IPC calls take a single Protobuf message as a + * request and returns a single Protobuf message as result. A service runs on * a port and is defined by a parameter class and a value class. * - *

This is the org.apache.hadoop.ipc.Client renamed as HBaseClient and - * moved into this package so can access package-private methods. - * - * See HBaseServer + *

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 connections; - private static final Map methodInstances = - new ConcurrentHashMap(); + private ReflectionCache reflectionCache = new ReflectionCache(); protected int counter; // counter for call ids protected final AtomicBoolean running = new AtomicBoolean(true); // if client runs @@ -122,6 +118,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; @@ -184,12 +183,10 @@ public class HBaseClient { return false; } - } + @SuppressWarnings("serial") public static class FailedServerException extends IOException { - private static final long serialVersionUID = -4744376109431464127L; - public FailedServerException(String s) { super(s); } @@ -201,6 +198,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 +234,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 +282,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 +296,7 @@ public class HBaseClient { } } - protected static Map> tokenHandlers = + protected final static Map> tokenHandlers = new HashMap>(); static { tokenHandlers.put(AuthenticationTokenIdentifier.AUTH_TOKEN_TYPE.toString(), @@ -292,8 +307,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 +329,8 @@ public class HBaseClient { private Token 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 calls = @@ -322,12 +341,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 +389,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; @@ -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); @@ -627,12 +653,11 @@ public class HBaseClient { @Override public void run() { if (LOG.isDebugEnabled()) - LOG.debug(getName() + ": starting, having connections " - + connections.size()); + LOG.debug("Starting, having 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); @@ -642,8 +667,7 @@ public class HBaseClient { close(); if (LOG.isDebugEnabled()) - LOG.debug(getName() + ": stopped, remaining connections " - + connections.size()); + LOG.debug("Stopped, remaining connections " + connections.size()); } private synchronized void disposeSasl() { @@ -691,7 +715,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). *

*/ @@ -775,7 +799,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 +812,22 @@ public class HBaseClient { } boolean continueSasl = false; try { - continueSasl = - ticket.doAs(new PrivilegedExceptionAction() { - @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() { + @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 +840,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,24 +867,23 @@ 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: + */ + private void writeConnectionHeaderPreamble(OutputStream outStream) throws IOException { + outStream.write(HConstants.RPC_HEADER.array()); + outStream.write(HConstants.RPC_CURRENT_VERSION); + outStream.write(authMethod.code); // This will write out a single byte. + 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); } /** Close the connection. */ @@ -902,129 +928,98 @@ public class HBaseClient { cleanupCalls(); } if (LOG.isDebugEnabled()) - LOG.debug(getName() + ": closed"); + LOG.debug("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 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, builder.build(), call.param, cellBlock); } } catch(IOException e) { markClosed(e); } } - - private Method getMethod(Class 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(); - } - int id = response.getCallId(); - if (LOG.isDebugEnabled()) - LOG.debug(getName() + " got value #" + id); - Call call = calls.get(id); + // Total size of the response. Unused. But have to read it in anyways. + int totalSize = in.readInt(); - Status status = response.getStatus(); - if (status == Status.SUCCESS) { + // Read the header + ResponseHeader responseHeader = ResponseHeader.parseDelimitedFrom(in); + int id = responseHeader.getCallId(); + if (LOG.isDebugEnabled()) { + LOG.debug("Got response header " + TextFormat.shortDebugString(responseHeader)); + } + Call call = calls.get(id); + 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(); - // 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); + Message value = null; + if (rpcResponseType != null) { + Builder builder = rpcResponseType.newBuilderForType(); + builder.mergeDelimitedFrom(in); + value = builder.build(); } - 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); + 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); } - } else if (status == Status.FATAL) { - RpcException exceptionResponse = RpcException.parseDelimitedFrom(in); - // Close the connection - markClosed(new RemoteException( - exceptionResponse.getExceptionName(), - exceptionResponse.getStackTrace())); + // 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.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 +1038,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 e + */ + 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 +1122,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. @@ -1167,7 +1146,10 @@ public class HBaseClient { if (LOG.isDebugEnabled()) { LOG.debug("The ping interval is" + 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( @@ -1176,6 +1158,35 @@ public class HBaseClient { } /** + * 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 */ @@ -1250,36 +1261,17 @@ public class HBaseClient { } /** Make a call, passing param, to the IPC server running at - * address, 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 param, to the IPC server running at * address which is servicing the protocol protocol, * with the ticket 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 protocol, - User ticket, int rpcTimeout) + public Pair call(Method method, Message param, CellScanner cells, + InetSocketAddress addr, Class 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 +1297,7 @@ public class HBaseClient { // local exception throw wrapException(addr, call.error); } - return call.value; + return new Pair(call.response, call.cells); } } @@ -1340,51 +1332,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 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 protocol, - User ticket, - int rpcTimeout, - Call call) - throws IOException, InterruptedException { + protected Connection getConnection(InetSocketAddress addr, Class 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"); @@ -1398,7 +1350,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); } } @@ -1465,4 +1417,4 @@ public class HBaseClient { (ticket == null ? 0 : ticket.hashCode()) )) ^ rpcTimeout; } } -} \ No newline at end of file +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java new file mode 100644 index 0000000..e015846 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java @@ -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", 4 * 1024); + this.cellBlockDecompressionMultiplier = + conf.getInt("hbase.ipc.cellblock.decompression.buffersize.multiplier", 3); + } + + /** + * Build a cell block using passed in codec + * @param codec + * @param compressor + * @Param cells + * @return Null or byte buffer filled with passed-in Cells encoded using passed in + * codec; 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 cellBlock + * @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 cellBlock + * @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 + * header, param, and cellBlock. + * @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 in + * @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; + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java new file mode 100644 index 0000000..d95de36 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java @@ -0,0 +1,104 @@ +/** + * 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.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. + */ +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 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 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(); + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ProtobufRpcClientEngine.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ProtobufRpcClientEngine.java index 0e16448..6c1ef0a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ProtobufRpcClientEngine.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ProtobufRpcClientEngine.java @@ -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; @@ -72,7 +73,8 @@ public class ProtobufRpcClientEngine implements RpcClientEngine { final private int rpcTimeout; public Invoker(Class 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; @@ -80,30 +82,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 @@ -117,33 +95,51 @@ public class ProtobufRpcClientEngine implements RpcClientEngine { * set as cause in ServiceException * * - * Note that the client calling protobuf RPC methods, must handle + *

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 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); } @@ -153,8 +149,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); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ReflectionCache.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ReflectionCache.java new file mode 100644 index 0000000..18564c0 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ReflectionCache.java @@ -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 methodArgCache = new ConcurrentHashMap(); + private final Map methodInstanceCache = new ConcurrentHashMap(); + + public ReflectionCache() { + super(); + } + + Method getMethod(Class 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; + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RemoteWithExtrasException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RemoteWithExtrasException.java new file mode 100644 index 0000000..7dbcd3b --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RemoteWithExtrasException.java @@ -0,0 +1,65 @@ +/** + * 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.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") +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; + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java index 23b3676..1641e15 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java @@ -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,43 +341,75 @@ 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(); - } - 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"); + 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)); } - byte[] value = qv.getValue().toByteArray(); - long ts = timestamp; - if (qv.hasTimestamp()) { - ts = qv.getTimestamp(); + 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.add(family, qualifier, ts, value); } } + put.setWriteToWAL(proto.getWriteToWAL()); + for (NameBytesPair attribute: proto.getAttributeList()) { + 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); - 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"); + 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)); } - byte[] value = qv.getValue().toByteArray(); - append.add(family, qualifier, value); + 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()); } 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()) { @@ -773,9 +880,9 @@ public final class ProtobufUtil { * @return a mutate * @throws IOException */ - public static Mutate toMutate(final MutateType mutateType, - final Mutation mutation) throws IOException { - Mutate.Builder mutateBuilder = Mutate.newBuilder(); + public static MutationProto toMutation(final MutationType mutateType, final Mutation mutation) + throws IOException { + MutationProto.Builder mutateBuilder = MutationProto.newBuilder(); mutateBuilder.setRow(ByteString.copyFrom(mutation.getRow())); mutateBuilder.setMutateType(mutateType); mutateBuilder.setWriteToWAL(mutation.getWriteToWAL()); @@ -799,7 +906,7 @@ 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 (mutateType == MutationType.DELETE) { KeyValue.Type keyValueType = KeyValue.Type.codeToType(kv.getType()); valueBuilder.setDeleteType(toDeleteType(keyValueType)); } @@ -821,25 +928,66 @@ 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 * * @param proto the protocol buffer Result to convert * @return the converted client Result */ public static Result toResult(final ClientProtos.Result proto) { - List values = proto.getKeyValueList(); - List keyValues = new ArrayList(values.size()); - for (HBaseProtos.KeyValue kv: values) { - keyValues.add(toKeyValue(kv)); + List values = proto.getCellList(); + List cells = new ArrayList(values.size()); + for (HBaseProtos.Cell c: values) { + cells.add(toCell(c)); + } + 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 cells = null; + if (proto.hasAssociatedCellCount()) { + int count = proto.getAssociatedCellCount(); + cells = new ArrayList(count); + for (int i = 0; i < count; i++) { + if (!scanner.advance()) throw new IOException("Failed get " + i + " of " + count); + cells.add(scanner.current()); + } } - return new Result(keyValues); + List values = proto.getCellList(); + if (cells == null) cells = new ArrayList(values.size()); + for (HBaseProtos.Cell c: values) { + cells.add(toCell(c)); + } + return new Result(cells); } /** @@ -1012,55 +1160,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 MultiResponse multi(final ClientProtocol client, - final MultiAction multi) throws IOException { - MultiResponse response = new MultiResponse(); - for (Map.Entry>> e: multi.actions.entrySet()) { - byte[] regionName = e.getKey(); - int rowMutations = 0; - List> actions = e.getValue(); - for (Action 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 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. * * @param client @@ -1731,33 +1830,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()); } /** diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java index ae2e6c3..8d4a47d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.protobuf; import com.google.protobuf.ByteString; 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,13 +59,14 @@ 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.NameBytesPair; 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.MasterAdminProtos.AddColumnRequest; @@ -94,6 +96,7 @@ import org.apache.hadoop.hbase.util.Pair; import java.io.IOException; import java.util.List; +import java.util.Map; /** * Helper utility to build protocol buffer requests, @@ -206,9 +209,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 +220,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 +247,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 +275,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 +294,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 +312,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 +329,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 +347,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,30 +360,91 @@ 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 cells 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 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 = toMutationNoData(type, mutation); + cells.add(mutation); + builder.addAction(MultiAction.newBuilder().setMutation(mp).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 mutation + * @return a mutate + * @throws IOException + */ + public static MutationProto toMutationNoData(final MutationType type, final Mutation mutation) + throws IOException { + MutationProto.Builder mutationBuilder = MutationProto.newBuilder(); + mutationBuilder.setWriteToWAL(mutation.getWriteToWAL()); + mutationBuilder.setMutateType(type); + Map attributes = mutation.getAttributesMap(); + if (!attributes.isEmpty()) { + NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder(); + for (Map.Entry attribute: attributes.entrySet()) { + attributeBuilder.setName(attribute.getKey()); + attributeBuilder.setValue(ByteString.copyFrom(attribute.getValue())); + mutationBuilder.addAttribute(attributeBuilder.build()); + } + } + mutationBuilder.setAssociatedCellCount(mutation.size()); + return mutationBuilder.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 * * @param regionName @@ -475,25 +538,22 @@ public final class RequestConverter { * @throws IOException */ public static MultiRequest buildMultiRequest(final byte[] regionName, - final List> actions) throws IOException { - MultiRequest.Builder builder = MultiRequest.newBuilder(); - RegionSpecifier region = buildRegionSpecifier( - RegionSpecifierType.REGION_NAME, regionName); - builder.setRegion(region); + final List> actions) + throws IOException { + MultiRequest.Builder builder = getMultiRequestBuilderWithRegionAndAtomicSet(regionName, false); for (Action 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 +565,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 + * data 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 MultiRequest buildNoDataMultiRequest(final byte[] regionName, + final List> actions, final List cells) + throws IOException { + MultiRequest.Builder builder = getMultiRequestBuilderWithRegionAndAtomicSet(regionName, false); + for (Action 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(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(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(toMutationNoData(MutationType.APPEND, a)); + } else if (row instanceof Increment) { + Increment i = (Increment)row; + cells.add(i); + protoAction.setMutation(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 diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java index 3967208..67cca13 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java @@ -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 proto. Can be null. + * @return the results that were in the MultiResponse (a Result or an Exception). * @throws IOException */ - public static List getResults( - final ClientProtos.MultiResponse proto) throws IOException { + public static List getResults(final ClientProtos.MultiResponse proto, + final CellScanner cells) + throws IOException { List results = new ArrayList(); List 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()); } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java new file mode 100644 index 0000000..47bee96 --- /dev/null +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java @@ -0,0 +1,81 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.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; + } +} \ No newline at end of file diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestPayloadCarryingRpcController.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestPayloadCarryingRpcController.java new file mode 100644 index 0000000..4fda2ff --- /dev/null +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestPayloadCarryingRpcController.java @@ -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 cells = new ArrayList(); + 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; + } + }; + } + }; + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 5fef359..6aab627 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -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. diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java index 5aff852..e387020 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java @@ -2048,15 +2048,19 @@ public final class ClientProtos { public interface ResultOrBuilder extends com.google.protobuf.MessageOrBuilder { - // repeated .KeyValue keyValue = 1; - java.util.List - getKeyValueList(); - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue getKeyValue(int index); - int getKeyValueCount(); - java.util.List - getKeyValueOrBuilderList(); - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValueOrBuilder getKeyValueOrBuilder( + // repeated .Cell cell = 1; + java.util.List + getCellList(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell getCell(int index); + int getCellCount(); + java.util.List + getCellOrBuilderList(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellOrBuilder getCellOrBuilder( int index); + + // optional int32 associatedCellCount = 2; + boolean hasAssociatedCellCount(); + int getAssociatedCellCount(); } public static final class Result extends com.google.protobuf.GeneratedMessage @@ -2086,41 +2090,47 @@ public final class ClientProtos { return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_Result_fieldAccessorTable; } - // repeated .KeyValue keyValue = 1; - public static final int KEYVALUE_FIELD_NUMBER = 1; - private java.util.List keyValue_; - public java.util.List getKeyValueList() { - return keyValue_; + private int bitField0_; + // repeated .Cell cell = 1; + public static final int CELL_FIELD_NUMBER = 1; + private java.util.List cell_; + public java.util.List getCellList() { + return cell_; } - public java.util.List - getKeyValueOrBuilderList() { - return keyValue_; + public java.util.List + getCellOrBuilderList() { + return cell_; } - public int getKeyValueCount() { - return keyValue_.size(); + public int getCellCount() { + return cell_.size(); } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue getKeyValue(int index) { - return keyValue_.get(index); + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell getCell(int index) { + return cell_.get(index); } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValueOrBuilder getKeyValueOrBuilder( + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellOrBuilder getCellOrBuilder( int index) { - return keyValue_.get(index); + return cell_.get(index); + } + + // optional int32 associatedCellCount = 2; + public static final int ASSOCIATEDCELLCOUNT_FIELD_NUMBER = 2; + private int associatedCellCount_; + public boolean hasAssociatedCellCount() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public int getAssociatedCellCount() { + return associatedCellCount_; } private void initFields() { - keyValue_ = java.util.Collections.emptyList(); + cell_ = java.util.Collections.emptyList(); + associatedCellCount_ = 0; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - for (int i = 0; i < getKeyValueCount(); i++) { - if (!getKeyValue(i).isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } memoizedIsInitialized = 1; return true; } @@ -2128,8 +2138,11 @@ public final class ClientProtos { public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); - for (int i = 0; i < keyValue_.size(); i++) { - output.writeMessage(1, keyValue_.get(i)); + for (int i = 0; i < cell_.size(); i++) { + output.writeMessage(1, cell_.get(i)); + } + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeInt32(2, associatedCellCount_); } getUnknownFields().writeTo(output); } @@ -2140,9 +2153,13 @@ public final class ClientProtos { if (size != -1) return size; size = 0; - for (int i = 0; i < keyValue_.size(); i++) { + for (int i = 0; i < cell_.size(); i++) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, cell_.get(i)); + } + if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, keyValue_.get(i)); + .computeInt32Size(2, associatedCellCount_); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; @@ -2167,8 +2184,13 @@ public final class ClientProtos { org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result) obj; boolean result = true; - result = result && getKeyValueList() - .equals(other.getKeyValueList()); + result = result && getCellList() + .equals(other.getCellList()); + result = result && (hasAssociatedCellCount() == other.hasAssociatedCellCount()); + if (hasAssociatedCellCount()) { + result = result && (getAssociatedCellCount() + == other.getAssociatedCellCount()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -2178,9 +2200,13 @@ public final class ClientProtos { public int hashCode() { int hash = 41; hash = (19 * hash) + getDescriptorForType().hashCode(); - if (getKeyValueCount() > 0) { - hash = (37 * hash) + KEYVALUE_FIELD_NUMBER; - hash = (53 * hash) + getKeyValueList().hashCode(); + if (getCellCount() > 0) { + hash = (37 * hash) + CELL_FIELD_NUMBER; + hash = (53 * hash) + getCellList().hashCode(); + } + if (hasAssociatedCellCount()) { + hash = (37 * hash) + ASSOCIATEDCELLCOUNT_FIELD_NUMBER; + hash = (53 * hash) + getAssociatedCellCount(); } hash = (29 * hash) + getUnknownFields().hashCode(); return hash; @@ -2290,7 +2316,7 @@ public final class ClientProtos { } private void maybeForceBuilderInitialization() { if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { - getKeyValueFieldBuilder(); + getCellFieldBuilder(); } } private static Builder create() { @@ -2299,12 +2325,14 @@ public final class ClientProtos { public Builder clear() { super.clear(); - if (keyValueBuilder_ == null) { - keyValue_ = java.util.Collections.emptyList(); + if (cellBuilder_ == null) { + cell_ = java.util.Collections.emptyList(); bitField0_ = (bitField0_ & ~0x00000001); } else { - keyValueBuilder_.clear(); + cellBuilder_.clear(); } + associatedCellCount_ = 0; + bitField0_ = (bitField0_ & ~0x00000002); return this; } @@ -2342,15 +2370,21 @@ public final class ClientProtos { public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result buildPartial() { org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result(this); int from_bitField0_ = bitField0_; - if (keyValueBuilder_ == null) { + int to_bitField0_ = 0; + if (cellBuilder_ == null) { if (((bitField0_ & 0x00000001) == 0x00000001)) { - keyValue_ = java.util.Collections.unmodifiableList(keyValue_); + cell_ = java.util.Collections.unmodifiableList(cell_); bitField0_ = (bitField0_ & ~0x00000001); } - result.keyValue_ = keyValue_; + result.cell_ = cell_; } else { - result.keyValue_ = keyValueBuilder_.build(); + result.cell_ = cellBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000001; } + result.associatedCellCount_ = associatedCellCount_; + result.bitField0_ = to_bitField0_; onBuilt(); return result; } @@ -2366,43 +2400,40 @@ public final class ClientProtos { public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result other) { if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.getDefaultInstance()) return this; - if (keyValueBuilder_ == null) { - if (!other.keyValue_.isEmpty()) { - if (keyValue_.isEmpty()) { - keyValue_ = other.keyValue_; + if (cellBuilder_ == null) { + if (!other.cell_.isEmpty()) { + if (cell_.isEmpty()) { + cell_ = other.cell_; bitField0_ = (bitField0_ & ~0x00000001); } else { - ensureKeyValueIsMutable(); - keyValue_.addAll(other.keyValue_); + ensureCellIsMutable(); + cell_.addAll(other.cell_); } onChanged(); } } else { - if (!other.keyValue_.isEmpty()) { - if (keyValueBuilder_.isEmpty()) { - keyValueBuilder_.dispose(); - keyValueBuilder_ = null; - keyValue_ = other.keyValue_; + if (!other.cell_.isEmpty()) { + if (cellBuilder_.isEmpty()) { + cellBuilder_.dispose(); + cellBuilder_ = null; + cell_ = other.cell_; bitField0_ = (bitField0_ & ~0x00000001); - keyValueBuilder_ = + cellBuilder_ = com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? - getKeyValueFieldBuilder() : null; + getCellFieldBuilder() : null; } else { - keyValueBuilder_.addAllMessages(other.keyValue_); + cellBuilder_.addAllMessages(other.cell_); } } } + if (other.hasAssociatedCellCount()) { + setAssociatedCellCount(other.getAssociatedCellCount()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } public final boolean isInitialized() { - for (int i = 0; i < getKeyValueCount(); i++) { - if (!getKeyValue(i).isInitialized()) { - - return false; - } - } return true; } @@ -2430,9 +2461,14 @@ public final class ClientProtos { break; } case 10: { - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.newBuilder(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.newBuilder(); input.readMessage(subBuilder, extensionRegistry); - addKeyValue(subBuilder.buildPartial()); + addCell(subBuilder.buildPartial()); + break; + } + case 16: { + bitField0_ |= 0x00000002; + associatedCellCount_ = input.readInt32(); break; } } @@ -2441,190 +2477,211 @@ public final class ClientProtos { private int bitField0_; - // repeated .KeyValue keyValue = 1; - private java.util.List keyValue_ = + // repeated .Cell cell = 1; + private java.util.List cell_ = java.util.Collections.emptyList(); - private void ensureKeyValueIsMutable() { + private void ensureCellIsMutable() { if (!((bitField0_ & 0x00000001) == 0x00000001)) { - keyValue_ = new java.util.ArrayList(keyValue_); + cell_ = new java.util.ArrayList(cell_); bitField0_ |= 0x00000001; } } private com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValueOrBuilder> keyValueBuilder_; + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellOrBuilder> cellBuilder_; - public java.util.List getKeyValueList() { - if (keyValueBuilder_ == null) { - return java.util.Collections.unmodifiableList(keyValue_); + public java.util.List getCellList() { + if (cellBuilder_ == null) { + return java.util.Collections.unmodifiableList(cell_); } else { - return keyValueBuilder_.getMessageList(); + return cellBuilder_.getMessageList(); } } - public int getKeyValueCount() { - if (keyValueBuilder_ == null) { - return keyValue_.size(); + public int getCellCount() { + if (cellBuilder_ == null) { + return cell_.size(); } else { - return keyValueBuilder_.getCount(); + return cellBuilder_.getCount(); } } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue getKeyValue(int index) { - if (keyValueBuilder_ == null) { - return keyValue_.get(index); + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell getCell(int index) { + if (cellBuilder_ == null) { + return cell_.get(index); } else { - return keyValueBuilder_.getMessage(index); + return cellBuilder_.getMessage(index); } } - public Builder setKeyValue( - int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue value) { - if (keyValueBuilder_ == null) { + public Builder setCell( + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell value) { + if (cellBuilder_ == null) { if (value == null) { throw new NullPointerException(); } - ensureKeyValueIsMutable(); - keyValue_.set(index, value); + ensureCellIsMutable(); + cell_.set(index, value); onChanged(); } else { - keyValueBuilder_.setMessage(index, value); + cellBuilder_.setMessage(index, value); } return this; } - public Builder setKeyValue( - int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder builderForValue) { - if (keyValueBuilder_ == null) { - ensureKeyValueIsMutable(); - keyValue_.set(index, builderForValue.build()); + public Builder setCell( + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.Builder builderForValue) { + if (cellBuilder_ == null) { + ensureCellIsMutable(); + cell_.set(index, builderForValue.build()); onChanged(); } else { - keyValueBuilder_.setMessage(index, builderForValue.build()); + cellBuilder_.setMessage(index, builderForValue.build()); } return this; } - public Builder addKeyValue(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue value) { - if (keyValueBuilder_ == null) { + public Builder addCell(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell value) { + if (cellBuilder_ == null) { if (value == null) { throw new NullPointerException(); } - ensureKeyValueIsMutable(); - keyValue_.add(value); + ensureCellIsMutable(); + cell_.add(value); onChanged(); } else { - keyValueBuilder_.addMessage(value); + cellBuilder_.addMessage(value); } return this; } - public Builder addKeyValue( - int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue value) { - if (keyValueBuilder_ == null) { + public Builder addCell( + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell value) { + if (cellBuilder_ == null) { if (value == null) { throw new NullPointerException(); } - ensureKeyValueIsMutable(); - keyValue_.add(index, value); + ensureCellIsMutable(); + cell_.add(index, value); onChanged(); } else { - keyValueBuilder_.addMessage(index, value); + cellBuilder_.addMessage(index, value); } return this; } - public Builder addKeyValue( - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder builderForValue) { - if (keyValueBuilder_ == null) { - ensureKeyValueIsMutable(); - keyValue_.add(builderForValue.build()); + public Builder addCell( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.Builder builderForValue) { + if (cellBuilder_ == null) { + ensureCellIsMutable(); + cell_.add(builderForValue.build()); onChanged(); } else { - keyValueBuilder_.addMessage(builderForValue.build()); + cellBuilder_.addMessage(builderForValue.build()); } return this; } - public Builder addKeyValue( - int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder builderForValue) { - if (keyValueBuilder_ == null) { - ensureKeyValueIsMutable(); - keyValue_.add(index, builderForValue.build()); + public Builder addCell( + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.Builder builderForValue) { + if (cellBuilder_ == null) { + ensureCellIsMutable(); + cell_.add(index, builderForValue.build()); onChanged(); } else { - keyValueBuilder_.addMessage(index, builderForValue.build()); + cellBuilder_.addMessage(index, builderForValue.build()); } return this; } - public Builder addAllKeyValue( - java.lang.Iterable values) { - if (keyValueBuilder_ == null) { - ensureKeyValueIsMutable(); - super.addAll(values, keyValue_); + public Builder addAllCell( + java.lang.Iterable values) { + if (cellBuilder_ == null) { + ensureCellIsMutable(); + super.addAll(values, cell_); onChanged(); } else { - keyValueBuilder_.addAllMessages(values); + cellBuilder_.addAllMessages(values); } return this; } - public Builder clearKeyValue() { - if (keyValueBuilder_ == null) { - keyValue_ = java.util.Collections.emptyList(); + public Builder clearCell() { + if (cellBuilder_ == null) { + cell_ = java.util.Collections.emptyList(); bitField0_ = (bitField0_ & ~0x00000001); onChanged(); } else { - keyValueBuilder_.clear(); + cellBuilder_.clear(); } return this; } - public Builder removeKeyValue(int index) { - if (keyValueBuilder_ == null) { - ensureKeyValueIsMutable(); - keyValue_.remove(index); + public Builder removeCell(int index) { + if (cellBuilder_ == null) { + ensureCellIsMutable(); + cell_.remove(index); onChanged(); } else { - keyValueBuilder_.remove(index); + cellBuilder_.remove(index); } return this; } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder getKeyValueBuilder( + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.Builder getCellBuilder( int index) { - return getKeyValueFieldBuilder().getBuilder(index); + return getCellFieldBuilder().getBuilder(index); } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValueOrBuilder getKeyValueOrBuilder( + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellOrBuilder getCellOrBuilder( int index) { - if (keyValueBuilder_ == null) { - return keyValue_.get(index); } else { - return keyValueBuilder_.getMessageOrBuilder(index); + if (cellBuilder_ == null) { + return cell_.get(index); } else { + return cellBuilder_.getMessageOrBuilder(index); } } - public java.util.List - getKeyValueOrBuilderList() { - if (keyValueBuilder_ != null) { - return keyValueBuilder_.getMessageOrBuilderList(); + public java.util.List + getCellOrBuilderList() { + if (cellBuilder_ != null) { + return cellBuilder_.getMessageOrBuilderList(); } else { - return java.util.Collections.unmodifiableList(keyValue_); + return java.util.Collections.unmodifiableList(cell_); } } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder addKeyValueBuilder() { - return getKeyValueFieldBuilder().addBuilder( - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.getDefaultInstance()); + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.Builder addCellBuilder() { + return getCellFieldBuilder().addBuilder( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.getDefaultInstance()); } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder addKeyValueBuilder( + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.Builder addCellBuilder( int index) { - return getKeyValueFieldBuilder().addBuilder( - index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.getDefaultInstance()); + return getCellFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.getDefaultInstance()); } - public java.util.List - getKeyValueBuilderList() { - return getKeyValueFieldBuilder().getBuilderList(); + public java.util.List + getCellBuilderList() { + return getCellFieldBuilder().getBuilderList(); } private com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValueOrBuilder> - getKeyValueFieldBuilder() { - if (keyValueBuilder_ == null) { - keyValueBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValueOrBuilder>( - keyValue_, + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellOrBuilder> + getCellFieldBuilder() { + if (cellBuilder_ == null) { + cellBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellOrBuilder>( + cell_, ((bitField0_ & 0x00000001) == 0x00000001), getParentForChildren(), isClean()); - keyValue_ = null; + cell_ = null; } - return keyValueBuilder_; + return cellBuilder_; + } + + // optional int32 associatedCellCount = 2; + private int associatedCellCount_ ; + public boolean hasAssociatedCellCount() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public int getAssociatedCellCount() { + return associatedCellCount_; + } + public Builder setAssociatedCellCount(int value) { + bitField0_ |= 0x00000002; + associatedCellCount_ = value; + onChanged(); + return this; + } + public Builder clearAssociatedCellCount() { + bitField0_ = (bitField0_ & ~0x00000002); + associatedCellCount_ = 0; + onChanged(); + return this; } // @@protoc_insertion_point(builder_scope:Result) @@ -4380,12 +4437,6 @@ public final class ClientProtos { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - if (hasResult()) { - if (!getResult().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } memoizedIsInitialized = 1; return true; } @@ -4668,12 +4719,6 @@ public final class ClientProtos { } public final boolean isInitialized() { - if (hasResult()) { - if (!getResult().isInitialized()) { - - return false; - } - } return true; } @@ -4932,12 +4977,6 @@ public final class ClientProtos { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - for (int i = 0; i < getResultCount(); i++) { - if (!getResult(i).isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } memoizedIsInitialized = 1; return true; } @@ -5246,12 +5285,6 @@ public final class ClientProtos { } public final boolean isInitialized() { - for (int i = 0; i < getResultCount(); i++) { - if (!getResult(i).isInitialized()) { - - return false; - } - } return true; } @@ -6333,28 +6366,32 @@ public final class ClientProtos { // @@protoc_insertion_point(class_scope:Condition) } - public interface MutateOrBuilder + public interface MutationProtoOrBuilder extends com.google.protobuf.MessageOrBuilder { - // required bytes row = 1; + // optional bytes row = 1; boolean hasRow(); com.google.protobuf.ByteString getRow(); - // required .Mutate.MutateType mutateType = 2; + // optional .MutationProto.MutationType mutateType = 2; boolean hasMutateType(); - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType getMutateType(); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType getMutateType(); - // repeated .Mutate.ColumnValue columnValue = 3; - java.util.List + // repeated .MutationProto.ColumnValue columnValue = 3; + java.util.List getColumnValueList(); - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue getColumnValue(int index); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue getColumnValue(int index); int getColumnValueCount(); - java.util.List + java.util.List getColumnValueOrBuilderList(); - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValueOrBuilder getColumnValueOrBuilder( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValueOrBuilder getColumnValueOrBuilder( int index); - // repeated .NameBytesPair attribute = 4; + // optional uint64 timestamp = 4; + boolean hasTimestamp(); + long getTimestamp(); + + // repeated .NameBytesPair attribute = 5; java.util.List getAttributeList(); org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getAttribute(int index); @@ -6364,48 +6401,48 @@ public final class ClientProtos { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getAttributeOrBuilder( int index); - // optional uint64 timestamp = 5; - boolean hasTimestamp(); - long getTimestamp(); - // optional bool writeToWAL = 6 [default = true]; boolean hasWriteToWAL(); boolean getWriteToWAL(); - // optional .TimeRange timeRange = 10; + // optional .TimeRange timeRange = 7; boolean hasTimeRange(); org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange getTimeRange(); org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRangeOrBuilder getTimeRangeOrBuilder(); + + // optional int32 associatedCellCount = 8; + boolean hasAssociatedCellCount(); + int getAssociatedCellCount(); } - public static final class Mutate extends + public static final class MutationProto extends com.google.protobuf.GeneratedMessage - implements MutateOrBuilder { - // Use Mutate.newBuilder() to construct. - private Mutate(Builder builder) { + implements MutationProtoOrBuilder { + // Use MutationProto.newBuilder() to construct. + private MutationProto(Builder builder) { super(builder); } - private Mutate(boolean noInit) {} + private MutationProto(boolean noInit) {} - private static final Mutate defaultInstance; - public static Mutate getDefaultInstance() { + private static final MutationProto defaultInstance; + public static MutationProto getDefaultInstance() { return defaultInstance; } - public Mutate getDefaultInstanceForType() { + public MutationProto getDefaultInstanceForType() { return defaultInstance; } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_Mutate_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MutationProto_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_Mutate_fieldAccessorTable; + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MutationProto_fieldAccessorTable; } - public enum MutateType + public enum MutationType implements com.google.protobuf.ProtocolMessageEnum { APPEND(0, 0), INCREMENT(1, 1), @@ -6421,7 +6458,7 @@ public final class ClientProtos { public final int getNumber() { return value; } - public static MutateType valueOf(int value) { + public static MutationType valueOf(int value) { switch (value) { case 0: return APPEND; case 1: return INCREMENT; @@ -6431,15 +6468,15 @@ public final class ClientProtos { } } - public static com.google.protobuf.Internal.EnumLiteMap + public static com.google.protobuf.Internal.EnumLiteMap internalGetValueMap() { return internalValueMap; } - private static com.google.protobuf.Internal.EnumLiteMap + private static com.google.protobuf.Internal.EnumLiteMap internalValueMap = - new com.google.protobuf.Internal.EnumLiteMap() { - public MutateType findValueByNumber(int number) { - return MutateType.valueOf(number); + new com.google.protobuf.Internal.EnumLiteMap() { + public MutationType findValueByNumber(int number) { + return MutationType.valueOf(number); } }; @@ -6453,14 +6490,14 @@ public final class ClientProtos { } public static final com.google.protobuf.Descriptors.EnumDescriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.getDescriptor().getEnumTypes().get(0); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDescriptor().getEnumTypes().get(0); } - private static final MutateType[] VALUES = { + private static final MutationType[] VALUES = { APPEND, INCREMENT, PUT, DELETE, }; - public static MutateType valueOf( + public static MutationType valueOf( com.google.protobuf.Descriptors.EnumValueDescriptor desc) { if (desc.getType() != getDescriptor()) { throw new java.lang.IllegalArgumentException( @@ -6472,12 +6509,12 @@ public final class ClientProtos { private final int index; private final int value; - private MutateType(int index, int value) { + private MutationType(int index, int value) { this.index = index; this.value = value; } - // @@protoc_insertion_point(enum_scope:Mutate.MutateType) + // @@protoc_insertion_point(enum_scope:MutationProto.MutationType) } public enum DeleteType @@ -6525,7 +6562,7 @@ public final class ClientProtos { } public static final com.google.protobuf.Descriptors.EnumDescriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.getDescriptor().getEnumTypes().get(1); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDescriptor().getEnumTypes().get(1); } private static final DeleteType[] VALUES = { @@ -6549,7 +6586,7 @@ public final class ClientProtos { this.value = value; } - // @@protoc_insertion_point(enum_scope:Mutate.DeleteType) + // @@protoc_insertion_point(enum_scope:MutationProto.DeleteType) } public interface ColumnValueOrBuilder @@ -6559,14 +6596,14 @@ public final class ClientProtos { boolean hasFamily(); com.google.protobuf.ByteString getFamily(); - // repeated .Mutate.ColumnValue.QualifierValue qualifierValue = 2; - java.util.List + // repeated .MutationProto.ColumnValue.QualifierValue qualifierValue = 2; + java.util.List getQualifierValueList(); - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue getQualifierValue(int index); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue getQualifierValue(int index); int getQualifierValueCount(); - java.util.List + java.util.List getQualifierValueOrBuilderList(); - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValueOrBuilder getQualifierValueOrBuilder( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValueOrBuilder getQualifierValueOrBuilder( int index); } public static final class ColumnValue extends @@ -6589,12 +6626,12 @@ public final class ClientProtos { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_Mutate_ColumnValue_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MutationProto_ColumnValue_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_Mutate_ColumnValue_fieldAccessorTable; + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MutationProto_ColumnValue_fieldAccessorTable; } public interface QualifierValueOrBuilder @@ -6612,9 +6649,9 @@ public final class ClientProtos { boolean hasTimestamp(); long getTimestamp(); - // optional .Mutate.DeleteType deleteType = 4; + // optional .MutationProto.DeleteType deleteType = 4; boolean hasDeleteType(); - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.DeleteType getDeleteType(); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType getDeleteType(); } public static final class QualifierValue extends com.google.protobuf.GeneratedMessage @@ -6636,12 +6673,12 @@ public final class ClientProtos { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_Mutate_ColumnValue_QualifierValue_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MutationProto_ColumnValue_QualifierValue_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_Mutate_ColumnValue_QualifierValue_fieldAccessorTable; + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MutationProto_ColumnValue_QualifierValue_fieldAccessorTable; } private int bitField0_; @@ -6675,13 +6712,13 @@ public final class ClientProtos { return timestamp_; } - // optional .Mutate.DeleteType deleteType = 4; + // optional .MutationProto.DeleteType deleteType = 4; public static final int DELETETYPE_FIELD_NUMBER = 4; - private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.DeleteType deleteType_; + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType deleteType_; public boolean hasDeleteType() { return ((bitField0_ & 0x00000008) == 0x00000008); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.DeleteType getDeleteType() { + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType getDeleteType() { return deleteType_; } @@ -6689,7 +6726,7 @@ public final class ClientProtos { qualifier_ = com.google.protobuf.ByteString.EMPTY; value_ = com.google.protobuf.ByteString.EMPTY; timestamp_ = 0L; - deleteType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.DeleteType.DELETE_ONE_VERSION; + deleteType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType.DELETE_ONE_VERSION; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -6757,10 +6794,10 @@ public final class ClientProtos { if (obj == this) { return true; } - if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue)) { + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue)) { return super.equals(obj); } - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue) obj; + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue) obj; boolean result = true; result = result && (hasQualifier() == other.hasQualifier()); @@ -6812,41 +6849,41 @@ public final class ClientProtos { return hash; } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue parseFrom(byte[] data) + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue parseFrom(java.io.InputStream input) + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue parseFrom(java.io.InputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { return newBuilder().mergeFrom(input, extensionRegistry) .buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue parseDelimitedFrom(java.io.InputStream input) + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -6855,7 +6892,7 @@ public final class ClientProtos { return null; } } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue parseDelimitedFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -6866,12 +6903,12 @@ public final class ClientProtos { return null; } } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -6881,7 +6918,7 @@ public final class ClientProtos { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue prototype) { + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } @@ -6894,18 +6931,18 @@ public final class ClientProtos { } public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder - implements org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValueOrBuilder { + implements org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValueOrBuilder { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_Mutate_ColumnValue_QualifierValue_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MutationProto_ColumnValue_QualifierValue_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_Mutate_ColumnValue_QualifierValue_fieldAccessorTable; + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MutationProto_ColumnValue_QualifierValue_fieldAccessorTable; } - // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue.newBuilder() + // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -6930,7 +6967,7 @@ public final class ClientProtos { bitField0_ = (bitField0_ & ~0x00000002); timestamp_ = 0L; bitField0_ = (bitField0_ & ~0x00000004); - deleteType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.DeleteType.DELETE_ONE_VERSION; + deleteType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType.DELETE_ONE_VERSION; bitField0_ = (bitField0_ & ~0x00000008); return this; } @@ -6941,24 +6978,24 @@ public final class ClientProtos { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue.getDescriptor(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue.getDescriptor(); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue getDefaultInstanceForType() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue.getDefaultInstance(); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue.getDefaultInstance(); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue build() { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue result = buildPartial(); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue build() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } return result; } - private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue buildParsed() + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue result = buildPartial(); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); @@ -6966,8 +7003,8 @@ public final class ClientProtos { return result; } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue buildPartial() { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue(this); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue(this); int from_bitField0_ = bitField0_; int to_bitField0_ = 0; if (((from_bitField0_ & 0x00000001) == 0x00000001)) { @@ -6992,16 +7029,16 @@ public final class ClientProtos { } public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue) { - return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue)other); + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue)other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue other) { - if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue.getDefaultInstance()) return this; + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue.getDefaultInstance()) return this; if (other.hasQualifier()) { setQualifier(other.getQualifier()); } @@ -7062,7 +7099,7 @@ public final class ClientProtos { } case 32: { int rawValue = input.readEnum(); - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.DeleteType value = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.DeleteType.valueOf(rawValue); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType value = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType.valueOf(rawValue); if (value == null) { unknownFields.mergeVarintField(4, rawValue); } else { @@ -7146,15 +7183,15 @@ public final class ClientProtos { return this; } - // optional .Mutate.DeleteType deleteType = 4; - private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.DeleteType deleteType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.DeleteType.DELETE_ONE_VERSION; + // optional .MutationProto.DeleteType deleteType = 4; + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType deleteType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType.DELETE_ONE_VERSION; public boolean hasDeleteType() { return ((bitField0_ & 0x00000008) == 0x00000008); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.DeleteType getDeleteType() { + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType getDeleteType() { return deleteType_; } - public Builder setDeleteType(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.DeleteType value) { + public Builder setDeleteType(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType value) { if (value == null) { throw new NullPointerException(); } @@ -7165,12 +7202,12 @@ public final class ClientProtos { } public Builder clearDeleteType() { bitField0_ = (bitField0_ & ~0x00000008); - deleteType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.DeleteType.DELETE_ONE_VERSION; + deleteType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType.DELETE_ONE_VERSION; onChanged(); return this; } - // @@protoc_insertion_point(builder_scope:Mutate.ColumnValue.QualifierValue) + // @@protoc_insertion_point(builder_scope:MutationProto.ColumnValue.QualifierValue) } static { @@ -7178,7 +7215,7 @@ public final class ClientProtos { defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:Mutate.ColumnValue.QualifierValue) + // @@protoc_insertion_point(class_scope:MutationProto.ColumnValue.QualifierValue) } private int bitField0_; @@ -7192,23 +7229,23 @@ public final class ClientProtos { return family_; } - // repeated .Mutate.ColumnValue.QualifierValue qualifierValue = 2; + // repeated .MutationProto.ColumnValue.QualifierValue qualifierValue = 2; public static final int QUALIFIERVALUE_FIELD_NUMBER = 2; - private java.util.List qualifierValue_; - public java.util.List getQualifierValueList() { + private java.util.List qualifierValue_; + public java.util.List getQualifierValueList() { return qualifierValue_; } - public java.util.List + public java.util.List getQualifierValueOrBuilderList() { return qualifierValue_; } public int getQualifierValueCount() { return qualifierValue_.size(); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue getQualifierValue(int index) { + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue getQualifierValue(int index) { return qualifierValue_.get(index); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValueOrBuilder getQualifierValueOrBuilder( + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValueOrBuilder getQualifierValueOrBuilder( int index) { return qualifierValue_.get(index); } @@ -7273,10 +7310,10 @@ public final class ClientProtos { if (obj == this) { return true; } - if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue)) { + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue)) { return super.equals(obj); } - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue) obj; + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue) obj; boolean result = true; result = result && (hasFamily() == other.hasFamily()); @@ -7307,41 +7344,41 @@ public final class ClientProtos { return hash; } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue parseFrom(byte[] data) + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue parseFrom(java.io.InputStream input) + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue parseFrom(java.io.InputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { return newBuilder().mergeFrom(input, extensionRegistry) .buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue parseDelimitedFrom(java.io.InputStream input) + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -7350,7 +7387,7 @@ public final class ClientProtos { return null; } } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue parseDelimitedFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -7361,12 +7398,12 @@ public final class ClientProtos { return null; } } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -7376,7 +7413,7 @@ public final class ClientProtos { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue prototype) { + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } @@ -7389,18 +7426,18 @@ public final class ClientProtos { } public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder - implements org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValueOrBuilder { + implements org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValueOrBuilder { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_Mutate_ColumnValue_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MutationProto_ColumnValue_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_Mutate_ColumnValue_fieldAccessorTable; + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MutationProto_ColumnValue_fieldAccessorTable; } - // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.newBuilder() + // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -7437,24 +7474,24 @@ public final class ClientProtos { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.getDescriptor(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.getDescriptor(); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue getDefaultInstanceForType() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.getDefaultInstance(); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.getDefaultInstance(); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue build() { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue result = buildPartial(); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue build() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } return result; } - private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue buildParsed() + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue result = buildPartial(); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); @@ -7462,8 +7499,8 @@ public final class ClientProtos { return result; } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue buildPartial() { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue(this); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue(this); int from_bitField0_ = bitField0_; int to_bitField0_ = 0; if (((from_bitField0_ & 0x00000001) == 0x00000001)) { @@ -7485,16 +7522,16 @@ public final class ClientProtos { } public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue) { - return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue)other); + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue)other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue other) { - if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.getDefaultInstance()) return this; + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.getDefaultInstance()) return this; if (other.hasFamily()) { setFamily(other.getFamily()); } @@ -7565,7 +7602,7 @@ public final class ClientProtos { break; } case 18: { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue.newBuilder(); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue.newBuilder(); input.readMessage(subBuilder, extensionRegistry); addQualifierValue(subBuilder.buildPartial()); break; @@ -7600,20 +7637,20 @@ public final class ClientProtos { return this; } - // repeated .Mutate.ColumnValue.QualifierValue qualifierValue = 2; - private java.util.List qualifierValue_ = + // repeated .MutationProto.ColumnValue.QualifierValue qualifierValue = 2; + private java.util.List qualifierValue_ = java.util.Collections.emptyList(); private void ensureQualifierValueIsMutable() { if (!((bitField0_ & 0x00000002) == 0x00000002)) { - qualifierValue_ = new java.util.ArrayList(qualifierValue_); + qualifierValue_ = new java.util.ArrayList(qualifierValue_); bitField0_ |= 0x00000002; } } private com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValueOrBuilder> qualifierValueBuilder_; + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValueOrBuilder> qualifierValueBuilder_; - public java.util.List getQualifierValueList() { + public java.util.List getQualifierValueList() { if (qualifierValueBuilder_ == null) { return java.util.Collections.unmodifiableList(qualifierValue_); } else { @@ -7627,7 +7664,7 @@ public final class ClientProtos { return qualifierValueBuilder_.getCount(); } } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue getQualifierValue(int index) { + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue getQualifierValue(int index) { if (qualifierValueBuilder_ == null) { return qualifierValue_.get(index); } else { @@ -7635,7 +7672,7 @@ public final class ClientProtos { } } public Builder setQualifierValue( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue value) { + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue value) { if (qualifierValueBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -7649,7 +7686,7 @@ public final class ClientProtos { return this; } public Builder setQualifierValue( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue.Builder builderForValue) { + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue.Builder builderForValue) { if (qualifierValueBuilder_ == null) { ensureQualifierValueIsMutable(); qualifierValue_.set(index, builderForValue.build()); @@ -7659,7 +7696,7 @@ public final class ClientProtos { } return this; } - public Builder addQualifierValue(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue value) { + public Builder addQualifierValue(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue value) { if (qualifierValueBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -7673,7 +7710,7 @@ public final class ClientProtos { return this; } public Builder addQualifierValue( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue value) { + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue value) { if (qualifierValueBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -7687,7 +7724,7 @@ public final class ClientProtos { return this; } public Builder addQualifierValue( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue.Builder builderForValue) { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue.Builder builderForValue) { if (qualifierValueBuilder_ == null) { ensureQualifierValueIsMutable(); qualifierValue_.add(builderForValue.build()); @@ -7698,7 +7735,7 @@ public final class ClientProtos { return this; } public Builder addQualifierValue( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue.Builder builderForValue) { + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue.Builder builderForValue) { if (qualifierValueBuilder_ == null) { ensureQualifierValueIsMutable(); qualifierValue_.add(index, builderForValue.build()); @@ -7709,7 +7746,7 @@ public final class ClientProtos { return this; } public Builder addAllQualifierValue( - java.lang.Iterable values) { + java.lang.Iterable values) { if (qualifierValueBuilder_ == null) { ensureQualifierValueIsMutable(); super.addAll(values, qualifierValue_); @@ -7739,18 +7776,18 @@ public final class ClientProtos { } return this; } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue.Builder getQualifierValueBuilder( + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue.Builder getQualifierValueBuilder( int index) { return getQualifierValueFieldBuilder().getBuilder(index); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValueOrBuilder getQualifierValueOrBuilder( + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValueOrBuilder getQualifierValueOrBuilder( int index) { if (qualifierValueBuilder_ == null) { return qualifierValue_.get(index); } else { return qualifierValueBuilder_.getMessageOrBuilder(index); } } - public java.util.List + public java.util.List getQualifierValueOrBuilderList() { if (qualifierValueBuilder_ != null) { return qualifierValueBuilder_.getMessageOrBuilderList(); @@ -7758,25 +7795,25 @@ public final class ClientProtos { return java.util.Collections.unmodifiableList(qualifierValue_); } } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue.Builder addQualifierValueBuilder() { + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue.Builder addQualifierValueBuilder() { return getQualifierValueFieldBuilder().addBuilder( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue.getDefaultInstance()); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue.getDefaultInstance()); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue.Builder addQualifierValueBuilder( + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue.Builder addQualifierValueBuilder( int index) { return getQualifierValueFieldBuilder().addBuilder( - index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue.getDefaultInstance()); + index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue.getDefaultInstance()); } - public java.util.List + public java.util.List getQualifierValueBuilderList() { return getQualifierValueFieldBuilder().getBuilderList(); } private com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValueOrBuilder> + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValueOrBuilder> getQualifierValueFieldBuilder() { if (qualifierValueBuilder_ == null) { qualifierValueBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValueOrBuilder>( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValueOrBuilder>( qualifierValue_, ((bitField0_ & 0x00000002) == 0x00000002), getParentForChildren(), @@ -7786,7 +7823,7 @@ public final class ClientProtos { return qualifierValueBuilder_; } - // @@protoc_insertion_point(builder_scope:Mutate.ColumnValue) + // @@protoc_insertion_point(builder_scope:MutationProto.ColumnValue) } static { @@ -7794,11 +7831,11 @@ public final class ClientProtos { defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:Mutate.ColumnValue) + // @@protoc_insertion_point(class_scope:MutationProto.ColumnValue) } private int bitField0_; - // required bytes row = 1; + // optional bytes row = 1; public static final int ROW_FIELD_NUMBER = 1; private com.google.protobuf.ByteString row_; public boolean hasRow() { @@ -7808,39 +7845,49 @@ public final class ClientProtos { return row_; } - // required .Mutate.MutateType mutateType = 2; + // optional .MutationProto.MutationType mutateType = 2; public static final int MUTATETYPE_FIELD_NUMBER = 2; - private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType mutateType_; + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType mutateType_; public boolean hasMutateType() { return ((bitField0_ & 0x00000002) == 0x00000002); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType getMutateType() { + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType getMutateType() { return mutateType_; } - // repeated .Mutate.ColumnValue columnValue = 3; + // repeated .MutationProto.ColumnValue columnValue = 3; public static final int COLUMNVALUE_FIELD_NUMBER = 3; - private java.util.List columnValue_; - public java.util.List getColumnValueList() { + private java.util.List columnValue_; + public java.util.List getColumnValueList() { return columnValue_; } - public java.util.List + public java.util.List getColumnValueOrBuilderList() { return columnValue_; } public int getColumnValueCount() { return columnValue_.size(); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue getColumnValue(int index) { + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue getColumnValue(int index) { return columnValue_.get(index); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValueOrBuilder getColumnValueOrBuilder( + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValueOrBuilder getColumnValueOrBuilder( int index) { return columnValue_.get(index); } - // repeated .NameBytesPair attribute = 4; - public static final int ATTRIBUTE_FIELD_NUMBER = 4; + // optional uint64 timestamp = 4; + public static final int TIMESTAMP_FIELD_NUMBER = 4; + private long timestamp_; + public boolean hasTimestamp() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTimestamp() { + return timestamp_; + } + + // repeated .NameBytesPair attribute = 5; + public static final int ATTRIBUTE_FIELD_NUMBER = 5; private java.util.List attribute_; public java.util.List getAttributeList() { return attribute_; @@ -7860,16 +7907,6 @@ public final class ClientProtos { return attribute_.get(index); } - // optional uint64 timestamp = 5; - public static final int TIMESTAMP_FIELD_NUMBER = 5; - private long timestamp_; - public boolean hasTimestamp() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTimestamp() { - return timestamp_; - } - // optional bool writeToWAL = 6 [default = true]; public static final int WRITETOWAL_FIELD_NUMBER = 6; private boolean writeToWAL_; @@ -7880,8 +7917,8 @@ public final class ClientProtos { return writeToWAL_; } - // optional .TimeRange timeRange = 10; - public static final int TIMERANGE_FIELD_NUMBER = 10; + // optional .TimeRange timeRange = 7; + public static final int TIMERANGE_FIELD_NUMBER = 7; private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange timeRange_; public boolean hasTimeRange() { return ((bitField0_ & 0x00000010) == 0x00000010); @@ -7893,28 +7930,31 @@ public final class ClientProtos { return timeRange_; } + // optional int32 associatedCellCount = 8; + public static final int ASSOCIATEDCELLCOUNT_FIELD_NUMBER = 8; + private int associatedCellCount_; + public boolean hasAssociatedCellCount() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public int getAssociatedCellCount() { + return associatedCellCount_; + } + private void initFields() { row_ = com.google.protobuf.ByteString.EMPTY; - mutateType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType.APPEND; + mutateType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType.APPEND; columnValue_ = java.util.Collections.emptyList(); - attribute_ = java.util.Collections.emptyList(); timestamp_ = 0L; + attribute_ = java.util.Collections.emptyList(); writeToWAL_ = true; timeRange_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.getDefaultInstance(); + associatedCellCount_ = 0; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - if (!hasRow()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasMutateType()) { - memoizedIsInitialized = 0; - return false; - } for (int i = 0; i < getColumnValueCount(); i++) { if (!getColumnValue(i).isInitialized()) { memoizedIsInitialized = 0; @@ -7943,17 +7983,20 @@ public final class ClientProtos { for (int i = 0; i < columnValue_.size(); i++) { output.writeMessage(3, columnValue_.get(i)); } - for (int i = 0; i < attribute_.size(); i++) { - output.writeMessage(4, attribute_.get(i)); - } if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeUInt64(5, timestamp_); + output.writeUInt64(4, timestamp_); + } + for (int i = 0; i < attribute_.size(); i++) { + output.writeMessage(5, attribute_.get(i)); } if (((bitField0_ & 0x00000008) == 0x00000008)) { output.writeBool(6, writeToWAL_); } if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeMessage(10, timeRange_); + output.writeMessage(7, timeRange_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeInt32(8, associatedCellCount_); } getUnknownFields().writeTo(output); } @@ -7976,13 +8019,13 @@ public final class ClientProtos { size += com.google.protobuf.CodedOutputStream .computeMessageSize(3, columnValue_.get(i)); } - for (int i = 0; i < attribute_.size(); i++) { + if (((bitField0_ & 0x00000004) == 0x00000004)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(4, attribute_.get(i)); + .computeUInt64Size(4, timestamp_); } - if (((bitField0_ & 0x00000004) == 0x00000004)) { + for (int i = 0; i < attribute_.size(); i++) { size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(5, timestamp_); + .computeMessageSize(5, attribute_.get(i)); } if (((bitField0_ & 0x00000008) == 0x00000008)) { size += com.google.protobuf.CodedOutputStream @@ -7990,7 +8033,11 @@ public final class ClientProtos { } if (((bitField0_ & 0x00000010) == 0x00000010)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(10, timeRange_); + .computeMessageSize(7, timeRange_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(8, associatedCellCount_); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; @@ -8009,10 +8056,10 @@ public final class ClientProtos { if (obj == this) { return true; } - if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate)) { + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto)) { return super.equals(obj); } - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate) obj; + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto) obj; boolean result = true; result = result && (hasRow() == other.hasRow()); @@ -8027,13 +8074,13 @@ public final class ClientProtos { } result = result && getColumnValueList() .equals(other.getColumnValueList()); - result = result && getAttributeList() - .equals(other.getAttributeList()); result = result && (hasTimestamp() == other.hasTimestamp()); if (hasTimestamp()) { result = result && (getTimestamp() == other.getTimestamp()); } + result = result && getAttributeList() + .equals(other.getAttributeList()); result = result && (hasWriteToWAL() == other.hasWriteToWAL()); if (hasWriteToWAL()) { result = result && (getWriteToWAL() @@ -8044,6 +8091,11 @@ public final class ClientProtos { result = result && getTimeRange() .equals(other.getTimeRange()); } + result = result && (hasAssociatedCellCount() == other.hasAssociatedCellCount()); + if (hasAssociatedCellCount()) { + result = result && (getAssociatedCellCount() + == other.getAssociatedCellCount()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -8065,14 +8117,14 @@ public final class ClientProtos { hash = (37 * hash) + COLUMNVALUE_FIELD_NUMBER; hash = (53 * hash) + getColumnValueList().hashCode(); } - if (getAttributeCount() > 0) { - hash = (37 * hash) + ATTRIBUTE_FIELD_NUMBER; - hash = (53 * hash) + getAttributeList().hashCode(); - } if (hasTimestamp()) { hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER; hash = (53 * hash) + hashLong(getTimestamp()); } + if (getAttributeCount() > 0) { + hash = (37 * hash) + ATTRIBUTE_FIELD_NUMBER; + hash = (53 * hash) + getAttributeList().hashCode(); + } if (hasWriteToWAL()) { hash = (37 * hash) + WRITETOWAL_FIELD_NUMBER; hash = (53 * hash) + hashBoolean(getWriteToWAL()); @@ -8081,45 +8133,49 @@ public final class ClientProtos { hash = (37 * hash) + TIMERANGE_FIELD_NUMBER; hash = (53 * hash) + getTimeRange().hashCode(); } + if (hasAssociatedCellCount()) { + hash = (37 * hash) + ASSOCIATEDCELLCOUNT_FIELD_NUMBER; + hash = (53 * hash) + getAssociatedCellCount(); + } hash = (29 * hash) + getUnknownFields().hashCode(); return hash; } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate parseFrom(byte[] data) + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate parseFrom(java.io.InputStream input) + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto parseFrom(java.io.InputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { return newBuilder().mergeFrom(input, extensionRegistry) .buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate parseDelimitedFrom(java.io.InputStream input) + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -8128,7 +8184,7 @@ public final class ClientProtos { return null; } } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate parseDelimitedFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -8139,12 +8195,12 @@ public final class ClientProtos { return null; } } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -8154,7 +8210,7 @@ public final class ClientProtos { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate prototype) { + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } @@ -8167,18 +8223,18 @@ public final class ClientProtos { } public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder - implements org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateOrBuilder { + implements org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_Mutate_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MutationProto_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_Mutate_fieldAccessorTable; + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MutationProto_fieldAccessorTable; } - // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.newBuilder() + // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -8202,7 +8258,7 @@ public final class ClientProtos { super.clear(); row_ = com.google.protobuf.ByteString.EMPTY; bitField0_ = (bitField0_ & ~0x00000001); - mutateType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType.APPEND; + mutateType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType.APPEND; bitField0_ = (bitField0_ & ~0x00000002); if (columnValueBuilder_ == null) { columnValue_ = java.util.Collections.emptyList(); @@ -8210,14 +8266,14 @@ public final class ClientProtos { } else { columnValueBuilder_.clear(); } + timestamp_ = 0L; + bitField0_ = (bitField0_ & ~0x00000008); if (attributeBuilder_ == null) { attribute_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000008); + bitField0_ = (bitField0_ & ~0x00000010); } else { attributeBuilder_.clear(); } - timestamp_ = 0L; - bitField0_ = (bitField0_ & ~0x00000010); writeToWAL_ = true; bitField0_ = (bitField0_ & ~0x00000020); if (timeRangeBuilder_ == null) { @@ -8226,6 +8282,8 @@ public final class ClientProtos { timeRangeBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000040); + associatedCellCount_ = 0; + bitField0_ = (bitField0_ & ~0x00000080); return this; } @@ -8235,24 +8293,24 @@ public final class ClientProtos { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.getDescriptor(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDescriptor(); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate getDefaultInstanceForType() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.getDefaultInstance(); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance(); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate build() { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate result = buildPartial(); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto build() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } return result; } - private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate buildParsed() + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate result = buildPartial(); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); @@ -8260,8 +8318,8 @@ public final class ClientProtos { return result; } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate buildPartial() { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate(this); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto(this); int from_bitField0_ = bitField0_; int to_bitField0_ = 0; if (((from_bitField0_ & 0x00000001) == 0x00000001)) { @@ -8281,19 +8339,19 @@ public final class ClientProtos { } else { result.columnValue_ = columnValueBuilder_.build(); } + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000004; + } + result.timestamp_ = timestamp_; if (attributeBuilder_ == null) { - if (((bitField0_ & 0x00000008) == 0x00000008)) { + if (((bitField0_ & 0x00000010) == 0x00000010)) { attribute_ = java.util.Collections.unmodifiableList(attribute_); - bitField0_ = (bitField0_ & ~0x00000008); + bitField0_ = (bitField0_ & ~0x00000010); } result.attribute_ = attribute_; } else { result.attribute_ = attributeBuilder_.build(); } - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000004; - } - result.timestamp_ = timestamp_; if (((from_bitField0_ & 0x00000020) == 0x00000020)) { to_bitField0_ |= 0x00000008; } @@ -8306,22 +8364,26 @@ public final class ClientProtos { } else { result.timeRange_ = timeRangeBuilder_.build(); } + if (((from_bitField0_ & 0x00000080) == 0x00000080)) { + to_bitField0_ |= 0x00000020; + } + result.associatedCellCount_ = associatedCellCount_; result.bitField0_ = to_bitField0_; onBuilt(); return result; } public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate) { - return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate)other); + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto)other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate other) { - if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.getDefaultInstance()) return this; + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance()) return this; if (other.hasRow()) { setRow(other.getRow()); } @@ -8354,11 +8416,14 @@ public final class ClientProtos { } } } + if (other.hasTimestamp()) { + setTimestamp(other.getTimestamp()); + } if (attributeBuilder_ == null) { if (!other.attribute_.isEmpty()) { if (attribute_.isEmpty()) { attribute_ = other.attribute_; - bitField0_ = (bitField0_ & ~0x00000008); + bitField0_ = (bitField0_ & ~0x00000010); } else { ensureAttributeIsMutable(); attribute_.addAll(other.attribute_); @@ -8371,7 +8436,7 @@ public final class ClientProtos { attributeBuilder_.dispose(); attributeBuilder_ = null; attribute_ = other.attribute_; - bitField0_ = (bitField0_ & ~0x00000008); + bitField0_ = (bitField0_ & ~0x00000010); attributeBuilder_ = com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? getAttributeFieldBuilder() : null; @@ -8380,28 +8445,20 @@ public final class ClientProtos { } } } - if (other.hasTimestamp()) { - setTimestamp(other.getTimestamp()); - } if (other.hasWriteToWAL()) { setWriteToWAL(other.getWriteToWAL()); } if (other.hasTimeRange()) { mergeTimeRange(other.getTimeRange()); } + if (other.hasAssociatedCellCount()) { + setAssociatedCellCount(other.getAssociatedCellCount()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } public final boolean isInitialized() { - if (!hasRow()) { - - return false; - } - if (!hasMutateType()) { - - return false; - } for (int i = 0; i < getColumnValueCount(); i++) { if (!getColumnValue(i).isInitialized()) { @@ -8447,7 +8504,7 @@ public final class ClientProtos { } case 16: { int rawValue = input.readEnum(); - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType value = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType.valueOf(rawValue); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType value = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType.valueOf(rawValue); if (value == null) { unknownFields.mergeVarintField(2, rawValue); } else { @@ -8457,28 +8514,28 @@ public final class ClientProtos { break; } case 26: { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.newBuilder(); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.newBuilder(); input.readMessage(subBuilder, extensionRegistry); addColumnValue(subBuilder.buildPartial()); break; } - case 34: { + case 32: { + bitField0_ |= 0x00000008; + timestamp_ = input.readUInt64(); + break; + } + case 42: { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.newBuilder(); input.readMessage(subBuilder, extensionRegistry); addAttribute(subBuilder.buildPartial()); break; } - case 40: { - bitField0_ |= 0x00000010; - timestamp_ = input.readUInt64(); - break; - } case 48: { bitField0_ |= 0x00000020; writeToWAL_ = input.readBool(); break; } - case 82: { + case 58: { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.newBuilder(); if (hasTimeRange()) { subBuilder.mergeFrom(getTimeRange()); @@ -8487,13 +8544,18 @@ public final class ClientProtos { setTimeRange(subBuilder.buildPartial()); break; } + case 64: { + bitField0_ |= 0x00000080; + associatedCellCount_ = input.readInt32(); + break; + } } } } private int bitField0_; - // required bytes row = 1; + // optional bytes row = 1; private com.google.protobuf.ByteString row_ = com.google.protobuf.ByteString.EMPTY; public boolean hasRow() { return ((bitField0_ & 0x00000001) == 0x00000001); @@ -8517,15 +8579,15 @@ public final class ClientProtos { return this; } - // required .Mutate.MutateType mutateType = 2; - private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType mutateType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType.APPEND; + // optional .MutationProto.MutationType mutateType = 2; + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType mutateType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType.APPEND; public boolean hasMutateType() { return ((bitField0_ & 0x00000002) == 0x00000002); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType getMutateType() { + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType getMutateType() { return mutateType_; } - public Builder setMutateType(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType value) { + public Builder setMutateType(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType value) { if (value == null) { throw new NullPointerException(); } @@ -8536,25 +8598,25 @@ public final class ClientProtos { } public Builder clearMutateType() { bitField0_ = (bitField0_ & ~0x00000002); - mutateType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType.APPEND; + mutateType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType.APPEND; onChanged(); return this; } - // repeated .Mutate.ColumnValue columnValue = 3; - private java.util.List columnValue_ = + // repeated .MutationProto.ColumnValue columnValue = 3; + private java.util.List columnValue_ = java.util.Collections.emptyList(); private void ensureColumnValueIsMutable() { if (!((bitField0_ & 0x00000004) == 0x00000004)) { - columnValue_ = new java.util.ArrayList(columnValue_); + columnValue_ = new java.util.ArrayList(columnValue_); bitField0_ |= 0x00000004; } } private com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValueOrBuilder> columnValueBuilder_; + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValueOrBuilder> columnValueBuilder_; - public java.util.List getColumnValueList() { + public java.util.List getColumnValueList() { if (columnValueBuilder_ == null) { return java.util.Collections.unmodifiableList(columnValue_); } else { @@ -8568,7 +8630,7 @@ public final class ClientProtos { return columnValueBuilder_.getCount(); } } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue getColumnValue(int index) { + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue getColumnValue(int index) { if (columnValueBuilder_ == null) { return columnValue_.get(index); } else { @@ -8576,7 +8638,7 @@ public final class ClientProtos { } } public Builder setColumnValue( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue value) { + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue value) { if (columnValueBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -8590,7 +8652,7 @@ public final class ClientProtos { return this; } public Builder setColumnValue( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.Builder builderForValue) { + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.Builder builderForValue) { if (columnValueBuilder_ == null) { ensureColumnValueIsMutable(); columnValue_.set(index, builderForValue.build()); @@ -8600,7 +8662,7 @@ public final class ClientProtos { } return this; } - public Builder addColumnValue(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue value) { + public Builder addColumnValue(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue value) { if (columnValueBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -8614,7 +8676,7 @@ public final class ClientProtos { return this; } public Builder addColumnValue( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue value) { + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue value) { if (columnValueBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -8628,7 +8690,7 @@ public final class ClientProtos { return this; } public Builder addColumnValue( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.Builder builderForValue) { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.Builder builderForValue) { if (columnValueBuilder_ == null) { ensureColumnValueIsMutable(); columnValue_.add(builderForValue.build()); @@ -8639,7 +8701,7 @@ public final class ClientProtos { return this; } public Builder addColumnValue( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.Builder builderForValue) { + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.Builder builderForValue) { if (columnValueBuilder_ == null) { ensureColumnValueIsMutable(); columnValue_.add(index, builderForValue.build()); @@ -8650,7 +8712,7 @@ public final class ClientProtos { return this; } public Builder addAllColumnValue( - java.lang.Iterable values) { + java.lang.Iterable values) { if (columnValueBuilder_ == null) { ensureColumnValueIsMutable(); super.addAll(values, columnValue_); @@ -8680,18 +8742,18 @@ public final class ClientProtos { } return this; } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.Builder getColumnValueBuilder( + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.Builder getColumnValueBuilder( int index) { return getColumnValueFieldBuilder().getBuilder(index); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValueOrBuilder getColumnValueOrBuilder( + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValueOrBuilder getColumnValueOrBuilder( int index) { if (columnValueBuilder_ == null) { return columnValue_.get(index); } else { return columnValueBuilder_.getMessageOrBuilder(index); } } - public java.util.List + public java.util.List getColumnValueOrBuilderList() { if (columnValueBuilder_ != null) { return columnValueBuilder_.getMessageOrBuilderList(); @@ -8699,25 +8761,25 @@ public final class ClientProtos { return java.util.Collections.unmodifiableList(columnValue_); } } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.Builder addColumnValueBuilder() { + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.Builder addColumnValueBuilder() { return getColumnValueFieldBuilder().addBuilder( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.getDefaultInstance()); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.getDefaultInstance()); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.Builder addColumnValueBuilder( + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.Builder addColumnValueBuilder( int index) { return getColumnValueFieldBuilder().addBuilder( - index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.getDefaultInstance()); + index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.getDefaultInstance()); } - public java.util.List + public java.util.List getColumnValueBuilderList() { return getColumnValueFieldBuilder().getBuilderList(); } private com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValueOrBuilder> + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValueOrBuilder> getColumnValueFieldBuilder() { if (columnValueBuilder_ == null) { columnValueBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValueOrBuilder>( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValueOrBuilder>( columnValue_, ((bitField0_ & 0x00000004) == 0x00000004), getParentForChildren(), @@ -8727,13 +8789,34 @@ public final class ClientProtos { return columnValueBuilder_; } - // repeated .NameBytesPair attribute = 4; + // optional uint64 timestamp = 4; + private long timestamp_ ; + public boolean hasTimestamp() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public long getTimestamp() { + return timestamp_; + } + public Builder setTimestamp(long value) { + bitField0_ |= 0x00000008; + timestamp_ = value; + onChanged(); + return this; + } + public Builder clearTimestamp() { + bitField0_ = (bitField0_ & ~0x00000008); + timestamp_ = 0L; + onChanged(); + return this; + } + + // repeated .NameBytesPair attribute = 5; private java.util.List attribute_ = java.util.Collections.emptyList(); private void ensureAttributeIsMutable() { - if (!((bitField0_ & 0x00000008) == 0x00000008)) { + if (!((bitField0_ & 0x00000010) == 0x00000010)) { attribute_ = new java.util.ArrayList(attribute_); - bitField0_ |= 0x00000008; + bitField0_ |= 0x00000010; } } @@ -8849,7 +8932,7 @@ public final class ClientProtos { public Builder clearAttribute() { if (attributeBuilder_ == null) { attribute_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000008); + bitField0_ = (bitField0_ & ~0x00000010); onChanged(); } else { attributeBuilder_.clear(); @@ -8905,7 +8988,7 @@ public final class ClientProtos { attributeBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder>( attribute_, - ((bitField0_ & 0x00000008) == 0x00000008), + ((bitField0_ & 0x00000010) == 0x00000010), getParentForChildren(), isClean()); attribute_ = null; @@ -8913,27 +8996,6 @@ public final class ClientProtos { return attributeBuilder_; } - // optional uint64 timestamp = 5; - private long timestamp_ ; - public boolean hasTimestamp() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public long getTimestamp() { - return timestamp_; - } - public Builder setTimestamp(long value) { - bitField0_ |= 0x00000010; - timestamp_ = value; - onChanged(); - return this; - } - public Builder clearTimestamp() { - bitField0_ = (bitField0_ & ~0x00000010); - timestamp_ = 0L; - onChanged(); - return this; - } - // optional bool writeToWAL = 6 [default = true]; private boolean writeToWAL_ = true; public boolean hasWriteToWAL() { @@ -8955,7 +9017,7 @@ public final class ClientProtos { return this; } - // optional .TimeRange timeRange = 10; + // optional .TimeRange timeRange = 7; private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange timeRange_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.getDefaultInstance(); private com.google.protobuf.SingleFieldBuilder< org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRangeOrBuilder> timeRangeBuilder_; @@ -9045,15 +9107,36 @@ public final class ClientProtos { return timeRangeBuilder_; } - // @@protoc_insertion_point(builder_scope:Mutate) + // optional int32 associatedCellCount = 8; + private int associatedCellCount_ ; + public boolean hasAssociatedCellCount() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + public int getAssociatedCellCount() { + return associatedCellCount_; + } + public Builder setAssociatedCellCount(int value) { + bitField0_ |= 0x00000080; + associatedCellCount_ = value; + onChanged(); + return this; + } + public Builder clearAssociatedCellCount() { + bitField0_ = (bitField0_ & ~0x00000080); + associatedCellCount_ = 0; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:MutationProto) } static { - defaultInstance = new Mutate(true); + defaultInstance = new MutationProto(true); defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:Mutate) + // @@protoc_insertion_point(class_scope:MutationProto) } public interface MutateRequestOrBuilder @@ -9064,10 +9147,10 @@ public final class ClientProtos { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(); org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(); - // required .Mutate mutate = 2; - boolean hasMutate(); - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate getMutate(); - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateOrBuilder getMutateOrBuilder(); + // required .MutationProto mutation = 2; + boolean hasMutation(); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto getMutation(); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder getMutationOrBuilder(); // optional .Condition condition = 3; boolean hasCondition(); @@ -9116,17 +9199,17 @@ public final class ClientProtos { return region_; } - // required .Mutate mutate = 2; - public static final int MUTATE_FIELD_NUMBER = 2; - private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate mutate_; - public boolean hasMutate() { + // required .MutationProto mutation = 2; + public static final int MUTATION_FIELD_NUMBER = 2; + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto mutation_; + public boolean hasMutation() { return ((bitField0_ & 0x00000002) == 0x00000002); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate getMutate() { - return mutate_; + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto getMutation() { + return mutation_; } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateOrBuilder getMutateOrBuilder() { - return mutate_; + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder getMutationOrBuilder() { + return mutation_; } // optional .Condition condition = 3; @@ -9144,7 +9227,7 @@ public final class ClientProtos { private void initFields() { region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); - mutate_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.getDefaultInstance(); + mutation_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance(); condition_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Condition.getDefaultInstance(); } private byte memoizedIsInitialized = -1; @@ -9156,7 +9239,7 @@ public final class ClientProtos { memoizedIsInitialized = 0; return false; } - if (!hasMutate()) { + if (!hasMutation()) { memoizedIsInitialized = 0; return false; } @@ -9164,7 +9247,7 @@ public final class ClientProtos { memoizedIsInitialized = 0; return false; } - if (!getMutate().isInitialized()) { + if (!getMutation().isInitialized()) { memoizedIsInitialized = 0; return false; } @@ -9185,7 +9268,7 @@ public final class ClientProtos { output.writeMessage(1, region_); } if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeMessage(2, mutate_); + output.writeMessage(2, mutation_); } if (((bitField0_ & 0x00000004) == 0x00000004)) { output.writeMessage(3, condition_); @@ -9205,7 +9288,7 @@ public final class ClientProtos { } if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, mutate_); + .computeMessageSize(2, mutation_); } if (((bitField0_ & 0x00000004) == 0x00000004)) { size += com.google.protobuf.CodedOutputStream @@ -9239,10 +9322,10 @@ public final class ClientProtos { result = result && getRegion() .equals(other.getRegion()); } - result = result && (hasMutate() == other.hasMutate()); - if (hasMutate()) { - result = result && getMutate() - .equals(other.getMutate()); + result = result && (hasMutation() == other.hasMutation()); + if (hasMutation()) { + result = result && getMutation() + .equals(other.getMutation()); } result = result && (hasCondition() == other.hasCondition()); if (hasCondition()) { @@ -9262,9 +9345,9 @@ public final class ClientProtos { hash = (37 * hash) + REGION_FIELD_NUMBER; hash = (53 * hash) + getRegion().hashCode(); } - if (hasMutate()) { - hash = (37 * hash) + MUTATE_FIELD_NUMBER; - hash = (53 * hash) + getMutate().hashCode(); + if (hasMutation()) { + hash = (37 * hash) + MUTATION_FIELD_NUMBER; + hash = (53 * hash) + getMutation().hashCode(); } if (hasCondition()) { hash = (37 * hash) + CONDITION_FIELD_NUMBER; @@ -9379,7 +9462,7 @@ public final class ClientProtos { private void maybeForceBuilderInitialization() { if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { getRegionFieldBuilder(); - getMutateFieldBuilder(); + getMutationFieldBuilder(); getConditionFieldBuilder(); } } @@ -9395,10 +9478,10 @@ public final class ClientProtos { regionBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000001); - if (mutateBuilder_ == null) { - mutate_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.getDefaultInstance(); + if (mutationBuilder_ == null) { + mutation_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance(); } else { - mutateBuilder_.clear(); + mutationBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000002); if (conditionBuilder_ == null) { @@ -9456,10 +9539,10 @@ public final class ClientProtos { if (((from_bitField0_ & 0x00000002) == 0x00000002)) { to_bitField0_ |= 0x00000002; } - if (mutateBuilder_ == null) { - result.mutate_ = mutate_; + if (mutationBuilder_ == null) { + result.mutation_ = mutation_; } else { - result.mutate_ = mutateBuilder_.build(); + result.mutation_ = mutationBuilder_.build(); } if (((from_bitField0_ & 0x00000004) == 0x00000004)) { to_bitField0_ |= 0x00000004; @@ -9488,8 +9571,8 @@ public final class ClientProtos { if (other.hasRegion()) { mergeRegion(other.getRegion()); } - if (other.hasMutate()) { - mergeMutate(other.getMutate()); + if (other.hasMutation()) { + mergeMutation(other.getMutation()); } if (other.hasCondition()) { mergeCondition(other.getCondition()); @@ -9503,7 +9586,7 @@ public final class ClientProtos { return false; } - if (!hasMutate()) { + if (!hasMutation()) { return false; } @@ -9511,7 +9594,7 @@ public final class ClientProtos { return false; } - if (!getMutate().isInitialized()) { + if (!getMutation().isInitialized()) { return false; } @@ -9557,12 +9640,12 @@ public final class ClientProtos { break; } case 18: { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.newBuilder(); - if (hasMutate()) { - subBuilder.mergeFrom(getMutate()); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.newBuilder(); + if (hasMutation()) { + subBuilder.mergeFrom(getMutation()); } input.readMessage(subBuilder, extensionRegistry); - setMutate(subBuilder.buildPartial()); + setMutation(subBuilder.buildPartial()); break; } case 26: { @@ -9670,94 +9753,94 @@ public final class ClientProtos { return regionBuilder_; } - // required .Mutate mutate = 2; - private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate mutate_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.getDefaultInstance(); + // required .MutationProto mutation = 2; + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto mutation_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance(); private com.google.protobuf.SingleFieldBuilder< - 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> mutateBuilder_; - public boolean hasMutate() { + 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> mutationBuilder_; + public boolean hasMutation() { return ((bitField0_ & 0x00000002) == 0x00000002); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate getMutate() { - if (mutateBuilder_ == null) { - return mutate_; + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto getMutation() { + if (mutationBuilder_ == null) { + return mutation_; } else { - return mutateBuilder_.getMessage(); + return mutationBuilder_.getMessage(); } } - public Builder setMutate(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate value) { - if (mutateBuilder_ == null) { + public Builder setMutation(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto value) { + if (mutationBuilder_ == null) { if (value == null) { throw new NullPointerException(); } - mutate_ = value; + mutation_ = value; onChanged(); } else { - mutateBuilder_.setMessage(value); + mutationBuilder_.setMessage(value); } bitField0_ |= 0x00000002; return this; } - public Builder setMutate( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.Builder builderForValue) { - if (mutateBuilder_ == null) { - mutate_ = builderForValue.build(); + public Builder setMutation( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder builderForValue) { + if (mutationBuilder_ == null) { + mutation_ = builderForValue.build(); onChanged(); } else { - mutateBuilder_.setMessage(builderForValue.build()); + mutationBuilder_.setMessage(builderForValue.build()); } bitField0_ |= 0x00000002; return this; } - public Builder mergeMutate(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate value) { - if (mutateBuilder_ == null) { + public Builder mergeMutation(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto value) { + if (mutationBuilder_ == null) { if (((bitField0_ & 0x00000002) == 0x00000002) && - mutate_ != org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.getDefaultInstance()) { - mutate_ = - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.newBuilder(mutate_).mergeFrom(value).buildPartial(); + mutation_ != org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance()) { + mutation_ = + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.newBuilder(mutation_).mergeFrom(value).buildPartial(); } else { - mutate_ = value; + mutation_ = value; } onChanged(); } else { - mutateBuilder_.mergeFrom(value); + mutationBuilder_.mergeFrom(value); } bitField0_ |= 0x00000002; return this; } - public Builder clearMutate() { - if (mutateBuilder_ == null) { - mutate_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.getDefaultInstance(); + public Builder clearMutation() { + if (mutationBuilder_ == null) { + mutation_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance(); onChanged(); } else { - mutateBuilder_.clear(); + mutationBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000002); return this; } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.Builder getMutateBuilder() { + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder getMutationBuilder() { bitField0_ |= 0x00000002; onChanged(); - return getMutateFieldBuilder().getBuilder(); + return getMutationFieldBuilder().getBuilder(); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateOrBuilder getMutateOrBuilder() { - if (mutateBuilder_ != null) { - return mutateBuilder_.getMessageOrBuilder(); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder getMutationOrBuilder() { + if (mutationBuilder_ != null) { + return mutationBuilder_.getMessageOrBuilder(); } else { - return mutate_; + return mutation_; } } private com.google.protobuf.SingleFieldBuilder< - 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> - getMutateFieldBuilder() { - if (mutateBuilder_ == null) { - mutateBuilder_ = new com.google.protobuf.SingleFieldBuilder< - 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>( - mutate_, + 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> + getMutationFieldBuilder() { + if (mutationBuilder_ == null) { + mutationBuilder_ = new com.google.protobuf.SingleFieldBuilder< + 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>( + mutation_, getParentForChildren(), isClean()); - mutate_ = null; + mutation_ = null; } - return mutateBuilder_; + return mutationBuilder_; } // optional .Condition condition = 3; @@ -9934,12 +10017,6 @@ public final class ClientProtos { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - if (hasResult()) { - if (!getResult().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } memoizedIsInitialized = 1; return true; } @@ -10222,12 +10299,6 @@ public final class ClientProtos { } public final boolean isInitialized() { - if (hasResult()) { - if (!getResult().isInitialized()) { - - return false; - } - } return true; } @@ -13227,12 +13298,6 @@ public final class ClientProtos { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - for (int i = 0; i < getResultCount(); i++) { - if (!getResult(i).isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } memoizedIsInitialized = 1; return true; } @@ -13611,12 +13676,6 @@ public final class ClientProtos { } public final boolean isInitialized() { - for (int i = 0; i < getResultCount(); i++) { - if (!getResult(i).isInitialized()) { - - return false; - } - } return true; } @@ -17624,10 +17683,10 @@ public final class ClientProtos { public interface MultiActionOrBuilder extends com.google.protobuf.MessageOrBuilder { - // optional .Mutate mutate = 1; - boolean hasMutate(); - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate getMutate(); - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateOrBuilder getMutateOrBuilder(); + // optional .MutationProto mutation = 1; + boolean hasMutation(); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto getMutation(); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder getMutationOrBuilder(); // optional .Get get = 2; boolean hasGet(); @@ -17663,17 +17722,17 @@ public final class ClientProtos { } private int bitField0_; - // optional .Mutate mutate = 1; - public static final int MUTATE_FIELD_NUMBER = 1; - private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate mutate_; - public boolean hasMutate() { + // optional .MutationProto mutation = 1; + public static final int MUTATION_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto mutation_; + public boolean hasMutation() { return ((bitField0_ & 0x00000001) == 0x00000001); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate getMutate() { - return mutate_; + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto getMutation() { + return mutation_; } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateOrBuilder getMutateOrBuilder() { - return mutate_; + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder getMutationOrBuilder() { + return mutation_; } // optional .Get get = 2; @@ -17690,7 +17749,7 @@ public final class ClientProtos { } private void initFields() { - mutate_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.getDefaultInstance(); + mutation_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance(); get_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.getDefaultInstance(); } private byte memoizedIsInitialized = -1; @@ -17698,8 +17757,8 @@ public final class ClientProtos { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - if (hasMutate()) { - if (!getMutate().isInitialized()) { + if (hasMutation()) { + if (!getMutation().isInitialized()) { memoizedIsInitialized = 0; return false; } @@ -17718,7 +17777,7 @@ public final class ClientProtos { throws java.io.IOException { getSerializedSize(); if (((bitField0_ & 0x00000001) == 0x00000001)) { - output.writeMessage(1, mutate_); + output.writeMessage(1, mutation_); } if (((bitField0_ & 0x00000002) == 0x00000002)) { output.writeMessage(2, get_); @@ -17734,7 +17793,7 @@ public final class ClientProtos { size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, mutate_); + .computeMessageSize(1, mutation_); } if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream @@ -17763,10 +17822,10 @@ public final class ClientProtos { org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction) obj; boolean result = true; - result = result && (hasMutate() == other.hasMutate()); - if (hasMutate()) { - result = result && getMutate() - .equals(other.getMutate()); + result = result && (hasMutation() == other.hasMutation()); + if (hasMutation()) { + result = result && getMutation() + .equals(other.getMutation()); } result = result && (hasGet() == other.hasGet()); if (hasGet()) { @@ -17782,9 +17841,9 @@ public final class ClientProtos { public int hashCode() { int hash = 41; hash = (19 * hash) + getDescriptorForType().hashCode(); - if (hasMutate()) { - hash = (37 * hash) + MUTATE_FIELD_NUMBER; - hash = (53 * hash) + getMutate().hashCode(); + if (hasMutation()) { + hash = (37 * hash) + MUTATION_FIELD_NUMBER; + hash = (53 * hash) + getMutation().hashCode(); } if (hasGet()) { hash = (37 * hash) + GET_FIELD_NUMBER; @@ -17898,7 +17957,7 @@ public final class ClientProtos { } private void maybeForceBuilderInitialization() { if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { - getMutateFieldBuilder(); + getMutationFieldBuilder(); getGetFieldBuilder(); } } @@ -17908,10 +17967,10 @@ public final class ClientProtos { public Builder clear() { super.clear(); - if (mutateBuilder_ == null) { - mutate_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.getDefaultInstance(); + if (mutationBuilder_ == null) { + mutation_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance(); } else { - mutateBuilder_.clear(); + mutationBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000001); if (getBuilder_ == null) { @@ -17961,10 +18020,10 @@ public final class ClientProtos { if (((from_bitField0_ & 0x00000001) == 0x00000001)) { to_bitField0_ |= 0x00000001; } - if (mutateBuilder_ == null) { - result.mutate_ = mutate_; + if (mutationBuilder_ == null) { + result.mutation_ = mutation_; } else { - result.mutate_ = mutateBuilder_.build(); + result.mutation_ = mutationBuilder_.build(); } if (((from_bitField0_ & 0x00000002) == 0x00000002)) { to_bitField0_ |= 0x00000002; @@ -17990,8 +18049,8 @@ public final class ClientProtos { public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction other) { if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.getDefaultInstance()) return this; - if (other.hasMutate()) { - mergeMutate(other.getMutate()); + if (other.hasMutation()) { + mergeMutation(other.getMutation()); } if (other.hasGet()) { mergeGet(other.getGet()); @@ -18001,8 +18060,8 @@ public final class ClientProtos { } public final boolean isInitialized() { - if (hasMutate()) { - if (!getMutate().isInitialized()) { + if (hasMutation()) { + if (!getMutation().isInitialized()) { return false; } @@ -18040,12 +18099,12 @@ public final class ClientProtos { break; } case 10: { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.newBuilder(); - if (hasMutate()) { - subBuilder.mergeFrom(getMutate()); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.newBuilder(); + if (hasMutation()) { + subBuilder.mergeFrom(getMutation()); } input.readMessage(subBuilder, extensionRegistry); - setMutate(subBuilder.buildPartial()); + setMutation(subBuilder.buildPartial()); break; } case 18: { @@ -18063,94 +18122,94 @@ public final class ClientProtos { private int bitField0_; - // optional .Mutate mutate = 1; - private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate mutate_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.getDefaultInstance(); + // optional .MutationProto mutation = 1; + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto mutation_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance(); private com.google.protobuf.SingleFieldBuilder< - 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> mutateBuilder_; - public boolean hasMutate() { + 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> mutationBuilder_; + public boolean hasMutation() { return ((bitField0_ & 0x00000001) == 0x00000001); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate getMutate() { - if (mutateBuilder_ == null) { - return mutate_; + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto getMutation() { + if (mutationBuilder_ == null) { + return mutation_; } else { - return mutateBuilder_.getMessage(); + return mutationBuilder_.getMessage(); } } - public Builder setMutate(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate value) { - if (mutateBuilder_ == null) { + public Builder setMutation(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto value) { + if (mutationBuilder_ == null) { if (value == null) { throw new NullPointerException(); } - mutate_ = value; + mutation_ = value; onChanged(); } else { - mutateBuilder_.setMessage(value); + mutationBuilder_.setMessage(value); } bitField0_ |= 0x00000001; return this; } - public Builder setMutate( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.Builder builderForValue) { - if (mutateBuilder_ == null) { - mutate_ = builderForValue.build(); + public Builder setMutation( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder builderForValue) { + if (mutationBuilder_ == null) { + mutation_ = builderForValue.build(); onChanged(); } else { - mutateBuilder_.setMessage(builderForValue.build()); + mutationBuilder_.setMessage(builderForValue.build()); } bitField0_ |= 0x00000001; return this; } - public Builder mergeMutate(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate value) { - if (mutateBuilder_ == null) { + public Builder mergeMutation(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto value) { + if (mutationBuilder_ == null) { if (((bitField0_ & 0x00000001) == 0x00000001) && - mutate_ != org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.getDefaultInstance()) { - mutate_ = - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.newBuilder(mutate_).mergeFrom(value).buildPartial(); + mutation_ != org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance()) { + mutation_ = + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.newBuilder(mutation_).mergeFrom(value).buildPartial(); } else { - mutate_ = value; + mutation_ = value; } onChanged(); } else { - mutateBuilder_.mergeFrom(value); + mutationBuilder_.mergeFrom(value); } bitField0_ |= 0x00000001; return this; } - public Builder clearMutate() { - if (mutateBuilder_ == null) { - mutate_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.getDefaultInstance(); + public Builder clearMutation() { + if (mutationBuilder_ == null) { + mutation_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance(); onChanged(); } else { - mutateBuilder_.clear(); + mutationBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000001); return this; } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.Builder getMutateBuilder() { + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder getMutationBuilder() { bitField0_ |= 0x00000001; onChanged(); - return getMutateFieldBuilder().getBuilder(); + return getMutationFieldBuilder().getBuilder(); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateOrBuilder getMutateOrBuilder() { - if (mutateBuilder_ != null) { - return mutateBuilder_.getMessageOrBuilder(); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder getMutationOrBuilder() { + if (mutationBuilder_ != null) { + return mutationBuilder_.getMessageOrBuilder(); } else { - return mutate_; + return mutation_; } } private com.google.protobuf.SingleFieldBuilder< - 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> - getMutateFieldBuilder() { - if (mutateBuilder_ == null) { - mutateBuilder_ = new com.google.protobuf.SingleFieldBuilder< - 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>( - mutate_, + 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> + getMutationFieldBuilder() { + if (mutationBuilder_ == null) { + mutationBuilder_ = new com.google.protobuf.SingleFieldBuilder< + 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>( + mutation_, getParentForChildren(), isClean()); - mutate_ = null; + mutation_ = null; } - return mutateBuilder_; + return mutationBuilder_; } // optional .Get get = 2; @@ -18331,12 +18390,6 @@ public final class ClientProtos { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - if (hasValue()) { - if (!getValue().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } if (hasException()) { if (!getException().isInitialized()) { memoizedIsInitialized = 0; @@ -18634,12 +18687,6 @@ public final class ClientProtos { } public final boolean isInitialized() { - if (hasValue()) { - if (!getValue().isInitialized()) { - - return false; - } - } if (hasException()) { if (!getException().isInitialized()) { @@ -20962,20 +21009,20 @@ public final class ClientProtos { com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_Condition_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_Mutate_descriptor; + internal_static_MutationProto_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_Mutate_fieldAccessorTable; + internal_static_MutationProto_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_Mutate_ColumnValue_descriptor; + internal_static_MutationProto_ColumnValue_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_Mutate_ColumnValue_fieldAccessorTable; + internal_static_MutationProto_ColumnValue_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_Mutate_ColumnValue_QualifierValue_descriptor; + internal_static_MutationProto_ColumnValue_QualifierValue_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_Mutate_ColumnValue_QualifierValue_fieldAccessorTable; + internal_static_MutationProto_ColumnValue_QualifierValue_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor internal_static_MutateRequest_descriptor; private static @@ -21068,83 +21115,86 @@ public final class ClientProtos { "timeRange\030\005 \001(\0132\n.TimeRange\022\026\n\013maxVersio" + "ns\030\006 \001(\r:\0011\022\031\n\013cacheBlocks\030\007 \001(\010:\004true\022\022" + "\n\nstoreLimit\030\010 \001(\r\022\023\n\013storeOffset\030\t \001(\r\"" + - "%\n\006Result\022\033\n\010keyValue\030\001 \003(\0132\t.KeyValue\"r" + - "\n\nGetRequest\022 \n\006region\030\001 \002(\0132\020.RegionSpe", - "cifier\022\021\n\003get\030\002 \002(\0132\004.Get\022\030\n\020closestRowB" + - "efore\030\003 \001(\010\022\025\n\rexistenceOnly\030\004 \001(\010\"w\n\017Mu" + - "ltiGetRequest\022 \n\006region\030\001 \002(\0132\020.RegionSp" + - "ecifier\022\021\n\003get\030\002 \003(\0132\004.Get\022\030\n\020closestRow" + - "Before\030\003 \001(\010\022\025\n\rexistenceOnly\030\004 \001(\010\"6\n\013G" + - "etResponse\022\027\n\006result\030\001 \001(\0132\007.Result\022\016\n\006e" + - "xists\030\002 \001(\010\";\n\020MultiGetResponse\022\027\n\006resul" + - "t\030\001 \003(\0132\007.Result\022\016\n\006exists\030\002 \003(\010\"\177\n\tCond" + - "ition\022\013\n\003row\030\001 \002(\014\022\016\n\006family\030\002 \002(\014\022\021\n\tqu" + - "alifier\030\003 \002(\014\022!\n\013compareType\030\004 \002(\0162\014.Com", - "pareType\022\037\n\ncomparator\030\005 \002(\0132\013.Comparato" + - "r\"\266\004\n\006Mutate\022\013\n\003row\030\001 \002(\014\022&\n\nmutateType\030" + - "\002 \002(\0162\022.Mutate.MutateType\022(\n\013columnValue" + - "\030\003 \003(\0132\023.Mutate.ColumnValue\022!\n\tattribute" + - "\030\004 \003(\0132\016.NameBytesPair\022\021\n\ttimestamp\030\005 \001(" + - "\004\022\030\n\nwriteToWAL\030\006 \001(\010:\004true\022\035\n\ttimeRange" + - "\030\n \001(\0132\n.TimeRange\032\310\001\n\013ColumnValue\022\016\n\006fa" + - "mily\030\001 \002(\014\022:\n\016qualifierValue\030\002 \003(\0132\".Mut" + - "ate.ColumnValue.QualifierValue\032m\n\016Qualif" + - "ierValue\022\021\n\tqualifier\030\001 \001(\014\022\r\n\005value\030\002 \001", - "(\014\022\021\n\ttimestamp\030\003 \001(\004\022&\n\ndeleteType\030\004 \001(" + - "\0162\022.Mutate.DeleteType\"<\n\nMutateType\022\n\n\006A" + - "PPEND\020\000\022\r\n\tINCREMENT\020\001\022\007\n\003PUT\020\002\022\n\n\006DELET" + - "E\020\003\"U\n\nDeleteType\022\026\n\022DELETE_ONE_VERSION\020" + - "\000\022\034\n\030DELETE_MULTIPLE_VERSIONS\020\001\022\021\n\rDELET" + - "E_FAMILY\020\002\"i\n\rMutateRequest\022 \n\006region\030\001 " + - "\002(\0132\020.RegionSpecifier\022\027\n\006mutate\030\002 \002(\0132\007." + - "Mutate\022\035\n\tcondition\030\003 \001(\0132\n.Condition\"<\n" + - "\016MutateResponse\022\027\n\006result\030\001 \001(\0132\007.Result" + - "\022\021\n\tprocessed\030\002 \001(\010\"\307\002\n\004Scan\022\027\n\006column\030\001", - " \003(\0132\007.Column\022!\n\tattribute\030\002 \003(\0132\016.NameB" + - "ytesPair\022\020\n\010startRow\030\003 \001(\014\022\017\n\007stopRow\030\004 " + - "\001(\014\022\027\n\006filter\030\005 \001(\0132\007.Filter\022\035\n\ttimeRang" + - "e\030\006 \001(\0132\n.TimeRange\022\026\n\013maxVersions\030\007 \001(\r" + - ":\0011\022\031\n\013cacheBlocks\030\010 \001(\010:\004true\022\021\n\tbatchS" + - "ize\030\t \001(\r\022\025\n\rmaxResultSize\030\n \001(\004\022\022\n\nstor" + - "eLimit\030\013 \001(\r\022\023\n\013storeOffset\030\014 \001(\r\022\"\n\032loa" + - "dColumnFamiliesOnDemand\030\r \001(\010\"\230\001\n\013ScanRe" + - "quest\022 \n\006region\030\001 \001(\0132\020.RegionSpecifier\022" + - "\023\n\004scan\030\002 \001(\0132\005.Scan\022\021\n\tscannerId\030\003 \001(\004\022", - "\024\n\014numberOfRows\030\004 \001(\r\022\024\n\014closeScanner\030\005 " + - "\001(\010\022\023\n\013nextCallSeq\030\006 \001(\004\"u\n\014ScanResponse" + - "\022\027\n\006result\030\001 \003(\0132\007.Result\022\021\n\tscannerId\030\002" + - " \001(\004\022\023\n\013moreResults\030\003 \001(\010\022\013\n\003ttl\030\004 \001(\r\022\027" + - "\n\017resultSizeBytes\030\005 \001(\004\"\260\001\n\024BulkLoadHFil" + - "eRequest\022 \n\006region\030\001 \002(\0132\020.RegionSpecifi" + - "er\0224\n\nfamilyPath\030\002 \003(\0132 .BulkLoadHFileRe" + - "quest.FamilyPath\022\024\n\014assignSeqNum\030\003 \001(\010\032*" + - "\n\nFamilyPath\022\016\n\006family\030\001 \002(\014\022\014\n\004path\030\002 \002" + - "(\t\"\'\n\025BulkLoadHFileResponse\022\016\n\006loaded\030\001 ", - "\002(\010\"_\n\026CoprocessorServiceCall\022\013\n\003row\030\001 \002" + - "(\014\022\023\n\013serviceName\030\002 \002(\t\022\022\n\nmethodName\030\003 " + - "\002(\t\022\017\n\007request\030\004 \002(\014\"d\n\031CoprocessorServi" + - "ceRequest\022 \n\006region\030\001 \002(\0132\020.RegionSpecif" + - "ier\022%\n\004call\030\002 \002(\0132\027.CoprocessorServiceCa" + - "ll\"]\n\032CoprocessorServiceResponse\022 \n\006regi" + - "on\030\001 \002(\0132\020.RegionSpecifier\022\035\n\005value\030\002 \002(" + - "\0132\016.NameBytesPair\"9\n\013MultiAction\022\027\n\006muta" + - "te\030\001 \001(\0132\007.Mutate\022\021\n\003get\030\002 \001(\0132\004.Get\"I\n\014" + - "ActionResult\022\026\n\005value\030\001 \001(\0132\007.Result\022!\n\t", - "exception\030\002 \001(\0132\016.NameBytesPair\"^\n\014Multi" + - "Request\022 \n\006region\030\001 \002(\0132\020.RegionSpecifie" + - "r\022\034\n\006action\030\002 \003(\0132\014.MultiAction\022\016\n\006atomi" + - "c\030\003 \001(\010\".\n\rMultiResponse\022\035\n\006result\030\001 \003(\013" + - "2\r.ActionResult2\342\002\n\rClientService\022 \n\003get" + - "\022\013.GetRequest\032\014.GetResponse\022/\n\010multiGet\022" + - "\020.MultiGetRequest\032\021.MultiGetResponse\022)\n\006" + - "mutate\022\016.MutateRequest\032\017.MutateResponse\022" + - "#\n\004scan\022\014.ScanRequest\032\r.ScanResponse\022>\n\r" + - "bulkLoadHFile\022\025.BulkLoadHFileRequest\032\026.B", - "ulkLoadHFileResponse\022F\n\013execService\022\032.Co" + - "processorServiceRequest\032\033.CoprocessorSer" + - "viceResponse\022&\n\005multi\022\r.MultiRequest\032\016.M" + - "ultiResponseBB\n*org.apache.hadoop.hbase." + - "protobuf.generatedB\014ClientProtosH\001\210\001\001\240\001\001" + ":\n\006Result\022\023\n\004cell\030\001 \003(\0132\005.Cell\022\033\n\023associ" + + "atedCellCount\030\002 \001(\005\"r\n\nGetRequest\022 \n\006reg", + "ion\030\001 \002(\0132\020.RegionSpecifier\022\021\n\003get\030\002 \002(\013" + + "2\004.Get\022\030\n\020closestRowBefore\030\003 \001(\010\022\025\n\rexis" + + "tenceOnly\030\004 \001(\010\"w\n\017MultiGetRequest\022 \n\006re" + + "gion\030\001 \002(\0132\020.RegionSpecifier\022\021\n\003get\030\002 \003(" + + "\0132\004.Get\022\030\n\020closestRowBefore\030\003 \001(\010\022\025\n\rexi" + + "stenceOnly\030\004 \001(\010\"6\n\013GetResponse\022\027\n\006resul" + + "t\030\001 \001(\0132\007.Result\022\016\n\006exists\030\002 \001(\010\";\n\020Mult" + + "iGetResponse\022\027\n\006result\030\001 \003(\0132\007.Result\022\016\n" + + "\006exists\030\002 \003(\010\"\177\n\tCondition\022\013\n\003row\030\001 \002(\014\022" + + "\016\n\006family\030\002 \002(\014\022\021\n\tqualifier\030\003 \002(\014\022!\n\013co", + "mpareType\030\004 \002(\0162\014.CompareType\022\037\n\ncompara" + + "tor\030\005 \002(\0132\013.Comparator\"\372\004\n\rMutationProto" + + "\022\013\n\003row\030\001 \001(\014\022/\n\nmutateType\030\002 \001(\0162\033.Muta" + + "tionProto.MutationType\022/\n\013columnValue\030\003 " + + "\003(\0132\032.MutationProto.ColumnValue\022\021\n\ttimes" + + "tamp\030\004 \001(\004\022!\n\tattribute\030\005 \003(\0132\016.NameByte" + + "sPair\022\030\n\nwriteToWAL\030\006 \001(\010:\004true\022\035\n\ttimeR" + + "ange\030\007 \001(\0132\n.TimeRange\022\033\n\023associatedCell" + + "Count\030\010 \001(\005\032\326\001\n\013ColumnValue\022\016\n\006family\030\001 " + + "\002(\014\022A\n\016qualifierValue\030\002 \003(\0132).MutationPr", + "oto.ColumnValue.QualifierValue\032t\n\016Qualif" + + "ierValue\022\021\n\tqualifier\030\001 \001(\014\022\r\n\005value\030\002 \001" + + "(\014\022\021\n\ttimestamp\030\003 \001(\004\022-\n\ndeleteType\030\004 \001(" + + "\0162\031.MutationProto.DeleteType\">\n\014Mutation" + + "Type\022\n\n\006APPEND\020\000\022\r\n\tINCREMENT\020\001\022\007\n\003PUT\020\002" + + "\022\n\n\006DELETE\020\003\"U\n\nDeleteType\022\026\n\022DELETE_ONE" + + "_VERSION\020\000\022\034\n\030DELETE_MULTIPLE_VERSIONS\020\001" + + "\022\021\n\rDELETE_FAMILY\020\002\"r\n\rMutateRequest\022 \n\006" + + "region\030\001 \002(\0132\020.RegionSpecifier\022 \n\010mutati" + + "on\030\002 \002(\0132\016.MutationProto\022\035\n\tcondition\030\003 ", + "\001(\0132\n.Condition\"<\n\016MutateResponse\022\027\n\006res" + + "ult\030\001 \001(\0132\007.Result\022\021\n\tprocessed\030\002 \001(\010\"\307\002" + + "\n\004Scan\022\027\n\006column\030\001 \003(\0132\007.Column\022!\n\tattri" + + "bute\030\002 \003(\0132\016.NameBytesPair\022\020\n\010startRow\030\003" + + " \001(\014\022\017\n\007stopRow\030\004 \001(\014\022\027\n\006filter\030\005 \001(\0132\007." + + "Filter\022\035\n\ttimeRange\030\006 \001(\0132\n.TimeRange\022\026\n" + + "\013maxVersions\030\007 \001(\r:\0011\022\031\n\013cacheBlocks\030\010 \001" + + "(\010:\004true\022\021\n\tbatchSize\030\t \001(\r\022\025\n\rmaxResult" + + "Size\030\n \001(\004\022\022\n\nstoreLimit\030\013 \001(\r\022\023\n\013storeO" + + "ffset\030\014 \001(\r\022\"\n\032loadColumnFamiliesOnDeman", + "d\030\r \001(\010\"\230\001\n\013ScanRequest\022 \n\006region\030\001 \001(\0132" + + "\020.RegionSpecifier\022\023\n\004scan\030\002 \001(\0132\005.Scan\022\021" + + "\n\tscannerId\030\003 \001(\004\022\024\n\014numberOfRows\030\004 \001(\r\022" + + "\024\n\014closeScanner\030\005 \001(\010\022\023\n\013nextCallSeq\030\006 \001" + + "(\004\"u\n\014ScanResponse\022\027\n\006result\030\001 \003(\0132\007.Res" + + "ult\022\021\n\tscannerId\030\002 \001(\004\022\023\n\013moreResults\030\003 " + + "\001(\010\022\013\n\003ttl\030\004 \001(\r\022\027\n\017resultSizeBytes\030\005 \001(" + + "\004\"\260\001\n\024BulkLoadHFileRequest\022 \n\006region\030\001 \002" + + "(\0132\020.RegionSpecifier\0224\n\nfamilyPath\030\002 \003(\013" + + "2 .BulkLoadHFileRequest.FamilyPath\022\024\n\014as", + "signSeqNum\030\003 \001(\010\032*\n\nFamilyPath\022\016\n\006family" + + "\030\001 \002(\014\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoadHFileRes" + + "ponse\022\016\n\006loaded\030\001 \002(\010\"_\n\026CoprocessorServ" + + "iceCall\022\013\n\003row\030\001 \002(\014\022\023\n\013serviceName\030\002 \002(" + + "\t\022\022\n\nmethodName\030\003 \002(\t\022\017\n\007request\030\004 \002(\014\"d" + + "\n\031CoprocessorServiceRequest\022 \n\006region\030\001 " + + "\002(\0132\020.RegionSpecifier\022%\n\004call\030\002 \002(\0132\027.Co" + + "processorServiceCall\"]\n\032CoprocessorServi" + + "ceResponse\022 \n\006region\030\001 \002(\0132\020.RegionSpeci" + + "fier\022\035\n\005value\030\002 \002(\0132\016.NameBytesPair\"B\n\013M", + "ultiAction\022 \n\010mutation\030\001 \001(\0132\016.MutationP" + + "roto\022\021\n\003get\030\002 \001(\0132\004.Get\"I\n\014ActionResult\022" + + "\026\n\005value\030\001 \001(\0132\007.Result\022!\n\texception\030\002 \001" + + "(\0132\016.NameBytesPair\"^\n\014MultiRequest\022 \n\006re" + + "gion\030\001 \002(\0132\020.RegionSpecifier\022\034\n\006action\030\002" + + " \003(\0132\014.MultiAction\022\016\n\006atomic\030\003 \001(\010\".\n\rMu" + + "ltiResponse\022\035\n\006result\030\001 \003(\0132\r.ActionResu" + + "lt2\342\002\n\rClientService\022 \n\003get\022\013.GetRequest" + + "\032\014.GetResponse\022/\n\010multiGet\022\020.MultiGetReq" + + "uest\032\021.MultiGetResponse\022)\n\006mutate\022\016.Muta", + "teRequest\032\017.MutateResponse\022#\n\004scan\022\014.Sca" + + "nRequest\032\r.ScanResponse\022>\n\rbulkLoadHFile" + + "\022\025.BulkLoadHFileRequest\032\026.BulkLoadHFileR" + + "esponse\022F\n\013execService\022\032.CoprocessorServ" + + "iceRequest\032\033.CoprocessorServiceResponse\022" + + "&\n\005multi\022\r.MultiRequest\032\016.MultiResponseB" + + "B\n*org.apache.hadoop.hbase.protobuf.gene" + + "ratedB\014ClientProtosH\001\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -21172,7 +21222,7 @@ public final class ClientProtos { internal_static_Result_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_Result_descriptor, - new java.lang.String[] { "KeyValue", }, + new java.lang.String[] { "Cell", "AssociatedCellCount", }, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder.class); internal_static_GetRequest_descriptor = @@ -21215,36 +21265,36 @@ public final class ClientProtos { new java.lang.String[] { "Row", "Family", "Qualifier", "CompareType", "Comparator", }, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Condition.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Condition.Builder.class); - internal_static_Mutate_descriptor = + internal_static_MutationProto_descriptor = getDescriptor().getMessageTypes().get(8); - internal_static_Mutate_fieldAccessorTable = new + internal_static_MutationProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_Mutate_descriptor, - new java.lang.String[] { "Row", "MutateType", "ColumnValue", "Attribute", "Timestamp", "WriteToWAL", "TimeRange", }, - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.class, - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.Builder.class); - internal_static_Mutate_ColumnValue_descriptor = - internal_static_Mutate_descriptor.getNestedTypes().get(0); - internal_static_Mutate_ColumnValue_fieldAccessorTable = new + internal_static_MutationProto_descriptor, + new java.lang.String[] { "Row", "MutateType", "ColumnValue", "Timestamp", "Attribute", "WriteToWAL", "TimeRange", "AssociatedCellCount", }, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.class, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder.class); + internal_static_MutationProto_ColumnValue_descriptor = + internal_static_MutationProto_descriptor.getNestedTypes().get(0); + internal_static_MutationProto_ColumnValue_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_Mutate_ColumnValue_descriptor, + internal_static_MutationProto_ColumnValue_descriptor, new java.lang.String[] { "Family", "QualifierValue", }, - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.class, - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.Builder.class); - internal_static_Mutate_ColumnValue_QualifierValue_descriptor = - internal_static_Mutate_ColumnValue_descriptor.getNestedTypes().get(0); - internal_static_Mutate_ColumnValue_QualifierValue_fieldAccessorTable = new + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.class, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.Builder.class); + internal_static_MutationProto_ColumnValue_QualifierValue_descriptor = + internal_static_MutationProto_ColumnValue_descriptor.getNestedTypes().get(0); + internal_static_MutationProto_ColumnValue_QualifierValue_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_Mutate_ColumnValue_QualifierValue_descriptor, + internal_static_MutationProto_ColumnValue_QualifierValue_descriptor, new java.lang.String[] { "Qualifier", "Value", "Timestamp", "DeleteType", }, - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue.class, - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue.Builder.class); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue.class, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue.Builder.class); internal_static_MutateRequest_descriptor = getDescriptor().getMessageTypes().get(9); internal_static_MutateRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_MutateRequest_descriptor, - new java.lang.String[] { "Region", "Mutate", "Condition", }, + new java.lang.String[] { "Region", "Mutation", "Condition", }, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest.Builder.class); internal_static_MutateResponse_descriptor = @@ -21332,7 +21382,7 @@ public final class ClientProtos { internal_static_MultiAction_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_MultiAction_descriptor, - new java.lang.String[] { "Mutate", "Get", }, + new java.lang.String[] { "Mutation", "Get", }, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.Builder.class); internal_static_ActionResult_descriptor = diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java index e0f83a2..23a4f4f 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java @@ -8,6 +8,87 @@ public final class HBaseProtos { public static void registerAllExtensions( com.google.protobuf.ExtensionRegistry registry) { } + public enum CellType + implements com.google.protobuf.ProtocolMessageEnum { + MINIMUM(0, 0), + PUT(1, 4), + DELETE(2, 8), + DELETE_COLUMN(3, 12), + DELETE_FAMILY(4, 14), + MAXIMUM(5, 255), + ; + + public static final int MINIMUM_VALUE = 0; + public static final int PUT_VALUE = 4; + public static final int DELETE_VALUE = 8; + public static final int DELETE_COLUMN_VALUE = 12; + public static final int DELETE_FAMILY_VALUE = 14; + public static final int MAXIMUM_VALUE = 255; + + + public final int getNumber() { return value; } + + public static CellType valueOf(int value) { + switch (value) { + case 0: return MINIMUM; + case 4: return PUT; + case 8: return DELETE; + case 12: return DELETE_COLUMN; + case 14: return DELETE_FAMILY; + case 255: return MAXIMUM; + default: return null; + } + } + + public static com.google.protobuf.Internal.EnumLiteMap + internalGetValueMap() { + return internalValueMap; + } + private static com.google.protobuf.Internal.EnumLiteMap + internalValueMap = + new com.google.protobuf.Internal.EnumLiteMap() { + public CellType findValueByNumber(int number) { + return CellType.valueOf(number); + } + }; + + public final com.google.protobuf.Descriptors.EnumValueDescriptor + getValueDescriptor() { + return getDescriptor().getValues().get(index); + } + public final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptorForType() { + return getDescriptor(); + } + public static final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.getDescriptor().getEnumTypes().get(0); + } + + private static final CellType[] VALUES = { + MINIMUM, PUT, DELETE, DELETE_COLUMN, DELETE_FAMILY, MAXIMUM, + }; + + public static CellType valueOf( + com.google.protobuf.Descriptors.EnumValueDescriptor desc) { + if (desc.getType() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "EnumValueDescriptor is not for this type."); + } + return VALUES[desc.getIndex()]; + } + + private final int index; + private final int value; + + private CellType(int index, int value) { + this.index = index; + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:CellType) + } + public enum CompareType implements com.google.protobuf.ProtocolMessageEnum { LESS(0, 0), @@ -41,136 +122,777 @@ public final class HBaseProtos { case 6: return NO_OP; default: return null; } - } - - public static com.google.protobuf.Internal.EnumLiteMap - internalGetValueMap() { - return internalValueMap; - } - private static com.google.protobuf.Internal.EnumLiteMap - internalValueMap = - new com.google.protobuf.Internal.EnumLiteMap() { - public CompareType findValueByNumber(int number) { - return CompareType.valueOf(number); - } - }; - - public final com.google.protobuf.Descriptors.EnumValueDescriptor - getValueDescriptor() { - return getDescriptor().getValues().get(index); - } - public final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptorForType() { - return getDescriptor(); - } - public static final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.getDescriptor().getEnumTypes().get(0); - } - - private static final CompareType[] VALUES = { - LESS, LESS_OR_EQUAL, EQUAL, NOT_EQUAL, GREATER_OR_EQUAL, GREATER, NO_OP, - }; - - public static CompareType valueOf( - com.google.protobuf.Descriptors.EnumValueDescriptor desc) { - if (desc.getType() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "EnumValueDescriptor is not for this type."); + } + + public static com.google.protobuf.Internal.EnumLiteMap + internalGetValueMap() { + return internalValueMap; + } + private static com.google.protobuf.Internal.EnumLiteMap + internalValueMap = + new com.google.protobuf.Internal.EnumLiteMap() { + public CompareType findValueByNumber(int number) { + return CompareType.valueOf(number); + } + }; + + public final com.google.protobuf.Descriptors.EnumValueDescriptor + getValueDescriptor() { + return getDescriptor().getValues().get(index); + } + public final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptorForType() { + return getDescriptor(); + } + public static final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.getDescriptor().getEnumTypes().get(1); + } + + private static final CompareType[] VALUES = { + LESS, LESS_OR_EQUAL, EQUAL, NOT_EQUAL, GREATER_OR_EQUAL, GREATER, NO_OP, + }; + + public static CompareType valueOf( + com.google.protobuf.Descriptors.EnumValueDescriptor desc) { + if (desc.getType() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "EnumValueDescriptor is not for this type."); + } + return VALUES[desc.getIndex()]; + } + + private final int index; + private final int value; + + private CompareType(int index, int value) { + this.index = index; + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:CompareType) + } + + public interface CellOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // optional bytes row = 1; + boolean hasRow(); + com.google.protobuf.ByteString getRow(); + + // optional bytes family = 2; + boolean hasFamily(); + com.google.protobuf.ByteString getFamily(); + + // optional bytes qualifier = 3; + boolean hasQualifier(); + com.google.protobuf.ByteString getQualifier(); + + // optional uint64 timestamp = 4; + boolean hasTimestamp(); + long getTimestamp(); + + // optional .CellType cellType = 5; + boolean hasCellType(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType getCellType(); + + // optional bytes value = 6; + boolean hasValue(); + com.google.protobuf.ByteString getValue(); + } + public static final class Cell extends + com.google.protobuf.GeneratedMessage + implements CellOrBuilder { + // Use Cell.newBuilder() to construct. + private Cell(Builder builder) { + super(builder); + } + private Cell(boolean noInit) {} + + private static final Cell defaultInstance; + public static Cell getDefaultInstance() { + return defaultInstance; + } + + public Cell getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_Cell_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_Cell_fieldAccessorTable; + } + + private int bitField0_; + // optional bytes row = 1; + public static final int ROW_FIELD_NUMBER = 1; + private com.google.protobuf.ByteString row_; + public boolean hasRow() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public com.google.protobuf.ByteString getRow() { + return row_; + } + + // optional bytes family = 2; + public static final int FAMILY_FIELD_NUMBER = 2; + private com.google.protobuf.ByteString family_; + public boolean hasFamily() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public com.google.protobuf.ByteString getFamily() { + return family_; + } + + // optional bytes qualifier = 3; + public static final int QUALIFIER_FIELD_NUMBER = 3; + private com.google.protobuf.ByteString qualifier_; + public boolean hasQualifier() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public com.google.protobuf.ByteString getQualifier() { + return qualifier_; + } + + // optional uint64 timestamp = 4; + public static final int TIMESTAMP_FIELD_NUMBER = 4; + private long timestamp_; + public boolean hasTimestamp() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public long getTimestamp() { + return timestamp_; + } + + // optional .CellType cellType = 5; + public static final int CELLTYPE_FIELD_NUMBER = 5; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType cellType_; + public boolean hasCellType() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType getCellType() { + return cellType_; + } + + // optional bytes value = 6; + public static final int VALUE_FIELD_NUMBER = 6; + private com.google.protobuf.ByteString value_; + public boolean hasValue() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public com.google.protobuf.ByteString getValue() { + return value_; + } + + private void initFields() { + row_ = com.google.protobuf.ByteString.EMPTY; + family_ = com.google.protobuf.ByteString.EMPTY; + qualifier_ = com.google.protobuf.ByteString.EMPTY; + timestamp_ = 0L; + cellType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType.MINIMUM; + value_ = com.google.protobuf.ByteString.EMPTY; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, row_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, family_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, qualifier_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeUInt64(4, timestamp_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeEnum(5, cellType_.getNumber()); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeBytes(6, value_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(1, row_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(2, family_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(3, qualifier_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(4, timestamp_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += com.google.protobuf.CodedOutputStream + .computeEnumSize(5, cellType_.getNumber()); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(6, value_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell other = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell) obj; + + boolean result = true; + result = result && (hasRow() == other.hasRow()); + if (hasRow()) { + result = result && getRow() + .equals(other.getRow()); + } + result = result && (hasFamily() == other.hasFamily()); + if (hasFamily()) { + result = result && getFamily() + .equals(other.getFamily()); + } + result = result && (hasQualifier() == other.hasQualifier()); + if (hasQualifier()) { + result = result && getQualifier() + .equals(other.getQualifier()); + } + result = result && (hasTimestamp() == other.hasTimestamp()); + if (hasTimestamp()) { + result = result && (getTimestamp() + == other.getTimestamp()); + } + result = result && (hasCellType() == other.hasCellType()); + if (hasCellType()) { + result = result && + (getCellType() == other.getCellType()); + } + result = result && (hasValue() == other.hasValue()); + if (hasValue()) { + result = result && getValue() + .equals(other.getValue()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasRow()) { + hash = (37 * hash) + ROW_FIELD_NUMBER; + hash = (53 * hash) + getRow().hashCode(); + } + if (hasFamily()) { + hash = (37 * hash) + FAMILY_FIELD_NUMBER; + hash = (53 * hash) + getFamily().hashCode(); + } + if (hasQualifier()) { + hash = (37 * hash) + QUALIFIER_FIELD_NUMBER; + hash = (53 * hash) + getQualifier().hashCode(); + } + if (hasTimestamp()) { + hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getTimestamp()); + } + if (hasCellType()) { + hash = (37 * hash) + CELLTYPE_FIELD_NUMBER; + hash = (53 * hash) + hashEnum(getCellType()); + } + if (hasValue()) { + hash = (37 * hash) + VALUE_FIELD_NUMBER; + hash = (53 * hash) + getValue().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_Cell_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_Cell_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + row_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000001); + family_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000002); + qualifier_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000004); + timestamp_ = 0L; + bitField0_ = (bitField0_ & ~0x00000008); + cellType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType.MINIMUM; + bitField0_ = (bitField0_ & ~0x00000010); + value_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000020); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell build() { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell result = new org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.row_ = row_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.family_ = family_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.qualifier_ = qualifier_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.timestamp_ = timestamp_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.cellType_ = cellType_; + if (((from_bitField0_ & 0x00000020) == 0x00000020)) { + to_bitField0_ |= 0x00000020; + } + result.value_ = value_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.getDefaultInstance()) return this; + if (other.hasRow()) { + setRow(other.getRow()); + } + if (other.hasFamily()) { + setFamily(other.getFamily()); + } + if (other.hasQualifier()) { + setQualifier(other.getQualifier()); + } + if (other.hasTimestamp()) { + setTimestamp(other.getTimestamp()); + } + if (other.hasCellType()) { + setCellType(other.getCellType()); + } + if (other.hasValue()) { + setValue(other.getValue()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + row_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + family_ = input.readBytes(); + break; + } + case 26: { + bitField0_ |= 0x00000004; + qualifier_ = input.readBytes(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + timestamp_ = input.readUInt64(); + break; + } + case 40: { + int rawValue = input.readEnum(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType value = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType.valueOf(rawValue); + if (value == null) { + unknownFields.mergeVarintField(5, rawValue); + } else { + bitField0_ |= 0x00000010; + cellType_ = value; + } + break; + } + case 50: { + bitField0_ |= 0x00000020; + value_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // optional bytes row = 1; + private com.google.protobuf.ByteString row_ = com.google.protobuf.ByteString.EMPTY; + public boolean hasRow() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public com.google.protobuf.ByteString getRow() { + return row_; + } + public Builder setRow(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + row_ = value; + onChanged(); + return this; + } + public Builder clearRow() { + bitField0_ = (bitField0_ & ~0x00000001); + row_ = getDefaultInstance().getRow(); + onChanged(); + return this; + } + + // optional bytes family = 2; + private com.google.protobuf.ByteString family_ = com.google.protobuf.ByteString.EMPTY; + public boolean hasFamily() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public com.google.protobuf.ByteString getFamily() { + return family_; } - return VALUES[desc.getIndex()]; - } - - private final int index; - private final int value; - - private CompareType(int index, int value) { - this.index = index; - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:CompareType) + public Builder setFamily(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); } - - public enum KeyType - implements com.google.protobuf.ProtocolMessageEnum { - MINIMUM(0, 0), - PUT(1, 4), - DELETE(2, 8), - DELETE_COLUMN(3, 12), - DELETE_FAMILY(4, 14), - MAXIMUM(5, 255), - ; - - public static final int MINIMUM_VALUE = 0; - public static final int PUT_VALUE = 4; - public static final int DELETE_VALUE = 8; - public static final int DELETE_COLUMN_VALUE = 12; - public static final int DELETE_FAMILY_VALUE = 14; - public static final int MAXIMUM_VALUE = 255; - - - public final int getNumber() { return value; } - - public static KeyType valueOf(int value) { - switch (value) { - case 0: return MINIMUM; - case 4: return PUT; - case 8: return DELETE; - case 12: return DELETE_COLUMN; - case 14: return DELETE_FAMILY; - case 255: return MAXIMUM; - default: return null; + bitField0_ |= 0x00000002; + family_ = value; + onChanged(); + return this; } - } - - public static com.google.protobuf.Internal.EnumLiteMap - internalGetValueMap() { - return internalValueMap; - } - private static com.google.protobuf.Internal.EnumLiteMap - internalValueMap = - new com.google.protobuf.Internal.EnumLiteMap() { - public KeyType findValueByNumber(int number) { - return KeyType.valueOf(number); - } - }; - - public final com.google.protobuf.Descriptors.EnumValueDescriptor - getValueDescriptor() { - return getDescriptor().getValues().get(index); - } - public final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptorForType() { - return getDescriptor(); - } - public static final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.getDescriptor().getEnumTypes().get(1); - } - - private static final KeyType[] VALUES = { - MINIMUM, PUT, DELETE, DELETE_COLUMN, DELETE_FAMILY, MAXIMUM, - }; - - public static KeyType valueOf( - com.google.protobuf.Descriptors.EnumValueDescriptor desc) { - if (desc.getType() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "EnumValueDescriptor is not for this type."); + public Builder clearFamily() { + bitField0_ = (bitField0_ & ~0x00000002); + family_ = getDefaultInstance().getFamily(); + onChanged(); + return this; } - return VALUES[desc.getIndex()]; + + // optional bytes qualifier = 3; + private com.google.protobuf.ByteString qualifier_ = com.google.protobuf.ByteString.EMPTY; + public boolean hasQualifier() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public com.google.protobuf.ByteString getQualifier() { + return qualifier_; + } + public Builder setQualifier(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + qualifier_ = value; + onChanged(); + return this; + } + public Builder clearQualifier() { + bitField0_ = (bitField0_ & ~0x00000004); + qualifier_ = getDefaultInstance().getQualifier(); + onChanged(); + return this; + } + + // optional uint64 timestamp = 4; + private long timestamp_ ; + public boolean hasTimestamp() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public long getTimestamp() { + return timestamp_; + } + public Builder setTimestamp(long value) { + bitField0_ |= 0x00000008; + timestamp_ = value; + onChanged(); + return this; + } + public Builder clearTimestamp() { + bitField0_ = (bitField0_ & ~0x00000008); + timestamp_ = 0L; + onChanged(); + return this; + } + + // optional .CellType cellType = 5; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType cellType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType.MINIMUM; + public boolean hasCellType() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType getCellType() { + return cellType_; + } + public Builder setCellType(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000010; + cellType_ = value; + onChanged(); + return this; + } + public Builder clearCellType() { + bitField0_ = (bitField0_ & ~0x00000010); + cellType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType.MINIMUM; + onChanged(); + return this; + } + + // optional bytes value = 6; + private com.google.protobuf.ByteString value_ = com.google.protobuf.ByteString.EMPTY; + public boolean hasValue() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public com.google.protobuf.ByteString getValue() { + return value_; + } + public Builder setValue(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000020; + value_ = value; + onChanged(); + return this; + } + public Builder clearValue() { + bitField0_ = (bitField0_ & ~0x00000020); + value_ = getDefaultInstance().getValue(); + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:Cell) } - private final int index; - private final int value; - - private KeyType(int index, int value) { - this.index = index; - this.value = value; + static { + defaultInstance = new Cell(true); + defaultInstance.initFields(); } - // @@protoc_insertion_point(enum_scope:KeyType) + // @@protoc_insertion_point(class_scope:Cell) } public interface TableSchemaOrBuilder @@ -7331,9 +8053,9 @@ public final class HBaseProtos { boolean hasTimestamp(); long getTimestamp(); - // optional .KeyType keyType = 5; + // optional .CellType keyType = 5; boolean hasKeyType(); - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType getKeyType(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType getKeyType(); // optional bytes value = 6; boolean hasValue(); @@ -7408,13 +8130,13 @@ public final class HBaseProtos { return timestamp_; } - // optional .KeyType keyType = 5; + // optional .CellType keyType = 5; public static final int KEYTYPE_FIELD_NUMBER = 5; - private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType keyType_; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType keyType_; public boolean hasKeyType() { return ((bitField0_ & 0x00000010) == 0x00000010); } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType getKeyType() { + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType getKeyType() { return keyType_; } @@ -7433,7 +8155,7 @@ public final class HBaseProtos { family_ = com.google.protobuf.ByteString.EMPTY; qualifier_ = com.google.protobuf.ByteString.EMPTY; timestamp_ = 0L; - keyType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType.MINIMUM; + keyType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType.MINIMUM; value_ = com.google.protobuf.ByteString.EMPTY; } private byte memoizedIsInitialized = -1; @@ -7721,7 +8443,7 @@ public final class HBaseProtos { bitField0_ = (bitField0_ & ~0x00000004); timestamp_ = 0L; bitField0_ = (bitField0_ & ~0x00000008); - keyType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType.MINIMUM; + keyType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType.MINIMUM; bitField0_ = (bitField0_ & ~0x00000010); value_ = com.google.protobuf.ByteString.EMPTY; bitField0_ = (bitField0_ & ~0x00000020); @@ -7886,7 +8608,7 @@ public final class HBaseProtos { } case 40: { int rawValue = input.readEnum(); - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType value = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType.valueOf(rawValue); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType value = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType.valueOf(rawValue); if (value == null) { unknownFields.mergeVarintField(5, rawValue); } else { @@ -7999,15 +8721,15 @@ public final class HBaseProtos { return this; } - // optional .KeyType keyType = 5; - private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType keyType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType.MINIMUM; + // optional .CellType keyType = 5; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType keyType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType.MINIMUM; public boolean hasKeyType() { return ((bitField0_ & 0x00000010) == 0x00000010); } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType getKeyType() { + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType getKeyType() { return keyType_; } - public Builder setKeyType(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType value) { + public Builder setKeyType(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType value) { if (value == null) { throw new NullPointerException(); } @@ -8018,7 +8740,7 @@ public final class HBaseProtos { } public Builder clearKeyType() { bitField0_ = (bitField0_ & ~0x00000010); - keyType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyType.MINIMUM; + keyType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CellType.MINIMUM; onChanged(); return this; } @@ -12824,6 +13546,11 @@ public final class HBaseProtos { } private static com.google.protobuf.Descriptors.Descriptor + internal_static_Cell_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_Cell_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor internal_static_TableSchema_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable @@ -12927,71 +13654,82 @@ public final class HBaseProtos { descriptor; static { java.lang.String[] descriptorData = { - "\n\013hbase.proto\"\225\001\n\013TableSchema\022\014\n\004name\030\001 " + - "\001(\014\022#\n\nattributes\030\002 \003(\0132\017.BytesBytesPair" + - "\022+\n\016columnFamilies\030\003 \003(\0132\023.ColumnFamilyS" + - "chema\022&\n\rconfiguration\030\004 \003(\0132\017.NameStrin" + - "gPair\"o\n\022ColumnFamilySchema\022\014\n\004name\030\001 \002(" + - "\014\022#\n\nattributes\030\002 \003(\0132\017.BytesBytesPair\022&" + - "\n\rconfiguration\030\003 \003(\0132\017.NameStringPair\"s" + - "\n\nRegionInfo\022\020\n\010regionId\030\001 \002(\004\022\021\n\ttableN" + - "ame\030\002 \002(\014\022\020\n\010startKey\030\003 \001(\014\022\016\n\006endKey\030\004 " + - "\001(\014\022\017\n\007offline\030\005 \001(\010\022\r\n\005split\030\006 \001(\010\"\225\001\n\017", - "RegionSpecifier\0222\n\004type\030\001 \002(\0162$.RegionSp" + - "ecifier.RegionSpecifierType\022\r\n\005value\030\002 \002" + - "(\014\"?\n\023RegionSpecifierType\022\017\n\013REGION_NAME" + - "\020\001\022\027\n\023ENCODED_REGION_NAME\020\002\"\260\003\n\nRegionLo" + - "ad\022)\n\017regionSpecifier\030\001 \002(\0132\020.RegionSpec" + - "ifier\022\016\n\006stores\030\002 \001(\r\022\022\n\nstorefiles\030\003 \001(" + - "\r\022\037\n\027storeUncompressedSizeMB\030\004 \001(\r\022\027\n\017st" + - "orefileSizeMB\030\005 \001(\r\022\026\n\016memstoreSizeMB\030\006 " + - "\001(\r\022\034\n\024storefileIndexSizeMB\030\007 \001(\r\022\031\n\021rea" + - "dRequestsCount\030\010 \001(\004\022\032\n\022writeRequestsCou", - "nt\030\t \001(\004\022\032\n\022totalCompactingKVs\030\n \001(\004\022\033\n\023" + - "currentCompactedKVs\030\013 \001(\004\022\027\n\017rootIndexSi" + - "zeKB\030\014 \001(\r\022\036\n\026totalStaticIndexSizeKB\030\r \001" + - "(\r\022\036\n\026totalStaticBloomSizeKB\030\016 \001(\r\022\032\n\022co" + - "mpleteSequenceId\030\017 \001(\004\"\372\001\n\nServerLoad\022\030\n" + - "\020numberOfRequests\030\001 \001(\r\022\035\n\025totalNumberOf" + - "Requests\030\002 \001(\r\022\022\n\nusedHeapMB\030\003 \001(\r\022\021\n\tma" + - "xHeapMB\030\004 \001(\r\022 \n\013regionLoads\030\005 \003(\0132\013.Reg" + - "ionLoad\022\"\n\014coprocessors\030\006 \003(\0132\014.Coproces" + - "sor\022\027\n\017reportStartTime\030\007 \001(\004\022\025\n\rreportEn", - "dTime\030\010 \001(\004\022\026\n\016infoServerPort\030\t \001(\r\"%\n\tT" + - "imeRange\022\014\n\004from\030\001 \001(\004\022\n\n\002to\030\002 \001(\004\"0\n\006Fi" + - "lter\022\014\n\004name\030\001 \002(\t\022\030\n\020serializedFilter\030\002" + - " \001(\014\"w\n\010KeyValue\022\013\n\003row\030\001 \002(\014\022\016\n\006family\030" + - "\002 \002(\014\022\021\n\tqualifier\030\003 \002(\014\022\021\n\ttimestamp\030\004 " + - "\001(\004\022\031\n\007keyType\030\005 \001(\0162\010.KeyType\022\r\n\005value\030" + - "\006 \001(\014\"?\n\nServerName\022\020\n\010hostName\030\001 \002(\t\022\014\n" + + "\n\013hbase.proto\"u\n\004Cell\022\013\n\003row\030\001 \001(\014\022\016\n\006fa" + + "mily\030\002 \001(\014\022\021\n\tqualifier\030\003 \001(\014\022\021\n\ttimesta" + + "mp\030\004 \001(\004\022\033\n\010cellType\030\005 \001(\0162\t.CellType\022\r\n" + + "\005value\030\006 \001(\014\"\225\001\n\013TableSchema\022\014\n\004name\030\001 \001" + + "(\014\022#\n\nattributes\030\002 \003(\0132\017.BytesBytesPair\022" + + "+\n\016columnFamilies\030\003 \003(\0132\023.ColumnFamilySc" + + "hema\022&\n\rconfiguration\030\004 \003(\0132\017.NameString" + + "Pair\"o\n\022ColumnFamilySchema\022\014\n\004name\030\001 \002(\014" + + "\022#\n\nattributes\030\002 \003(\0132\017.BytesBytesPair\022&\n" + + "\rconfiguration\030\003 \003(\0132\017.NameStringPair\"s\n", + "\nRegionInfo\022\020\n\010regionId\030\001 \002(\004\022\021\n\ttableNa" + + "me\030\002 \002(\014\022\020\n\010startKey\030\003 \001(\014\022\016\n\006endKey\030\004 \001" + + "(\014\022\017\n\007offline\030\005 \001(\010\022\r\n\005split\030\006 \001(\010\"\225\001\n\017R" + + "egionSpecifier\0222\n\004type\030\001 \002(\0162$.RegionSpe" + + "cifier.RegionSpecifierType\022\r\n\005value\030\002 \002(" + + "\014\"?\n\023RegionSpecifierType\022\017\n\013REGION_NAME\020" + + "\001\022\027\n\023ENCODED_REGION_NAME\020\002\"\260\003\n\nRegionLoa" + + "d\022)\n\017regionSpecifier\030\001 \002(\0132\020.RegionSpeci" + + "fier\022\016\n\006stores\030\002 \001(\r\022\022\n\nstorefiles\030\003 \001(\r" + + "\022\037\n\027storeUncompressedSizeMB\030\004 \001(\r\022\027\n\017sto", + "refileSizeMB\030\005 \001(\r\022\026\n\016memstoreSizeMB\030\006 \001" + + "(\r\022\034\n\024storefileIndexSizeMB\030\007 \001(\r\022\031\n\021read" + + "RequestsCount\030\010 \001(\004\022\032\n\022writeRequestsCoun" + + "t\030\t \001(\004\022\032\n\022totalCompactingKVs\030\n \001(\004\022\033\n\023c" + + "urrentCompactedKVs\030\013 \001(\004\022\027\n\017rootIndexSiz" + + "eKB\030\014 \001(\r\022\036\n\026totalStaticIndexSizeKB\030\r \001(" + + "\r\022\036\n\026totalStaticBloomSizeKB\030\016 \001(\r\022\032\n\022com" + + "pleteSequenceId\030\017 \001(\004\"\372\001\n\nServerLoad\022\030\n\020" + + "numberOfRequests\030\001 \001(\r\022\035\n\025totalNumberOfR" + + "equests\030\002 \001(\r\022\022\n\nusedHeapMB\030\003 \001(\r\022\021\n\tmax", + "HeapMB\030\004 \001(\r\022 \n\013regionLoads\030\005 \003(\0132\013.Regi" + + "onLoad\022\"\n\014coprocessors\030\006 \003(\0132\014.Coprocess" + + "or\022\027\n\017reportStartTime\030\007 \001(\004\022\025\n\rreportEnd" + + "Time\030\010 \001(\004\022\026\n\016infoServerPort\030\t \001(\r\"%\n\tTi" + + "meRange\022\014\n\004from\030\001 \001(\004\022\n\n\002to\030\002 \001(\004\"0\n\006Fil" + + "ter\022\014\n\004name\030\001 \002(\t\022\030\n\020serializedFilter\030\002 " + + "\001(\014\"x\n\010KeyValue\022\013\n\003row\030\001 \002(\014\022\016\n\006family\030\002" + + " \002(\014\022\021\n\tqualifier\030\003 \002(\014\022\021\n\ttimestamp\030\004 \001" + + "(\004\022\032\n\007keyType\030\005 \001(\0162\t.CellType\022\r\n\005value\030" + + "\006 \001(\014\"?\n\nServerName\022\020\n\010hostName\030\001 \002(\t\022\014\n", "\004port\030\002 \001(\r\022\021\n\tstartCode\030\003 \001(\004\"\033\n\013Coproc" + "essor\022\014\n\004name\030\001 \002(\t\"-\n\016NameStringPair\022\014\n" + - "\004name\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\",\n\rNameBytesP", + "\004name\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\",\n\rNameBytesP" + "air\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \001(\014\"/\n\016Byte" + "sBytesPair\022\r\n\005first\030\001 \002(\014\022\016\n\006second\030\002 \002(" + "\014\",\n\rNameInt64Pair\022\014\n\004name\030\001 \001(\t\022\r\n\005valu" + "e\030\002 \001(\003\"\255\001\n\023SnapshotDescription\022\014\n\004name\030" + "\001 \002(\t\022\r\n\005table\030\002 \001(\t\022\027\n\014creationTime\030\003 \001" + "(\003:\0010\022.\n\004type\030\004 \001(\0162\031.SnapshotDescriptio" + - "n.Type:\005FLUSH\022\017\n\007version\030\005 \001(\005\"\037\n\004Type\022\014" + + "n.Type:\005FLUSH\022\017\n\007version\030\005 \001(\005\"\037\n\004Type\022\014", "\n\010DISABLED\020\000\022\t\n\005FLUSH\020\001\"\n\n\010EmptyMsg\"\032\n\007L" + "ongMsg\022\017\n\007longMsg\030\001 \002(\003\"&\n\rBigDecimalMsg" + - "\022\025\n\rbigdecimalMsg\030\001 \002(\014*r\n\013CompareType\022\010", - "\n\004LESS\020\000\022\021\n\rLESS_OR_EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r" + - "\n\tNOT_EQUAL\020\003\022\024\n\020GREATER_OR_EQUAL\020\004\022\013\n\007G" + - "REATER\020\005\022\t\n\005NO_OP\020\006*_\n\007KeyType\022\013\n\007MINIMU" + - "M\020\000\022\007\n\003PUT\020\004\022\n\n\006DELETE\020\010\022\021\n\rDELETE_COLUM" + - "N\020\014\022\021\n\rDELETE_FAMILY\020\016\022\014\n\007MAXIMUM\020\377\001B>\n*" + - "org.apache.hadoop.hbase.protobuf.generat" + - "edB\013HBaseProtosH\001\240\001\001" + "\022\025\n\rbigdecimalMsg\030\001 \002(\014*`\n\010CellType\022\013\n\007M" + + "INIMUM\020\000\022\007\n\003PUT\020\004\022\n\n\006DELETE\020\010\022\021\n\rDELETE_" + + "COLUMN\020\014\022\021\n\rDELETE_FAMILY\020\016\022\014\n\007MAXIMUM\020\377" + + "\001*r\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_EQ" + + "UAL\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020GREAT" + + "ER_OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_OP\020\006B>\n" + + "*org.apache.hadoop.hbase.protobuf.genera" + + "tedB\013HBaseProtosH\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { public com.google.protobuf.ExtensionRegistry assignDescriptors( com.google.protobuf.Descriptors.FileDescriptor root) { descriptor = root; - internal_static_TableSchema_descriptor = + internal_static_Cell_descriptor = getDescriptor().getMessageTypes().get(0); + internal_static_Cell_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_Cell_descriptor, + new java.lang.String[] { "Row", "Family", "Qualifier", "Timestamp", "CellType", "Value", }, + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.class, + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Cell.Builder.class); + internal_static_TableSchema_descriptor = + getDescriptor().getMessageTypes().get(1); internal_static_TableSchema_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_TableSchema_descriptor, @@ -12999,7 +13737,7 @@ public final class HBaseProtos { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder.class); internal_static_ColumnFamilySchema_descriptor = - getDescriptor().getMessageTypes().get(1); + getDescriptor().getMessageTypes().get(2); internal_static_ColumnFamilySchema_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ColumnFamilySchema_descriptor, @@ -13007,7 +13745,7 @@ public final class HBaseProtos { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder.class); internal_static_RegionInfo_descriptor = - getDescriptor().getMessageTypes().get(2); + getDescriptor().getMessageTypes().get(3); internal_static_RegionInfo_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_RegionInfo_descriptor, @@ -13015,7 +13753,7 @@ public final class HBaseProtos { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder.class); internal_static_RegionSpecifier_descriptor = - getDescriptor().getMessageTypes().get(3); + getDescriptor().getMessageTypes().get(4); internal_static_RegionSpecifier_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_RegionSpecifier_descriptor, @@ -13023,7 +13761,7 @@ public final class HBaseProtos { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder.class); internal_static_RegionLoad_descriptor = - getDescriptor().getMessageTypes().get(4); + getDescriptor().getMessageTypes().get(5); internal_static_RegionLoad_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_RegionLoad_descriptor, @@ -13031,7 +13769,7 @@ public final class HBaseProtos { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionLoad.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionLoad.Builder.class); internal_static_ServerLoad_descriptor = - getDescriptor().getMessageTypes().get(5); + getDescriptor().getMessageTypes().get(6); internal_static_ServerLoad_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ServerLoad_descriptor, @@ -13039,7 +13777,7 @@ public final class HBaseProtos { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerLoad.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerLoad.Builder.class); internal_static_TimeRange_descriptor = - getDescriptor().getMessageTypes().get(6); + getDescriptor().getMessageTypes().get(7); internal_static_TimeRange_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_TimeRange_descriptor, @@ -13047,7 +13785,7 @@ public final class HBaseProtos { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.Builder.class); internal_static_Filter_descriptor = - getDescriptor().getMessageTypes().get(7); + getDescriptor().getMessageTypes().get(8); internal_static_Filter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_Filter_descriptor, @@ -13055,7 +13793,7 @@ public final class HBaseProtos { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.Builder.class); internal_static_KeyValue_descriptor = - getDescriptor().getMessageTypes().get(8); + getDescriptor().getMessageTypes().get(9); internal_static_KeyValue_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_KeyValue_descriptor, @@ -13063,7 +13801,7 @@ public final class HBaseProtos { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder.class); internal_static_ServerName_descriptor = - getDescriptor().getMessageTypes().get(9); + getDescriptor().getMessageTypes().get(10); internal_static_ServerName_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ServerName_descriptor, @@ -13071,7 +13809,7 @@ public final class HBaseProtos { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder.class); internal_static_Coprocessor_descriptor = - getDescriptor().getMessageTypes().get(10); + getDescriptor().getMessageTypes().get(11); internal_static_Coprocessor_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_Coprocessor_descriptor, @@ -13079,7 +13817,7 @@ public final class HBaseProtos { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor.Builder.class); internal_static_NameStringPair_descriptor = - getDescriptor().getMessageTypes().get(11); + getDescriptor().getMessageTypes().get(12); internal_static_NameStringPair_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_NameStringPair_descriptor, @@ -13087,7 +13825,7 @@ public final class HBaseProtos { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair.Builder.class); internal_static_NameBytesPair_descriptor = - getDescriptor().getMessageTypes().get(12); + getDescriptor().getMessageTypes().get(13); internal_static_NameBytesPair_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_NameBytesPair_descriptor, @@ -13095,7 +13833,7 @@ public final class HBaseProtos { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder.class); internal_static_BytesBytesPair_descriptor = - getDescriptor().getMessageTypes().get(13); + getDescriptor().getMessageTypes().get(14); internal_static_BytesBytesPair_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_BytesBytesPair_descriptor, @@ -13103,7 +13841,7 @@ public final class HBaseProtos { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder.class); internal_static_NameInt64Pair_descriptor = - getDescriptor().getMessageTypes().get(14); + getDescriptor().getMessageTypes().get(15); internal_static_NameInt64Pair_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_NameInt64Pair_descriptor, @@ -13111,7 +13849,7 @@ public final class HBaseProtos { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameInt64Pair.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameInt64Pair.Builder.class); internal_static_SnapshotDescription_descriptor = - getDescriptor().getMessageTypes().get(15); + getDescriptor().getMessageTypes().get(16); internal_static_SnapshotDescription_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_SnapshotDescription_descriptor, @@ -13119,7 +13857,7 @@ public final class HBaseProtos { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Builder.class); internal_static_EmptyMsg_descriptor = - getDescriptor().getMessageTypes().get(16); + getDescriptor().getMessageTypes().get(17); internal_static_EmptyMsg_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_EmptyMsg_descriptor, @@ -13127,7 +13865,7 @@ public final class HBaseProtos { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.EmptyMsg.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.EmptyMsg.Builder.class); internal_static_LongMsg_descriptor = - getDescriptor().getMessageTypes().get(17); + getDescriptor().getMessageTypes().get(18); internal_static_LongMsg_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_LongMsg_descriptor, @@ -13135,7 +13873,7 @@ public final class HBaseProtos { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LongMsg.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LongMsg.Builder.class); internal_static_BigDecimalMsg_descriptor = - getDescriptor().getMessageTypes().get(18); + getDescriptor().getMessageTypes().get(19); internal_static_BigDecimalMsg_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_BigDecimalMsg_descriptor, diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MultiRowMutation.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MultiRowMutation.java index f619f9c..3227635 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MultiRowMutation.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MultiRowMutation.java @@ -11,14 +11,14 @@ public final class MultiRowMutation { public interface MultiMutateRequestOrBuilder extends com.google.protobuf.MessageOrBuilder { - // repeated .Mutate mutationRequest = 1; - java.util.List + // repeated .MutationProto mutationRequest = 1; + java.util.List 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 + java.util.List 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 mutationRequest_; - public java.util.List getMutationRequestList() { + private java.util.List mutationRequest_; + public java.util.List getMutationRequestList() { return mutationRequest_; } - public java.util.List + public java.util.List 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 mutationRequest_ = + // repeated .MutationProto mutationRequest = 1; + private java.util.List mutationRequest_ = java.util.Collections.emptyList(); private void ensureMutationRequestIsMutable() { if (!((bitField0_ & 0x00000001) == 0x00000001)) { - mutationRequest_ = new java.util.ArrayList(mutationRequest_); + mutationRequest_ = new java.util.ArrayList(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 getMutationRequestList() { + public java.util.List 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 values) { + java.lang.Iterable 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 + public java.util.List 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 + public java.util.List 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() { diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java index d900c0a..b94ef49 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java @@ -538,6 +538,14 @@ public final class RPCProtos { // optional string protocol = 2 [default = "org.apache.hadoop.hbase.client.ClientProtocol"]; boolean hasProtocol(); String getProtocol(); + + // optional string cellBlockCodecClass = 3 [default = "org.apache.hadoop.hbase.codec.KeyValueCodec"]; + boolean hasCellBlockCodecClass(); + String getCellBlockCodecClass(); + + // optional string cellBlockCompressorClass = 4; + boolean hasCellBlockCompressorClass(); + String getCellBlockCompressorClass(); } public static final class ConnectionHeader extends com.google.protobuf.GeneratedMessage @@ -613,9 +621,75 @@ public final class RPCProtos { } } + // optional string cellBlockCodecClass = 3 [default = "org.apache.hadoop.hbase.codec.KeyValueCodec"]; + public static final int CELLBLOCKCODECCLASS_FIELD_NUMBER = 3; + private java.lang.Object cellBlockCodecClass_; + public boolean hasCellBlockCodecClass() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getCellBlockCodecClass() { + java.lang.Object ref = cellBlockCodecClass_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + cellBlockCodecClass_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getCellBlockCodecClassBytes() { + java.lang.Object ref = cellBlockCodecClass_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + cellBlockCodecClass_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // optional string cellBlockCompressorClass = 4; + public static final int CELLBLOCKCOMPRESSORCLASS_FIELD_NUMBER = 4; + private java.lang.Object cellBlockCompressorClass_; + public boolean hasCellBlockCompressorClass() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public String getCellBlockCompressorClass() { + java.lang.Object ref = cellBlockCompressorClass_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + cellBlockCompressorClass_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getCellBlockCompressorClassBytes() { + java.lang.Object ref = cellBlockCompressorClass_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + cellBlockCompressorClass_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + private void initFields() { userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance(); protocol_ = "org.apache.hadoop.hbase.client.ClientProtocol"; + cellBlockCodecClass_ = "org.apache.hadoop.hbase.codec.KeyValueCodec"; + cellBlockCompressorClass_ = ""; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -641,6 +715,12 @@ public final class RPCProtos { if (((bitField0_ & 0x00000002) == 0x00000002)) { output.writeBytes(2, getProtocolBytes()); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, getCellBlockCodecClassBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBytes(4, getCellBlockCompressorClassBytes()); + } getUnknownFields().writeTo(output); } @@ -658,6 +738,14 @@ public final class RPCProtos { size += com.google.protobuf.CodedOutputStream .computeBytesSize(2, getProtocolBytes()); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(3, getCellBlockCodecClassBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(4, getCellBlockCompressorClassBytes()); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -691,6 +779,16 @@ public final class RPCProtos { result = result && getProtocol() .equals(other.getProtocol()); } + result = result && (hasCellBlockCodecClass() == other.hasCellBlockCodecClass()); + if (hasCellBlockCodecClass()) { + result = result && getCellBlockCodecClass() + .equals(other.getCellBlockCodecClass()); + } + result = result && (hasCellBlockCompressorClass() == other.hasCellBlockCompressorClass()); + if (hasCellBlockCompressorClass()) { + result = result && getCellBlockCompressorClass() + .equals(other.getCellBlockCompressorClass()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -708,6 +806,14 @@ public final class RPCProtos { hash = (37 * hash) + PROTOCOL_FIELD_NUMBER; hash = (53 * hash) + getProtocol().hashCode(); } + if (hasCellBlockCodecClass()) { + hash = (37 * hash) + CELLBLOCKCODECCLASS_FIELD_NUMBER; + hash = (53 * hash) + getCellBlockCodecClass().hashCode(); + } + if (hasCellBlockCompressorClass()) { + hash = (37 * hash) + CELLBLOCKCOMPRESSORCLASS_FIELD_NUMBER; + hash = (53 * hash) + getCellBlockCompressorClass().hashCode(); + } hash = (29 * hash) + getUnknownFields().hashCode(); return hash; } @@ -833,6 +939,10 @@ public final class RPCProtos { bitField0_ = (bitField0_ & ~0x00000001); protocol_ = "org.apache.hadoop.hbase.client.ClientProtocol"; bitField0_ = (bitField0_ & ~0x00000002); + cellBlockCodecClass_ = "org.apache.hadoop.hbase.codec.KeyValueCodec"; + bitField0_ = (bitField0_ & ~0x00000004); + cellBlockCompressorClass_ = ""; + bitField0_ = (bitField0_ & ~0x00000008); return this; } @@ -883,6 +993,14 @@ public final class RPCProtos { to_bitField0_ |= 0x00000002; } result.protocol_ = protocol_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.cellBlockCodecClass_ = cellBlockCodecClass_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.cellBlockCompressorClass_ = cellBlockCompressorClass_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -905,6 +1023,12 @@ public final class RPCProtos { if (other.hasProtocol()) { setProtocol(other.getProtocol()); } + if (other.hasCellBlockCodecClass()) { + setCellBlockCodecClass(other.getCellBlockCodecClass()); + } + if (other.hasCellBlockCompressorClass()) { + setCellBlockCompressorClass(other.getCellBlockCompressorClass()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -956,6 +1080,16 @@ public final class RPCProtos { protocol_ = input.readBytes(); break; } + case 26: { + bitField0_ |= 0x00000004; + cellBlockCodecClass_ = input.readBytes(); + break; + } + case 34: { + bitField0_ |= 0x00000008; + cellBlockCompressorClass_ = input.readBytes(); + break; + } } } } @@ -1088,6 +1222,78 @@ public final class RPCProtos { onChanged(); } + // optional string cellBlockCodecClass = 3 [default = "org.apache.hadoop.hbase.codec.KeyValueCodec"]; + private java.lang.Object cellBlockCodecClass_ = "org.apache.hadoop.hbase.codec.KeyValueCodec"; + public boolean hasCellBlockCodecClass() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getCellBlockCodecClass() { + java.lang.Object ref = cellBlockCodecClass_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + cellBlockCodecClass_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setCellBlockCodecClass(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + cellBlockCodecClass_ = value; + onChanged(); + return this; + } + public Builder clearCellBlockCodecClass() { + bitField0_ = (bitField0_ & ~0x00000004); + cellBlockCodecClass_ = getDefaultInstance().getCellBlockCodecClass(); + onChanged(); + return this; + } + void setCellBlockCodecClass(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000004; + cellBlockCodecClass_ = value; + onChanged(); + } + + // optional string cellBlockCompressorClass = 4; + private java.lang.Object cellBlockCompressorClass_ = ""; + public boolean hasCellBlockCompressorClass() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public String getCellBlockCompressorClass() { + java.lang.Object ref = cellBlockCompressorClass_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + cellBlockCompressorClass_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setCellBlockCompressorClass(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + cellBlockCompressorClass_ = value; + onChanged(); + return this; + } + public Builder clearCellBlockCompressorClass() { + bitField0_ = (bitField0_ & ~0x00000008); + cellBlockCompressorClass_ = getDefaultInstance().getCellBlockCompressorClass(); + onChanged(); + return this; + } + void setCellBlockCompressorClass(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000008; + cellBlockCompressorClass_ = value; + onChanged(); + } + // @@protoc_insertion_point(builder_scope:ConnectionHeader) } @@ -1099,83 +1305,60 @@ public final class RPCProtos { // @@protoc_insertion_point(class_scope:ConnectionHeader) } - public interface RpcRequestHeaderOrBuilder + public interface CellBlockMetaOrBuilder extends com.google.protobuf.MessageOrBuilder { - // required uint32 callId = 1; - boolean hasCallId(); - int getCallId(); - - // optional .RPCTInfo tinfo = 2; - boolean hasTinfo(); - org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo getTinfo(); - org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfoOrBuilder getTinfoOrBuilder(); + // optional uint32 length = 1; + boolean hasLength(); + int getLength(); } - public static final class RpcRequestHeader extends + public static final class CellBlockMeta extends com.google.protobuf.GeneratedMessage - implements RpcRequestHeaderOrBuilder { - // Use RpcRequestHeader.newBuilder() to construct. - private RpcRequestHeader(Builder builder) { + implements CellBlockMetaOrBuilder { + // Use CellBlockMeta.newBuilder() to construct. + private CellBlockMeta(Builder builder) { super(builder); } - private RpcRequestHeader(boolean noInit) {} + private CellBlockMeta(boolean noInit) {} - private static final RpcRequestHeader defaultInstance; - public static RpcRequestHeader getDefaultInstance() { + private static final CellBlockMeta defaultInstance; + public static CellBlockMeta getDefaultInstance() { return defaultInstance; } - public RpcRequestHeader getDefaultInstanceForType() { + public CellBlockMeta getDefaultInstanceForType() { return defaultInstance; } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcRequestHeader_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_CellBlockMeta_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcRequestHeader_fieldAccessorTable; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_CellBlockMeta_fieldAccessorTable; } private int bitField0_; - // required uint32 callId = 1; - public static final int CALLID_FIELD_NUMBER = 1; - private int callId_; - public boolean hasCallId() { + // optional uint32 length = 1; + public static final int LENGTH_FIELD_NUMBER = 1; + private int length_; + public boolean hasLength() { return ((bitField0_ & 0x00000001) == 0x00000001); } - public int getCallId() { - return callId_; - } - - // optional .RPCTInfo tinfo = 2; - public static final int TINFO_FIELD_NUMBER = 2; - private org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo tinfo_; - public boolean hasTinfo() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo getTinfo() { - return tinfo_; - } - public org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfoOrBuilder getTinfoOrBuilder() { - return tinfo_; + public int getLength() { + return length_; } private void initFields() { - callId_ = 0; - tinfo_ = org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.getDefaultInstance(); + length_ = 0; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - if (!hasCallId()) { - memoizedIsInitialized = 0; - return false; - } memoizedIsInitialized = 1; return true; } @@ -1184,10 +1367,7 @@ public final class RPCProtos { throws java.io.IOException { getSerializedSize(); if (((bitField0_ & 0x00000001) == 0x00000001)) { - output.writeUInt32(1, callId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeMessage(2, tinfo_); + output.writeUInt32(1, length_); } getUnknownFields().writeTo(output); } @@ -1200,11 +1380,7 @@ public final class RPCProtos { size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeUInt32Size(1, callId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, tinfo_); + .computeUInt32Size(1, length_); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; @@ -1223,21 +1399,16 @@ public final class RPCProtos { if (obj == this) { return true; } - if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader)) { + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta)) { return super.equals(obj); } - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader other = (org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader) obj; + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta other = (org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta) obj; boolean result = true; - result = result && (hasCallId() == other.hasCallId()); - if (hasCallId()) { - result = result && (getCallId() - == other.getCallId()); - } - result = result && (hasTinfo() == other.hasTinfo()); - if (hasTinfo()) { - result = result && getTinfo() - .equals(other.getTinfo()); + result = result && (hasLength() == other.hasLength()); + if (hasLength()) { + result = result && (getLength() + == other.getLength()); } result = result && getUnknownFields().equals(other.getUnknownFields()); @@ -1248,53 +1419,49 @@ public final class RPCProtos { public int hashCode() { int hash = 41; hash = (19 * hash) + getDescriptorForType().hashCode(); - if (hasCallId()) { - hash = (37 * hash) + CALLID_FIELD_NUMBER; - hash = (53 * hash) + getCallId(); - } - if (hasTinfo()) { - hash = (37 * hash) + TINFO_FIELD_NUMBER; - hash = (53 * hash) + getTinfo().hashCode(); + if (hasLength()) { + hash = (37 * hash) + LENGTH_FIELD_NUMBER; + hash = (53 * hash) + getLength(); } hash = (29 * hash) + getUnknownFields().hashCode(); return hash; } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader parseFrom(byte[] data) + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader parseFrom(java.io.InputStream input) + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta parseFrom(java.io.InputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { return newBuilder().mergeFrom(input, extensionRegistry) .buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader parseDelimitedFrom(java.io.InputStream input) + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -1303,7 +1470,7 @@ public final class RPCProtos { return null; } } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader parseDelimitedFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -1314,12 +1481,12 @@ public final class RPCProtos { return null; } } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -1329,7 +1496,7 @@ public final class RPCProtos { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader prototype) { + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } @@ -1342,18 +1509,18 @@ public final class RPCProtos { } public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder - implements org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeaderOrBuilder { + implements org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMetaOrBuilder { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcRequestHeader_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_CellBlockMeta_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcRequestHeader_fieldAccessorTable; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_CellBlockMeta_fieldAccessorTable; } - // Construct using org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader.newBuilder() + // Construct using org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -1364,7 +1531,6 @@ public final class RPCProtos { } private void maybeForceBuilderInitialization() { if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { - getTinfoFieldBuilder(); } } private static Builder create() { @@ -1373,14 +1539,8 @@ public final class RPCProtos { public Builder clear() { super.clear(); - callId_ = 0; + length_ = 0; bitField0_ = (bitField0_ & ~0x00000001); - if (tinfoBuilder_ == null) { - tinfo_ = org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.getDefaultInstance(); - } else { - tinfoBuilder_.clear(); - } - bitField0_ = (bitField0_ & ~0x00000002); return this; } @@ -1390,24 +1550,24 @@ public final class RPCProtos { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader.getDescriptor(); + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.getDescriptor(); } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader getDefaultInstanceForType() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader.getDefaultInstance(); + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.getDefaultInstance(); } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader build() { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader result = buildPartial(); + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta build() { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } return result; } - private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader buildParsed() + private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader result = buildPartial(); + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); @@ -1415,53 +1575,38 @@ public final class RPCProtos { return result; } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader buildPartial() { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader result = new org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader(this); + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta result = new org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta(this); int from_bitField0_ = bitField0_; int to_bitField0_ = 0; if (((from_bitField0_ & 0x00000001) == 0x00000001)) { to_bitField0_ |= 0x00000001; } - result.callId_ = callId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - if (tinfoBuilder_ == null) { - result.tinfo_ = tinfo_; - } else { - result.tinfo_ = tinfoBuilder_.build(); - } + result.length_ = length_; result.bitField0_ = to_bitField0_; onBuilt(); return result; } public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader) { - return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader)other); + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta)other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader other) { - if (other == org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader.getDefaultInstance()) return this; - if (other.hasCallId()) { - setCallId(other.getCallId()); - } - if (other.hasTinfo()) { - mergeTinfo(other.getTinfo()); + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.getDefaultInstance()) return this; + if (other.hasLength()) { + setLength(other.getLength()); } this.mergeUnknownFields(other.getUnknownFields()); return this; } public final boolean isInitialized() { - if (!hasCallId()) { - - return false; - } return true; } @@ -1490,16 +1635,7 @@ public final class RPCProtos { } case 8: { bitField0_ |= 0x00000001; - callId_ = input.readUInt32(); - break; - } - case 18: { - org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.newBuilder(); - if (hasTinfo()) { - subBuilder.mergeFrom(getTinfo()); - } - input.readMessage(subBuilder, extensionRegistry); - setTinfo(subBuilder.buildPartial()); + length_ = input.readUInt32(); break; } } @@ -1508,180 +1644,98 @@ public final class RPCProtos { private int bitField0_; - // required uint32 callId = 1; - private int callId_ ; - public boolean hasCallId() { + // optional uint32 length = 1; + private int length_ ; + public boolean hasLength() { return ((bitField0_ & 0x00000001) == 0x00000001); } - public int getCallId() { - return callId_; + public int getLength() { + return length_; } - public Builder setCallId(int value) { + public Builder setLength(int value) { bitField0_ |= 0x00000001; - callId_ = value; + length_ = value; onChanged(); return this; } - public Builder clearCallId() { + public Builder clearLength() { bitField0_ = (bitField0_ & ~0x00000001); - callId_ = 0; + length_ = 0; onChanged(); return this; } - // optional .RPCTInfo tinfo = 2; - private org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo tinfo_ = org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.getDefaultInstance(); - private com.google.protobuf.SingleFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo, org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfoOrBuilder> tinfoBuilder_; - public boolean hasTinfo() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo getTinfo() { - if (tinfoBuilder_ == null) { - return tinfo_; - } else { - return tinfoBuilder_.getMessage(); - } - } - public Builder setTinfo(org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo value) { - if (tinfoBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - tinfo_ = value; - onChanged(); - } else { - tinfoBuilder_.setMessage(value); - } - bitField0_ |= 0x00000002; - return this; - } - public Builder setTinfo( - org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.Builder builderForValue) { - if (tinfoBuilder_ == null) { - tinfo_ = builderForValue.build(); - onChanged(); - } else { - tinfoBuilder_.setMessage(builderForValue.build()); - } - bitField0_ |= 0x00000002; - return this; - } - public Builder mergeTinfo(org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo value) { - if (tinfoBuilder_ == null) { - if (((bitField0_ & 0x00000002) == 0x00000002) && - tinfo_ != org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.getDefaultInstance()) { - tinfo_ = - org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.newBuilder(tinfo_).mergeFrom(value).buildPartial(); - } else { - tinfo_ = value; - } - onChanged(); - } else { - tinfoBuilder_.mergeFrom(value); - } - bitField0_ |= 0x00000002; - return this; - } - public Builder clearTinfo() { - if (tinfoBuilder_ == null) { - tinfo_ = org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.getDefaultInstance(); - onChanged(); - } else { - tinfoBuilder_.clear(); - } - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - public org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.Builder getTinfoBuilder() { - bitField0_ |= 0x00000002; - onChanged(); - return getTinfoFieldBuilder().getBuilder(); - } - public org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfoOrBuilder getTinfoOrBuilder() { - if (tinfoBuilder_ != null) { - return tinfoBuilder_.getMessageOrBuilder(); - } else { - return tinfo_; - } - } - private com.google.protobuf.SingleFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo, org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfoOrBuilder> - getTinfoFieldBuilder() { - if (tinfoBuilder_ == null) { - tinfoBuilder_ = new com.google.protobuf.SingleFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo, org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfoOrBuilder>( - tinfo_, - getParentForChildren(), - isClean()); - tinfo_ = null; - } - return tinfoBuilder_; - } - - // @@protoc_insertion_point(builder_scope:RpcRequestHeader) + // @@protoc_insertion_point(builder_scope:CellBlockMeta) } static { - defaultInstance = new RpcRequestHeader(true); + defaultInstance = new CellBlockMeta(true); defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:RpcRequestHeader) + // @@protoc_insertion_point(class_scope:CellBlockMeta) } - public interface RpcRequestBodyOrBuilder + public interface ExceptionResponseOrBuilder extends com.google.protobuf.MessageOrBuilder { - // required string methodName = 1; - boolean hasMethodName(); - String getMethodName(); + // optional string exceptionClassName = 1; + boolean hasExceptionClassName(); + String getExceptionClassName(); + + // optional string stackTrace = 2; + boolean hasStackTrace(); + String getStackTrace(); + + // optional string hostname = 3; + boolean hasHostname(); + String getHostname(); - // optional bytes request = 2; - boolean hasRequest(); - com.google.protobuf.ByteString getRequest(); + // optional int32 port = 4; + boolean hasPort(); + int getPort(); - // optional string requestClassName = 4; - boolean hasRequestClassName(); - String getRequestClassName(); + // optional bool doNotRetry = 5; + boolean hasDoNotRetry(); + boolean getDoNotRetry(); } - public static final class RpcRequestBody extends + public static final class ExceptionResponse extends com.google.protobuf.GeneratedMessage - implements RpcRequestBodyOrBuilder { - // Use RpcRequestBody.newBuilder() to construct. - private RpcRequestBody(Builder builder) { + implements ExceptionResponseOrBuilder { + // Use ExceptionResponse.newBuilder() to construct. + private ExceptionResponse(Builder builder) { super(builder); } - private RpcRequestBody(boolean noInit) {} + private ExceptionResponse(boolean noInit) {} - private static final RpcRequestBody defaultInstance; - public static RpcRequestBody getDefaultInstance() { + private static final ExceptionResponse defaultInstance; + public static ExceptionResponse getDefaultInstance() { return defaultInstance; } - public RpcRequestBody getDefaultInstanceForType() { + public ExceptionResponse getDefaultInstanceForType() { return defaultInstance; } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcRequestBody_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_ExceptionResponse_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcRequestBody_fieldAccessorTable; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_ExceptionResponse_fieldAccessorTable; } private int bitField0_; - // required string methodName = 1; - public static final int METHODNAME_FIELD_NUMBER = 1; - private java.lang.Object methodName_; - public boolean hasMethodName() { + // optional string exceptionClassName = 1; + public static final int EXCEPTIONCLASSNAME_FIELD_NUMBER = 1; + private java.lang.Object exceptionClassName_; + public boolean hasExceptionClassName() { return ((bitField0_ & 0x00000001) == 0x00000001); } - public String getMethodName() { - java.lang.Object ref = methodName_; + public String getExceptionClassName() { + java.lang.Object ref = exceptionClassName_; if (ref instanceof String) { return (String) ref; } else { @@ -1689,41 +1743,31 @@ public final class RPCProtos { (com.google.protobuf.ByteString) ref; String s = bs.toStringUtf8(); if (com.google.protobuf.Internal.isValidUtf8(bs)) { - methodName_ = s; + exceptionClassName_ = s; } return s; } } - private com.google.protobuf.ByteString getMethodNameBytes() { - java.lang.Object ref = methodName_; + private com.google.protobuf.ByteString getExceptionClassNameBytes() { + java.lang.Object ref = exceptionClassName_; if (ref instanceof String) { com.google.protobuf.ByteString b = com.google.protobuf.ByteString.copyFromUtf8((String) ref); - methodName_ = b; + exceptionClassName_ = b; return b; } else { return (com.google.protobuf.ByteString) ref; } } - // optional bytes request = 2; - public static final int REQUEST_FIELD_NUMBER = 2; - private com.google.protobuf.ByteString request_; - public boolean hasRequest() { + // optional string stackTrace = 2; + public static final int STACKTRACE_FIELD_NUMBER = 2; + private java.lang.Object stackTrace_; + public boolean hasStackTrace() { return ((bitField0_ & 0x00000002) == 0x00000002); } - public com.google.protobuf.ByteString getRequest() { - return request_; - } - - // optional string requestClassName = 4; - public static final int REQUESTCLASSNAME_FIELD_NUMBER = 4; - private java.lang.Object requestClassName_; - public boolean hasRequestClassName() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public String getRequestClassName() { - java.lang.Object ref = requestClassName_; + public String getStackTrace() { + java.lang.Object ref = stackTrace_; if (ref instanceof String) { return (String) ref; } else { @@ -1731,37 +1775,87 @@ public final class RPCProtos { (com.google.protobuf.ByteString) ref; String s = bs.toStringUtf8(); if (com.google.protobuf.Internal.isValidUtf8(bs)) { - requestClassName_ = s; + stackTrace_ = s; } return s; } } - private com.google.protobuf.ByteString getRequestClassNameBytes() { - java.lang.Object ref = requestClassName_; - if (ref instanceof String) { + private com.google.protobuf.ByteString getStackTraceBytes() { + java.lang.Object ref = stackTrace_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + stackTrace_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // optional string hostname = 3; + public static final int HOSTNAME_FIELD_NUMBER = 3; + private java.lang.Object hostname_; + public boolean hasHostname() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getHostname() { + java.lang.Object ref = hostname_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + hostname_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getHostnameBytes() { + java.lang.Object ref = hostname_; + if (ref instanceof String) { com.google.protobuf.ByteString b = com.google.protobuf.ByteString.copyFromUtf8((String) ref); - requestClassName_ = b; + hostname_ = b; return b; } else { return (com.google.protobuf.ByteString) ref; } } + // optional int32 port = 4; + public static final int PORT_FIELD_NUMBER = 4; + private int port_; + public boolean hasPort() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public int getPort() { + return port_; + } + + // optional bool doNotRetry = 5; + public static final int DONOTRETRY_FIELD_NUMBER = 5; + private boolean doNotRetry_; + public boolean hasDoNotRetry() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public boolean getDoNotRetry() { + return doNotRetry_; + } + private void initFields() { - methodName_ = ""; - request_ = com.google.protobuf.ByteString.EMPTY; - requestClassName_ = ""; + exceptionClassName_ = ""; + stackTrace_ = ""; + hostname_ = ""; + port_ = 0; + doNotRetry_ = false; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - if (!hasMethodName()) { - memoizedIsInitialized = 0; - return false; - } memoizedIsInitialized = 1; return true; } @@ -1770,13 +1864,19 @@ public final class RPCProtos { throws java.io.IOException { getSerializedSize(); if (((bitField0_ & 0x00000001) == 0x00000001)) { - output.writeBytes(1, getMethodNameBytes()); + output.writeBytes(1, getExceptionClassNameBytes()); } if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeBytes(2, request_); + output.writeBytes(2, getStackTraceBytes()); } if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeBytes(4, getRequestClassNameBytes()); + output.writeBytes(3, getHostnameBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeInt32(4, port_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBool(5, doNotRetry_); } getUnknownFields().writeTo(output); } @@ -1789,15 +1889,23 @@ public final class RPCProtos { size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeBytesSize(1, getMethodNameBytes()); + .computeBytesSize(1, getExceptionClassNameBytes()); } if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeBytesSize(2, request_); + .computeBytesSize(2, getStackTraceBytes()); } if (((bitField0_ & 0x00000004) == 0x00000004)) { size += com.google.protobuf.CodedOutputStream - .computeBytesSize(4, getRequestClassNameBytes()); + .computeBytesSize(3, getHostnameBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(4, port_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += com.google.protobuf.CodedOutputStream + .computeBoolSize(5, doNotRetry_); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; @@ -1816,26 +1924,36 @@ public final class RPCProtos { if (obj == this) { return true; } - if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody)) { + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse)) { return super.equals(obj); } - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody other = (org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody) obj; + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse other = (org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse) obj; boolean result = true; - result = result && (hasMethodName() == other.hasMethodName()); - if (hasMethodName()) { - result = result && getMethodName() - .equals(other.getMethodName()); + result = result && (hasExceptionClassName() == other.hasExceptionClassName()); + if (hasExceptionClassName()) { + result = result && getExceptionClassName() + .equals(other.getExceptionClassName()); + } + result = result && (hasStackTrace() == other.hasStackTrace()); + if (hasStackTrace()) { + result = result && getStackTrace() + .equals(other.getStackTrace()); } - result = result && (hasRequest() == other.hasRequest()); - if (hasRequest()) { - result = result && getRequest() - .equals(other.getRequest()); + result = result && (hasHostname() == other.hasHostname()); + if (hasHostname()) { + result = result && getHostname() + .equals(other.getHostname()); } - result = result && (hasRequestClassName() == other.hasRequestClassName()); - if (hasRequestClassName()) { - result = result && getRequestClassName() - .equals(other.getRequestClassName()); + result = result && (hasPort() == other.hasPort()); + if (hasPort()) { + result = result && (getPort() + == other.getPort()); + } + result = result && (hasDoNotRetry() == other.hasDoNotRetry()); + if (hasDoNotRetry()) { + result = result && (getDoNotRetry() + == other.getDoNotRetry()); } result = result && getUnknownFields().equals(other.getUnknownFields()); @@ -1846,57 +1964,65 @@ public final class RPCProtos { public int hashCode() { int hash = 41; hash = (19 * hash) + getDescriptorForType().hashCode(); - if (hasMethodName()) { - hash = (37 * hash) + METHODNAME_FIELD_NUMBER; - hash = (53 * hash) + getMethodName().hashCode(); + if (hasExceptionClassName()) { + hash = (37 * hash) + EXCEPTIONCLASSNAME_FIELD_NUMBER; + hash = (53 * hash) + getExceptionClassName().hashCode(); + } + if (hasStackTrace()) { + hash = (37 * hash) + STACKTRACE_FIELD_NUMBER; + hash = (53 * hash) + getStackTrace().hashCode(); } - if (hasRequest()) { - hash = (37 * hash) + REQUEST_FIELD_NUMBER; - hash = (53 * hash) + getRequest().hashCode(); + if (hasHostname()) { + hash = (37 * hash) + HOSTNAME_FIELD_NUMBER; + hash = (53 * hash) + getHostname().hashCode(); } - if (hasRequestClassName()) { - hash = (37 * hash) + REQUESTCLASSNAME_FIELD_NUMBER; - hash = (53 * hash) + getRequestClassName().hashCode(); + if (hasPort()) { + hash = (37 * hash) + PORT_FIELD_NUMBER; + hash = (53 * hash) + getPort(); + } + if (hasDoNotRetry()) { + hash = (37 * hash) + DONOTRETRY_FIELD_NUMBER; + hash = (53 * hash) + hashBoolean(getDoNotRetry()); } hash = (29 * hash) + getUnknownFields().hashCode(); return hash; } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody parseFrom(byte[] data) + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody parseFrom(java.io.InputStream input) + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse parseFrom(java.io.InputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { return newBuilder().mergeFrom(input, extensionRegistry) .buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody parseDelimitedFrom(java.io.InputStream input) + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -1905,7 +2031,7 @@ public final class RPCProtos { return null; } } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody parseDelimitedFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -1916,12 +2042,12 @@ public final class RPCProtos { return null; } } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -1931,7 +2057,7 @@ public final class RPCProtos { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody prototype) { + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } @@ -1944,18 +2070,18 @@ public final class RPCProtos { } public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder - implements org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBodyOrBuilder { + implements org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponseOrBuilder { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcRequestBody_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_ExceptionResponse_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcRequestBody_fieldAccessorTable; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_ExceptionResponse_fieldAccessorTable; } - // Construct using org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody.newBuilder() + // Construct using org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -1974,12 +2100,16 @@ public final class RPCProtos { public Builder clear() { super.clear(); - methodName_ = ""; + exceptionClassName_ = ""; bitField0_ = (bitField0_ & ~0x00000001); - request_ = com.google.protobuf.ByteString.EMPTY; + stackTrace_ = ""; bitField0_ = (bitField0_ & ~0x00000002); - requestClassName_ = ""; + hostname_ = ""; bitField0_ = (bitField0_ & ~0x00000004); + port_ = 0; + bitField0_ = (bitField0_ & ~0x00000008); + doNotRetry_ = false; + bitField0_ = (bitField0_ & ~0x00000010); return this; } @@ -1989,24 +2119,24 @@ public final class RPCProtos { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody.getDescriptor(); + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse.getDescriptor(); } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody getDefaultInstanceForType() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody.getDefaultInstance(); + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse.getDefaultInstance(); } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody build() { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody result = buildPartial(); + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse build() { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } return result; } - private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody buildParsed() + private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody result = buildPartial(); + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); @@ -2014,56 +2144,66 @@ public final class RPCProtos { return result; } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody buildPartial() { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody result = new org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody(this); + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse result = new org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse(this); int from_bitField0_ = bitField0_; int to_bitField0_ = 0; if (((from_bitField0_ & 0x00000001) == 0x00000001)) { to_bitField0_ |= 0x00000001; } - result.methodName_ = methodName_; + result.exceptionClassName_ = exceptionClassName_; if (((from_bitField0_ & 0x00000002) == 0x00000002)) { to_bitField0_ |= 0x00000002; } - result.request_ = request_; + result.stackTrace_ = stackTrace_; if (((from_bitField0_ & 0x00000004) == 0x00000004)) { to_bitField0_ |= 0x00000004; } - result.requestClassName_ = requestClassName_; + result.hostname_ = hostname_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.port_ = port_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.doNotRetry_ = doNotRetry_; result.bitField0_ = to_bitField0_; onBuilt(); return result; } public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody) { - return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody)other); + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse)other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody other) { - if (other == org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody.getDefaultInstance()) return this; - if (other.hasMethodName()) { - setMethodName(other.getMethodName()); + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse.getDefaultInstance()) return this; + if (other.hasExceptionClassName()) { + setExceptionClassName(other.getExceptionClassName()); + } + if (other.hasStackTrace()) { + setStackTrace(other.getStackTrace()); } - if (other.hasRequest()) { - setRequest(other.getRequest()); + if (other.hasHostname()) { + setHostname(other.getHostname()); } - if (other.hasRequestClassName()) { - setRequestClassName(other.getRequestClassName()); + if (other.hasPort()) { + setPort(other.getPort()); + } + if (other.hasDoNotRetry()) { + setDoNotRetry(other.getDoNotRetry()); } this.mergeUnknownFields(other.getUnknownFields()); return this; } public final boolean isInitialized() { - if (!hasMethodName()) { - - return false; - } return true; } @@ -2092,17 +2232,27 @@ public final class RPCProtos { } case 10: { bitField0_ |= 0x00000001; - methodName_ = input.readBytes(); + exceptionClassName_ = input.readBytes(); break; } case 18: { bitField0_ |= 0x00000002; - request_ = input.readBytes(); + stackTrace_ = input.readBytes(); break; } - case 34: { + case 26: { bitField0_ |= 0x00000004; - requestClassName_ = input.readBytes(); + hostname_ = input.readBytes(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + port_ = input.readInt32(); + break; + } + case 40: { + bitField0_ |= 0x00000010; + doNotRetry_ = input.readBool(); break; } } @@ -2111,226 +2261,222 @@ public final class RPCProtos { private int bitField0_; - // required string methodName = 1; - private java.lang.Object methodName_ = ""; - public boolean hasMethodName() { + // optional string exceptionClassName = 1; + private java.lang.Object exceptionClassName_ = ""; + public boolean hasExceptionClassName() { return ((bitField0_ & 0x00000001) == 0x00000001); } - public String getMethodName() { - java.lang.Object ref = methodName_; + public String getExceptionClassName() { + java.lang.Object ref = exceptionClassName_; if (!(ref instanceof String)) { String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); - methodName_ = s; + exceptionClassName_ = s; return s; } else { return (String) ref; } } - public Builder setMethodName(String value) { + public Builder setExceptionClassName(String value) { if (value == null) { throw new NullPointerException(); } bitField0_ |= 0x00000001; - methodName_ = value; + exceptionClassName_ = value; onChanged(); return this; } - public Builder clearMethodName() { + public Builder clearExceptionClassName() { bitField0_ = (bitField0_ & ~0x00000001); - methodName_ = getDefaultInstance().getMethodName(); + exceptionClassName_ = getDefaultInstance().getExceptionClassName(); onChanged(); return this; } - void setMethodName(com.google.protobuf.ByteString value) { + void setExceptionClassName(com.google.protobuf.ByteString value) { bitField0_ |= 0x00000001; - methodName_ = value; + exceptionClassName_ = value; onChanged(); } - // optional bytes request = 2; - private com.google.protobuf.ByteString request_ = com.google.protobuf.ByteString.EMPTY; - public boolean hasRequest() { + // optional string stackTrace = 2; + private java.lang.Object stackTrace_ = ""; + public boolean hasStackTrace() { return ((bitField0_ & 0x00000002) == 0x00000002); } - public com.google.protobuf.ByteString getRequest() { - return request_; + public String getStackTrace() { + java.lang.Object ref = stackTrace_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + stackTrace_ = s; + return s; + } else { + return (String) ref; + } } - public Builder setRequest(com.google.protobuf.ByteString value) { + public Builder setStackTrace(String value) { if (value == null) { throw new NullPointerException(); } bitField0_ |= 0x00000002; - request_ = value; + stackTrace_ = value; onChanged(); return this; } - public Builder clearRequest() { + public Builder clearStackTrace() { bitField0_ = (bitField0_ & ~0x00000002); - request_ = getDefaultInstance().getRequest(); + stackTrace_ = getDefaultInstance().getStackTrace(); onChanged(); return this; } + void setStackTrace(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000002; + stackTrace_ = value; + onChanged(); + } - // optional string requestClassName = 4; - private java.lang.Object requestClassName_ = ""; - public boolean hasRequestClassName() { + // optional string hostname = 3; + private java.lang.Object hostname_ = ""; + public boolean hasHostname() { return ((bitField0_ & 0x00000004) == 0x00000004); } - public String getRequestClassName() { - java.lang.Object ref = requestClassName_; + public String getHostname() { + java.lang.Object ref = hostname_; if (!(ref instanceof String)) { String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); - requestClassName_ = s; + hostname_ = s; return s; } else { return (String) ref; } } - public Builder setRequestClassName(String value) { + public Builder setHostname(String value) { if (value == null) { throw new NullPointerException(); } bitField0_ |= 0x00000004; - requestClassName_ = value; + hostname_ = value; onChanged(); return this; } - public Builder clearRequestClassName() { + public Builder clearHostname() { bitField0_ = (bitField0_ & ~0x00000004); - requestClassName_ = getDefaultInstance().getRequestClassName(); + hostname_ = getDefaultInstance().getHostname(); onChanged(); return this; } - void setRequestClassName(com.google.protobuf.ByteString value) { + void setHostname(com.google.protobuf.ByteString value) { bitField0_ |= 0x00000004; - requestClassName_ = value; + hostname_ = value; + onChanged(); + } + + // optional int32 port = 4; + private int port_ ; + public boolean hasPort() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public int getPort() { + return port_; + } + public Builder setPort(int value) { + bitField0_ |= 0x00000008; + port_ = value; + onChanged(); + return this; + } + public Builder clearPort() { + bitField0_ = (bitField0_ & ~0x00000008); + port_ = 0; + onChanged(); + return this; + } + + // optional bool doNotRetry = 5; + private boolean doNotRetry_ ; + public boolean hasDoNotRetry() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public boolean getDoNotRetry() { + return doNotRetry_; + } + public Builder setDoNotRetry(boolean value) { + bitField0_ |= 0x00000010; + doNotRetry_ = value; + onChanged(); + return this; + } + public Builder clearDoNotRetry() { + bitField0_ = (bitField0_ & ~0x00000010); + doNotRetry_ = false; onChanged(); + return this; } - // @@protoc_insertion_point(builder_scope:RpcRequestBody) + // @@protoc_insertion_point(builder_scope:ExceptionResponse) } static { - defaultInstance = new RpcRequestBody(true); + defaultInstance = new ExceptionResponse(true); defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:RpcRequestBody) + // @@protoc_insertion_point(class_scope:ExceptionResponse) } - public interface RpcResponseHeaderOrBuilder + public interface RequestHeaderOrBuilder extends com.google.protobuf.MessageOrBuilder { - // required uint32 callId = 1; + // optional uint32 callId = 1; boolean hasCallId(); int getCallId(); - // required .RpcResponseHeader.Status status = 2; - boolean hasStatus(); - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Status getStatus(); + // optional .RPCTInfo traceInfo = 2; + boolean hasTraceInfo(); + org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo getTraceInfo(); + org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfoOrBuilder getTraceInfoOrBuilder(); + + // optional string methodName = 3; + boolean hasMethodName(); + String getMethodName(); + + // optional bool requestParam = 4; + boolean hasRequestParam(); + boolean getRequestParam(); + + // optional .CellBlockMeta cellBlockMeta = 5; + boolean hasCellBlockMeta(); + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta getCellBlockMeta(); + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMetaOrBuilder getCellBlockMetaOrBuilder(); } - public static final class RpcResponseHeader extends + public static final class RequestHeader extends com.google.protobuf.GeneratedMessage - implements RpcResponseHeaderOrBuilder { - // Use RpcResponseHeader.newBuilder() to construct. - private RpcResponseHeader(Builder builder) { + implements RequestHeaderOrBuilder { + // Use RequestHeader.newBuilder() to construct. + private RequestHeader(Builder builder) { super(builder); } - private RpcResponseHeader(boolean noInit) {} + private RequestHeader(boolean noInit) {} - private static final RpcResponseHeader defaultInstance; - public static RpcResponseHeader getDefaultInstance() { + private static final RequestHeader defaultInstance; + public static RequestHeader getDefaultInstance() { return defaultInstance; } - public RpcResponseHeader getDefaultInstanceForType() { + public RequestHeader getDefaultInstanceForType() { return defaultInstance; } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcResponseHeader_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RequestHeader_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcResponseHeader_fieldAccessorTable; - } - - public enum Status - implements com.google.protobuf.ProtocolMessageEnum { - SUCCESS(0, 0), - ERROR(1, 1), - FATAL(2, 2), - ; - - public static final int SUCCESS_VALUE = 0; - public static final int ERROR_VALUE = 1; - public static final int FATAL_VALUE = 2; - - - public final int getNumber() { return value; } - - public static Status valueOf(int value) { - switch (value) { - case 0: return SUCCESS; - case 1: return ERROR; - case 2: return FATAL; - default: return null; - } - } - - public static com.google.protobuf.Internal.EnumLiteMap - internalGetValueMap() { - return internalValueMap; - } - private static com.google.protobuf.Internal.EnumLiteMap - internalValueMap = - new com.google.protobuf.Internal.EnumLiteMap() { - public Status findValueByNumber(int number) { - return Status.valueOf(number); - } - }; - - public final com.google.protobuf.Descriptors.EnumValueDescriptor - getValueDescriptor() { - return getDescriptor().getValues().get(index); - } - public final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptorForType() { - return getDescriptor(); - } - public static final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.getDescriptor().getEnumTypes().get(0); - } - - private static final Status[] VALUES = { - SUCCESS, ERROR, FATAL, - }; - - public static Status valueOf( - com.google.protobuf.Descriptors.EnumValueDescriptor desc) { - if (desc.getType() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "EnumValueDescriptor is not for this type."); - } - return VALUES[desc.getIndex()]; - } - - private final int index; - private final int value; - - private Status(int index, int value) { - this.index = index; - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:RpcResponseHeader.Status) + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RequestHeader_fieldAccessorTable; } private int bitField0_; - // required uint32 callId = 1; + // optional uint32 callId = 1; public static final int CALLID_FIELD_NUMBER = 1; private int callId_; public boolean hasCallId() { @@ -2340,33 +2486,86 @@ public final class RPCProtos { return callId_; } - // required .RpcResponseHeader.Status status = 2; - public static final int STATUS_FIELD_NUMBER = 2; - private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Status status_; - public boolean hasStatus() { + // optional .RPCTInfo traceInfo = 2; + public static final int TRACEINFO_FIELD_NUMBER = 2; + private org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo traceInfo_; + public boolean hasTraceInfo() { return ((bitField0_ & 0x00000002) == 0x00000002); } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Status getStatus() { - return status_; + public org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo getTraceInfo() { + return traceInfo_; + } + public org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfoOrBuilder getTraceInfoOrBuilder() { + return traceInfo_; + } + + // optional string methodName = 3; + public static final int METHODNAME_FIELD_NUMBER = 3; + private java.lang.Object methodName_; + public boolean hasMethodName() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getMethodName() { + java.lang.Object ref = methodName_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + methodName_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getMethodNameBytes() { + java.lang.Object ref = methodName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + methodName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // optional bool requestParam = 4; + public static final int REQUESTPARAM_FIELD_NUMBER = 4; + private boolean requestParam_; + public boolean hasRequestParam() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public boolean getRequestParam() { + return requestParam_; + } + + // optional .CellBlockMeta cellBlockMeta = 5; + public static final int CELLBLOCKMETA_FIELD_NUMBER = 5; + private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta cellBlockMeta_; + public boolean hasCellBlockMeta() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta getCellBlockMeta() { + return cellBlockMeta_; + } + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMetaOrBuilder getCellBlockMetaOrBuilder() { + return cellBlockMeta_; } private void initFields() { callId_ = 0; - status_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Status.SUCCESS; + traceInfo_ = org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.getDefaultInstance(); + methodName_ = ""; + requestParam_ = false; + cellBlockMeta_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.getDefaultInstance(); } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - if (!hasCallId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasStatus()) { - memoizedIsInitialized = 0; - return false; - } memoizedIsInitialized = 1; return true; } @@ -2378,7 +2577,16 @@ public final class RPCProtos { output.writeUInt32(1, callId_); } if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeEnum(2, status_.getNumber()); + output.writeMessage(2, traceInfo_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, getMethodNameBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBool(4, requestParam_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeMessage(5, cellBlockMeta_); } getUnknownFields().writeTo(output); } @@ -2395,7 +2603,19 @@ public final class RPCProtos { } if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeEnumSize(2, status_.getNumber()); + .computeMessageSize(2, traceInfo_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(3, getMethodNameBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeBoolSize(4, requestParam_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(5, cellBlockMeta_); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; @@ -2414,10 +2634,10 @@ public final class RPCProtos { if (obj == this) { return true; } - if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader)) { + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader)) { return super.equals(obj); } - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader other = (org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader) obj; + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader other = (org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader) obj; boolean result = true; result = result && (hasCallId() == other.hasCallId()); @@ -2425,10 +2645,25 @@ public final class RPCProtos { result = result && (getCallId() == other.getCallId()); } - result = result && (hasStatus() == other.hasStatus()); - if (hasStatus()) { - result = result && - (getStatus() == other.getStatus()); + result = result && (hasTraceInfo() == other.hasTraceInfo()); + if (hasTraceInfo()) { + result = result && getTraceInfo() + .equals(other.getTraceInfo()); + } + result = result && (hasMethodName() == other.hasMethodName()); + if (hasMethodName()) { + result = result && getMethodName() + .equals(other.getMethodName()); + } + result = result && (hasRequestParam() == other.hasRequestParam()); + if (hasRequestParam()) { + result = result && (getRequestParam() + == other.getRequestParam()); + } + result = result && (hasCellBlockMeta() == other.hasCellBlockMeta()); + if (hasCellBlockMeta()) { + result = result && getCellBlockMeta() + .equals(other.getCellBlockMeta()); } result = result && getUnknownFields().equals(other.getUnknownFields()); @@ -2443,49 +2678,61 @@ public final class RPCProtos { hash = (37 * hash) + CALLID_FIELD_NUMBER; hash = (53 * hash) + getCallId(); } - if (hasStatus()) { - hash = (37 * hash) + STATUS_FIELD_NUMBER; - hash = (53 * hash) + hashEnum(getStatus()); + if (hasTraceInfo()) { + hash = (37 * hash) + TRACEINFO_FIELD_NUMBER; + hash = (53 * hash) + getTraceInfo().hashCode(); + } + if (hasMethodName()) { + hash = (37 * hash) + METHODNAME_FIELD_NUMBER; + hash = (53 * hash) + getMethodName().hashCode(); + } + if (hasRequestParam()) { + hash = (37 * hash) + REQUESTPARAM_FIELD_NUMBER; + hash = (53 * hash) + hashBoolean(getRequestParam()); + } + if (hasCellBlockMeta()) { + hash = (37 * hash) + CELLBLOCKMETA_FIELD_NUMBER; + hash = (53 * hash) + getCellBlockMeta().hashCode(); } hash = (29 * hash) + getUnknownFields().hashCode(); return hash; } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader parseFrom(byte[] data) + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader parseFrom(java.io.InputStream input) + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader parseFrom(java.io.InputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { return newBuilder().mergeFrom(input, extensionRegistry) .buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader parseDelimitedFrom(java.io.InputStream input) + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -2494,7 +2741,7 @@ public final class RPCProtos { return null; } } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader parseDelimitedFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -2505,12 +2752,12 @@ public final class RPCProtos { return null; } } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -2520,7 +2767,7 @@ public final class RPCProtos { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader prototype) { + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } @@ -2533,18 +2780,18 @@ public final class RPCProtos { } public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder - implements org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeaderOrBuilder { + implements org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeaderOrBuilder { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcResponseHeader_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RequestHeader_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcResponseHeader_fieldAccessorTable; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RequestHeader_fieldAccessorTable; } - // Construct using org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.newBuilder() + // Construct using org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -2555,6 +2802,8 @@ public final class RPCProtos { } private void maybeForceBuilderInitialization() { if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getTraceInfoFieldBuilder(); + getCellBlockMetaFieldBuilder(); } } private static Builder create() { @@ -2565,8 +2814,22 @@ public final class RPCProtos { super.clear(); callId_ = 0; bitField0_ = (bitField0_ & ~0x00000001); - status_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Status.SUCCESS; + if (traceInfoBuilder_ == null) { + traceInfo_ = org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.getDefaultInstance(); + } else { + traceInfoBuilder_.clear(); + } bitField0_ = (bitField0_ & ~0x00000002); + methodName_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); + requestParam_ = false; + bitField0_ = (bitField0_ & ~0x00000008); + if (cellBlockMetaBuilder_ == null) { + cellBlockMeta_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.getDefaultInstance(); + } else { + cellBlockMetaBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000010); return this; } @@ -2576,24 +2839,24 @@ public final class RPCProtos { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.getDescriptor(); + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader.getDescriptor(); } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader getDefaultInstanceForType() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.getDefaultInstance(); + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader.getDefaultInstance(); } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader build() { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader result = buildPartial(); + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader build() { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } return result; } - private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader buildParsed() + private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader result = buildPartial(); + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); @@ -2601,8 +2864,8 @@ public final class RPCProtos { return result; } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader buildPartial() { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader result = new org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader(this); + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader result = new org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader(this); int from_bitField0_ = bitField0_; int to_bitField0_ = 0; if (((from_bitField0_ & 0x00000001) == 0x00000001)) { @@ -2612,42 +2875,63 @@ public final class RPCProtos { if (((from_bitField0_ & 0x00000002) == 0x00000002)) { to_bitField0_ |= 0x00000002; } - result.status_ = status_; + if (traceInfoBuilder_ == null) { + result.traceInfo_ = traceInfo_; + } else { + result.traceInfo_ = traceInfoBuilder_.build(); + } + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.methodName_ = methodName_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.requestParam_ = requestParam_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + if (cellBlockMetaBuilder_ == null) { + result.cellBlockMeta_ = cellBlockMeta_; + } else { + result.cellBlockMeta_ = cellBlockMetaBuilder_.build(); + } result.bitField0_ = to_bitField0_; onBuilt(); return result; } public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader) { - return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader)other); + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader)other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader other) { - if (other == org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.getDefaultInstance()) return this; + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader.getDefaultInstance()) return this; if (other.hasCallId()) { setCallId(other.getCallId()); } - if (other.hasStatus()) { - setStatus(other.getStatus()); + if (other.hasTraceInfo()) { + mergeTraceInfo(other.getTraceInfo()); + } + if (other.hasMethodName()) { + setMethodName(other.getMethodName()); + } + if (other.hasRequestParam()) { + setRequestParam(other.getRequestParam()); + } + if (other.hasCellBlockMeta()) { + mergeCellBlockMeta(other.getCellBlockMeta()); } this.mergeUnknownFields(other.getUnknownFields()); return this; } public final boolean isInitialized() { - if (!hasCallId()) { - - return false; - } - if (!hasStatus()) { - - return false; - } return true; } @@ -2679,15 +2963,32 @@ public final class RPCProtos { callId_ = input.readUInt32(); break; } - case 16: { - int rawValue = input.readEnum(); - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Status value = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Status.valueOf(rawValue); - if (value == null) { - unknownFields.mergeVarintField(2, rawValue); - } else { - bitField0_ |= 0x00000002; - status_ = value; + case 18: { + org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.newBuilder(); + if (hasTraceInfo()) { + subBuilder.mergeFrom(getTraceInfo()); + } + input.readMessage(subBuilder, extensionRegistry); + setTraceInfo(subBuilder.buildPartial()); + break; + } + case 26: { + bitField0_ |= 0x00000004; + methodName_ = input.readBytes(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + requestParam_ = input.readBool(); + break; + } + case 42: { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.newBuilder(); + if (hasCellBlockMeta()) { + subBuilder.mergeFrom(getCellBlockMeta()); } + input.readMessage(subBuilder, extensionRegistry); + setCellBlockMeta(subBuilder.buildPartial()); break; } } @@ -2696,7 +2997,7 @@ public final class RPCProtos { private int bitField0_; - // required uint32 callId = 1; + // optional uint32 callId = 1; private int callId_ ; public boolean hasCallId() { return ((bitField0_ & 0x00000001) == 0x00000001); @@ -2717,532 +3018,346 @@ public final class RPCProtos { return this; } - // required .RpcResponseHeader.Status status = 2; - private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Status status_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Status.SUCCESS; - public boolean hasStatus() { + // optional .RPCTInfo traceInfo = 2; + private org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo traceInfo_ = org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo, org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfoOrBuilder> traceInfoBuilder_; + public boolean hasTraceInfo() { return ((bitField0_ & 0x00000002) == 0x00000002); } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Status getStatus() { - return status_; + public org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo getTraceInfo() { + if (traceInfoBuilder_ == null) { + return traceInfo_; + } else { + return traceInfoBuilder_.getMessage(); + } } - public Builder setStatus(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Status value) { - if (value == null) { - throw new NullPointerException(); + public Builder setTraceInfo(org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo value) { + if (traceInfoBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + traceInfo_ = value; + onChanged(); + } else { + traceInfoBuilder_.setMessage(value); } bitField0_ |= 0x00000002; - status_ = value; - onChanged(); return this; } - public Builder clearStatus() { - bitField0_ = (bitField0_ & ~0x00000002); - status_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Status.SUCCESS; - onChanged(); + public Builder setTraceInfo( + org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.Builder builderForValue) { + if (traceInfoBuilder_ == null) { + traceInfo_ = builderForValue.build(); + onChanged(); + } else { + traceInfoBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; return this; } - - // @@protoc_insertion_point(builder_scope:RpcResponseHeader) - } - - static { - defaultInstance = new RpcResponseHeader(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:RpcResponseHeader) - } - - public interface RpcResponseBodyOrBuilder - extends com.google.protobuf.MessageOrBuilder { - - // optional bytes response = 1; - boolean hasResponse(); - com.google.protobuf.ByteString getResponse(); - } - public static final class RpcResponseBody extends - com.google.protobuf.GeneratedMessage - implements RpcResponseBodyOrBuilder { - // Use RpcResponseBody.newBuilder() to construct. - private RpcResponseBody(Builder builder) { - super(builder); - } - private RpcResponseBody(boolean noInit) {} - - private static final RpcResponseBody defaultInstance; - public static RpcResponseBody getDefaultInstance() { - return defaultInstance; - } - - public RpcResponseBody getDefaultInstanceForType() { - return defaultInstance; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcResponseBody_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcResponseBody_fieldAccessorTable; - } - - private int bitField0_; - // optional bytes response = 1; - public static final int RESPONSE_FIELD_NUMBER = 1; - private com.google.protobuf.ByteString response_; - public boolean hasResponse() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public com.google.protobuf.ByteString getResponse() { - return response_; - } - - private void initFields() { - response_ = com.google.protobuf.ByteString.EMPTY; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - memoizedIsInitialized = 1; - return true; - } - - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getSerializedSize(); - if (((bitField0_ & 0x00000001) == 0x00000001)) { - output.writeBytes(1, response_); - } - getUnknownFields().writeTo(output); - } - - private int memoizedSerializedSize = -1; - public int getSerializedSize() { - int size = memoizedSerializedSize; - if (size != -1) return size; - - size = 0; - if (((bitField0_ & 0x00000001) == 0x00000001)) { - size += com.google.protobuf.CodedOutputStream - .computeBytesSize(1, response_); - } - size += getUnknownFields().getSerializedSize(); - memoizedSerializedSize = size; - return size; - } - - private static final long serialVersionUID = 0L; - @java.lang.Override - protected java.lang.Object writeReplace() - throws java.io.ObjectStreamException { - return super.writeReplace(); - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody)) { - return super.equals(obj); - } - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody other = (org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody) obj; - - boolean result = true; - result = result && (hasResponse() == other.hasResponse()); - if (hasResponse()) { - result = result && getResponse() - .equals(other.getResponse()); - } - result = result && - getUnknownFields().equals(other.getUnknownFields()); - return result; - } - - @java.lang.Override - public int hashCode() { - int hash = 41; - hash = (19 * hash) + getDescriptorForType().hashCode(); - if (hasResponse()) { - hash = (37 * hash) + RESPONSE_FIELD_NUMBER; - hash = (53 * hash) + getResponse().hashCode(); - } - hash = (29 * hash) + getUnknownFields().hashCode(); - return hash; - } - - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; + public Builder mergeTraceInfo(org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo value) { + if (traceInfoBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + traceInfo_ != org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.getDefaultInstance()) { + traceInfo_ = + org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.newBuilder(traceInfo_).mergeFrom(value).buildPartial(); + } else { + traceInfo_ = value; + } + onChanged(); + } else { + traceInfoBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000002; + return this; } - } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; + public Builder clearTraceInfo() { + if (traceInfoBuilder_ == null) { + traceInfo_ = org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.getDefaultInstance(); + onChanged(); + } else { + traceInfoBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; } - } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - - public static Builder newBuilder() { return Builder.create(); } - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody prototype) { - return newBuilder().mergeFrom(prototype); - } - public Builder toBuilder() { return newBuilder(this); } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder - implements org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBodyOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcResponseBody_descriptor; + public org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.Builder getTraceInfoBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getTraceInfoFieldBuilder().getBuilder(); } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcResponseBody_fieldAccessorTable; + public org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfoOrBuilder getTraceInfoOrBuilder() { + if (traceInfoBuilder_ != null) { + return traceInfoBuilder_.getMessageOrBuilder(); + } else { + return traceInfo_; + } } - - // Construct using org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody.newBuilder() - private Builder() { - maybeForceBuilderInitialization(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo, org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfoOrBuilder> + getTraceInfoFieldBuilder() { + if (traceInfoBuilder_ == null) { + traceInfoBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo, org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfoOrBuilder>( + traceInfo_, + getParentForChildren(), + isClean()); + traceInfo_ = null; + } + return traceInfoBuilder_; } - private Builder(BuilderParent parent) { - super(parent); - maybeForceBuilderInitialization(); + // optional string methodName = 3; + private java.lang.Object methodName_ = ""; + public boolean hasMethodName() { + return ((bitField0_ & 0x00000004) == 0x00000004); } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + public String getMethodName() { + java.lang.Object ref = methodName_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + methodName_ = s; + return s; + } else { + return (String) ref; } } - private static Builder create() { - return new Builder(); + public Builder setMethodName(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + methodName_ = value; + onChanged(); + return this; } - - public Builder clear() { - super.clear(); - response_ = com.google.protobuf.ByteString.EMPTY; - bitField0_ = (bitField0_ & ~0x00000001); + public Builder clearMethodName() { + bitField0_ = (bitField0_ & ~0x00000004); + methodName_ = getDefaultInstance().getMethodName(); + onChanged(); return this; } - - public Builder clone() { - return create().mergeFrom(buildPartial()); + void setMethodName(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000004; + methodName_ = value; + onChanged(); } - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody.getDescriptor(); + // optional bool requestParam = 4; + private boolean requestParam_ ; + public boolean hasRequestParam() { + return ((bitField0_ & 0x00000008) == 0x00000008); } - - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody getDefaultInstanceForType() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody.getDefaultInstance(); + public boolean getRequestParam() { + return requestParam_; } - - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody build() { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; + public Builder setRequestParam(boolean value) { + bitField0_ |= 0x00000008; + requestParam_ = value; + onChanged(); + return this; } - - private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; + public Builder clearRequestParam() { + bitField0_ = (bitField0_ & ~0x00000008); + requestParam_ = false; + onChanged(); + return this; } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody buildPartial() { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody result = new org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody(this); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.response_ = response_; - result.bitField0_ = to_bitField0_; - onBuilt(); - return result; + // optional .CellBlockMeta cellBlockMeta = 5; + private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta cellBlockMeta_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMetaOrBuilder> cellBlockMetaBuilder_; + public boolean hasCellBlockMeta() { + return ((bitField0_ & 0x00000010) == 0x00000010); } - - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody) { - return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody)other); + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta getCellBlockMeta() { + if (cellBlockMetaBuilder_ == null) { + return cellBlockMeta_; } else { - super.mergeFrom(other); - return this; + return cellBlockMetaBuilder_.getMessage(); } } - - public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody other) { - if (other == org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody.getDefaultInstance()) return this; - if (other.hasResponse()) { - setResponse(other.getResponse()); + public Builder setCellBlockMeta(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta value) { + if (cellBlockMetaBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + cellBlockMeta_ = value; + onChanged(); + } else { + cellBlockMetaBuilder_.setMessage(value); } - this.mergeUnknownFields(other.getUnknownFields()); + bitField0_ |= 0x00000010; return this; } - - public final boolean isInitialized() { - return true; + public Builder setCellBlockMeta( + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.Builder builderForValue) { + if (cellBlockMetaBuilder_ == null) { + cellBlockMeta_ = builderForValue.build(); + onChanged(); + } else { + cellBlockMetaBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000010; + return this; } - - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - } - break; - } - case 10: { - bitField0_ |= 0x00000001; - response_ = input.readBytes(); - break; - } + public Builder mergeCellBlockMeta(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta value) { + if (cellBlockMetaBuilder_ == null) { + if (((bitField0_ & 0x00000010) == 0x00000010) && + cellBlockMeta_ != org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.getDefaultInstance()) { + cellBlockMeta_ = + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.newBuilder(cellBlockMeta_).mergeFrom(value).buildPartial(); + } else { + cellBlockMeta_ = value; } + onChanged(); + } else { + cellBlockMetaBuilder_.mergeFrom(value); } + bitField0_ |= 0x00000010; + return this; } - - private int bitField0_; - - // optional bytes response = 1; - private com.google.protobuf.ByteString response_ = com.google.protobuf.ByteString.EMPTY; - public boolean hasResponse() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public com.google.protobuf.ByteString getResponse() { - return response_; - } - public Builder setResponse(com.google.protobuf.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000001; - response_ = value; - onChanged(); + public Builder clearCellBlockMeta() { + if (cellBlockMetaBuilder_ == null) { + cellBlockMeta_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.getDefaultInstance(); + onChanged(); + } else { + cellBlockMetaBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000010); return this; } - public Builder clearResponse() { - bitField0_ = (bitField0_ & ~0x00000001); - response_ = getDefaultInstance().getResponse(); + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.Builder getCellBlockMetaBuilder() { + bitField0_ |= 0x00000010; onChanged(); - return this; + return getCellBlockMetaFieldBuilder().getBuilder(); + } + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMetaOrBuilder getCellBlockMetaOrBuilder() { + if (cellBlockMetaBuilder_ != null) { + return cellBlockMetaBuilder_.getMessageOrBuilder(); + } else { + return cellBlockMeta_; + } + } + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMetaOrBuilder> + getCellBlockMetaFieldBuilder() { + if (cellBlockMetaBuilder_ == null) { + cellBlockMetaBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMetaOrBuilder>( + cellBlockMeta_, + getParentForChildren(), + isClean()); + cellBlockMeta_ = null; + } + return cellBlockMetaBuilder_; } - // @@protoc_insertion_point(builder_scope:RpcResponseBody) + // @@protoc_insertion_point(builder_scope:RequestHeader) } static { - defaultInstance = new RpcResponseBody(true); + defaultInstance = new RequestHeader(true); defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:RpcResponseBody) + // @@protoc_insertion_point(class_scope:RequestHeader) } - public interface RpcExceptionOrBuilder + public interface ResponseHeaderOrBuilder extends com.google.protobuf.MessageOrBuilder { - // required string exceptionName = 1; - boolean hasExceptionName(); - String getExceptionName(); + // optional uint32 callId = 1; + boolean hasCallId(); + int getCallId(); - // optional string stackTrace = 2; - boolean hasStackTrace(); - String getStackTrace(); + // optional .ExceptionResponse exception = 2; + boolean hasException(); + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse getException(); + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponseOrBuilder getExceptionOrBuilder(); + + // optional .CellBlockMeta cellBlockMeta = 3; + boolean hasCellBlockMeta(); + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta getCellBlockMeta(); + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMetaOrBuilder getCellBlockMetaOrBuilder(); } - public static final class RpcException extends + public static final class ResponseHeader extends com.google.protobuf.GeneratedMessage - implements RpcExceptionOrBuilder { - // Use RpcException.newBuilder() to construct. - private RpcException(Builder builder) { + implements ResponseHeaderOrBuilder { + // Use ResponseHeader.newBuilder() to construct. + private ResponseHeader(Builder builder) { super(builder); } - private RpcException(boolean noInit) {} + private ResponseHeader(boolean noInit) {} - private static final RpcException defaultInstance; - public static RpcException getDefaultInstance() { + private static final ResponseHeader defaultInstance; + public static ResponseHeader getDefaultInstance() { return defaultInstance; } - public RpcException getDefaultInstanceForType() { + public ResponseHeader getDefaultInstanceForType() { return defaultInstance; } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcException_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_ResponseHeader_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcException_fieldAccessorTable; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_ResponseHeader_fieldAccessorTable; } private int bitField0_; - // required string exceptionName = 1; - public static final int EXCEPTIONNAME_FIELD_NUMBER = 1; - private java.lang.Object exceptionName_; - public boolean hasExceptionName() { + // optional uint32 callId = 1; + public static final int CALLID_FIELD_NUMBER = 1; + private int callId_; + public boolean hasCallId() { return ((bitField0_ & 0x00000001) == 0x00000001); } - public String getExceptionName() { - java.lang.Object ref = exceptionName_; - if (ref instanceof String) { - return (String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - String s = bs.toStringUtf8(); - if (com.google.protobuf.Internal.isValidUtf8(bs)) { - exceptionName_ = s; - } - return s; - } - } - private com.google.protobuf.ByteString getExceptionNameBytes() { - java.lang.Object ref = exceptionName_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8((String) ref); - exceptionName_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } + public int getCallId() { + return callId_; } - // optional string stackTrace = 2; - public static final int STACKTRACE_FIELD_NUMBER = 2; - private java.lang.Object stackTrace_; - public boolean hasStackTrace() { + // optional .ExceptionResponse exception = 2; + public static final int EXCEPTION_FIELD_NUMBER = 2; + private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse exception_; + public boolean hasException() { return ((bitField0_ & 0x00000002) == 0x00000002); } - public String getStackTrace() { - java.lang.Object ref = stackTrace_; - if (ref instanceof String) { - return (String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - String s = bs.toStringUtf8(); - if (com.google.protobuf.Internal.isValidUtf8(bs)) { - stackTrace_ = s; - } - return s; - } + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse getException() { + return exception_; } - private com.google.protobuf.ByteString getStackTraceBytes() { - java.lang.Object ref = stackTrace_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8((String) ref); - stackTrace_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponseOrBuilder getExceptionOrBuilder() { + return exception_; + } + + // optional .CellBlockMeta cellBlockMeta = 3; + public static final int CELLBLOCKMETA_FIELD_NUMBER = 3; + private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta cellBlockMeta_; + public boolean hasCellBlockMeta() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta getCellBlockMeta() { + return cellBlockMeta_; + } + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMetaOrBuilder getCellBlockMetaOrBuilder() { + return cellBlockMeta_; } private void initFields() { - exceptionName_ = ""; - stackTrace_ = ""; + callId_ = 0; + exception_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse.getDefaultInstance(); + cellBlockMeta_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.getDefaultInstance(); } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - if (!hasExceptionName()) { - memoizedIsInitialized = 0; - return false; - } memoizedIsInitialized = 1; return true; } @@ -3251,10 +3366,13 @@ public final class RPCProtos { throws java.io.IOException { getSerializedSize(); if (((bitField0_ & 0x00000001) == 0x00000001)) { - output.writeBytes(1, getExceptionNameBytes()); + output.writeUInt32(1, callId_); } if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeBytes(2, getStackTraceBytes()); + output.writeMessage(2, exception_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeMessage(3, cellBlockMeta_); } getUnknownFields().writeTo(output); } @@ -3267,11 +3385,15 @@ public final class RPCProtos { size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeBytesSize(1, getExceptionNameBytes()); + .computeUInt32Size(1, callId_); } if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeBytesSize(2, getStackTraceBytes()); + .computeMessageSize(2, exception_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(3, cellBlockMeta_); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; @@ -3290,21 +3412,26 @@ public final class RPCProtos { if (obj == this) { return true; } - if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException)) { + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader)) { return super.equals(obj); } - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException other = (org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException) obj; + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader other = (org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader) obj; boolean result = true; - result = result && (hasExceptionName() == other.hasExceptionName()); - if (hasExceptionName()) { - result = result && getExceptionName() - .equals(other.getExceptionName()); + result = result && (hasCallId() == other.hasCallId()); + if (hasCallId()) { + result = result && (getCallId() + == other.getCallId()); } - result = result && (hasStackTrace() == other.hasStackTrace()); - if (hasStackTrace()) { - result = result && getStackTrace() - .equals(other.getStackTrace()); + result = result && (hasException() == other.hasException()); + if (hasException()) { + result = result && getException() + .equals(other.getException()); + } + result = result && (hasCellBlockMeta() == other.hasCellBlockMeta()); + if (hasCellBlockMeta()) { + result = result && getCellBlockMeta() + .equals(other.getCellBlockMeta()); } result = result && getUnknownFields().equals(other.getUnknownFields()); @@ -3315,53 +3442,57 @@ public final class RPCProtos { public int hashCode() { int hash = 41; hash = (19 * hash) + getDescriptorForType().hashCode(); - if (hasExceptionName()) { - hash = (37 * hash) + EXCEPTIONNAME_FIELD_NUMBER; - hash = (53 * hash) + getExceptionName().hashCode(); + if (hasCallId()) { + hash = (37 * hash) + CALLID_FIELD_NUMBER; + hash = (53 * hash) + getCallId(); } - if (hasStackTrace()) { - hash = (37 * hash) + STACKTRACE_FIELD_NUMBER; - hash = (53 * hash) + getStackTrace().hashCode(); + if (hasException()) { + hash = (37 * hash) + EXCEPTION_FIELD_NUMBER; + hash = (53 * hash) + getException().hashCode(); + } + if (hasCellBlockMeta()) { + hash = (37 * hash) + CELLBLOCKMETA_FIELD_NUMBER; + hash = (53 * hash) + getCellBlockMeta().hashCode(); } hash = (29 * hash) + getUnknownFields().hashCode(); return hash; } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException parseFrom(byte[] data) + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException parseFrom(java.io.InputStream input) + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader parseFrom(java.io.InputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { return newBuilder().mergeFrom(input, extensionRegistry) .buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException parseDelimitedFrom(java.io.InputStream input) + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -3370,7 +3501,7 @@ public final class RPCProtos { return null; } } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException parseDelimitedFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -3381,12 +3512,12 @@ public final class RPCProtos { return null; } } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -3396,7 +3527,7 @@ public final class RPCProtos { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException prototype) { + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } @@ -3409,18 +3540,18 @@ public final class RPCProtos { } public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder - implements org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcExceptionOrBuilder { + implements org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeaderOrBuilder { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcException_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_ResponseHeader_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcException_fieldAccessorTable; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_ResponseHeader_fieldAccessorTable; } - // Construct using org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.newBuilder() + // Construct using org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -3431,6 +3562,8 @@ public final class RPCProtos { } private void maybeForceBuilderInitialization() { if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getExceptionFieldBuilder(); + getCellBlockMetaFieldBuilder(); } } private static Builder create() { @@ -3439,10 +3572,20 @@ public final class RPCProtos { public Builder clear() { super.clear(); - exceptionName_ = ""; + callId_ = 0; bitField0_ = (bitField0_ & ~0x00000001); - stackTrace_ = ""; + if (exceptionBuilder_ == null) { + exception_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse.getDefaultInstance(); + } else { + exceptionBuilder_.clear(); + } bitField0_ = (bitField0_ & ~0x00000002); + if (cellBlockMetaBuilder_ == null) { + cellBlockMeta_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.getDefaultInstance(); + } else { + cellBlockMetaBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); return this; } @@ -3452,24 +3595,24 @@ public final class RPCProtos { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.getDescriptor(); + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader.getDescriptor(); } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException getDefaultInstanceForType() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.getDefaultInstance(); + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader.getDefaultInstance(); } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException build() { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException result = buildPartial(); + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader build() { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } return result; } - private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException buildParsed() + private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException result = buildPartial(); + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); @@ -3477,49 +3620,60 @@ public final class RPCProtos { return result; } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException buildPartial() { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException result = new org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException(this); + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader result = new org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader(this); int from_bitField0_ = bitField0_; int to_bitField0_ = 0; if (((from_bitField0_ & 0x00000001) == 0x00000001)) { to_bitField0_ |= 0x00000001; } - result.exceptionName_ = exceptionName_; + result.callId_ = callId_; if (((from_bitField0_ & 0x00000002) == 0x00000002)) { to_bitField0_ |= 0x00000002; } - result.stackTrace_ = stackTrace_; + if (exceptionBuilder_ == null) { + result.exception_ = exception_; + } else { + result.exception_ = exceptionBuilder_.build(); + } + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + if (cellBlockMetaBuilder_ == null) { + result.cellBlockMeta_ = cellBlockMeta_; + } else { + result.cellBlockMeta_ = cellBlockMetaBuilder_.build(); + } result.bitField0_ = to_bitField0_; onBuilt(); return result; } public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException) { - return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException)other); + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader)other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException other) { - if (other == org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.getDefaultInstance()) return this; - if (other.hasExceptionName()) { - setExceptionName(other.getExceptionName()); + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader.getDefaultInstance()) return this; + if (other.hasCallId()) { + setCallId(other.getCallId()); } - if (other.hasStackTrace()) { - setStackTrace(other.getStackTrace()); + if (other.hasException()) { + mergeException(other.getException()); + } + if (other.hasCellBlockMeta()) { + mergeCellBlockMeta(other.getCellBlockMeta()); } this.mergeUnknownFields(other.getUnknownFields()); return this; } public final boolean isInitialized() { - if (!hasExceptionName()) { - - return false; - } return true; } @@ -3546,14 +3700,27 @@ public final class RPCProtos { } break; } - case 10: { + case 8: { bitField0_ |= 0x00000001; - exceptionName_ = input.readBytes(); + callId_ = input.readUInt32(); break; } case 18: { - bitField0_ |= 0x00000002; - stackTrace_ = input.readBytes(); + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse.newBuilder(); + if (hasException()) { + subBuilder.mergeFrom(getException()); + } + input.readMessage(subBuilder, extensionRegistry); + setException(subBuilder.buildPartial()); + break; + } + case 26: { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.newBuilder(); + if (hasCellBlockMeta()) { + subBuilder.mergeFrom(getCellBlockMeta()); + } + input.readMessage(subBuilder, extensionRegistry); + setCellBlockMeta(subBuilder.buildPartial()); break; } } @@ -3562,87 +3729,216 @@ public final class RPCProtos { private int bitField0_; - // required string exceptionName = 1; - private java.lang.Object exceptionName_ = ""; - public boolean hasExceptionName() { + // optional uint32 callId = 1; + private int callId_ ; + public boolean hasCallId() { return ((bitField0_ & 0x00000001) == 0x00000001); } - public String getExceptionName() { - java.lang.Object ref = exceptionName_; - if (!(ref instanceof String)) { - String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); - exceptionName_ = s; - return s; - } else { - return (String) ref; - } + public int getCallId() { + return callId_; } - public Builder setExceptionName(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000001; - exceptionName_ = value; + public Builder setCallId(int value) { + bitField0_ |= 0x00000001; + callId_ = value; onChanged(); return this; } - public Builder clearExceptionName() { + public Builder clearCallId() { bitField0_ = (bitField0_ & ~0x00000001); - exceptionName_ = getDefaultInstance().getExceptionName(); + callId_ = 0; onChanged(); return this; } - void setExceptionName(com.google.protobuf.ByteString value) { - bitField0_ |= 0x00000001; - exceptionName_ = value; - onChanged(); - } - // optional string stackTrace = 2; - private java.lang.Object stackTrace_ = ""; - public boolean hasStackTrace() { + // optional .ExceptionResponse exception = 2; + private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse exception_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponseOrBuilder> exceptionBuilder_; + public boolean hasException() { return ((bitField0_ & 0x00000002) == 0x00000002); } - public String getStackTrace() { - java.lang.Object ref = stackTrace_; - if (!(ref instanceof String)) { - String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); - stackTrace_ = s; - return s; + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse getException() { + if (exceptionBuilder_ == null) { + return exception_; } else { - return (String) ref; + return exceptionBuilder_.getMessage(); } } - public Builder setStackTrace(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000002; - stackTrace_ = value; - onChanged(); + public Builder setException(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse value) { + if (exceptionBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + exception_ = value; + onChanged(); + } else { + exceptionBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; return this; } - public Builder clearStackTrace() { + public Builder setException( + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse.Builder builderForValue) { + if (exceptionBuilder_ == null) { + exception_ = builderForValue.build(); + onChanged(); + } else { + exceptionBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder mergeException(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse value) { + if (exceptionBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + exception_ != org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse.getDefaultInstance()) { + exception_ = + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse.newBuilder(exception_).mergeFrom(value).buildPartial(); + } else { + exception_ = value; + } + onChanged(); + } else { + exceptionBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder clearException() { + if (exceptionBuilder_ == null) { + exception_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse.getDefaultInstance(); + onChanged(); + } else { + exceptionBuilder_.clear(); + } bitField0_ = (bitField0_ & ~0x00000002); - stackTrace_ = getDefaultInstance().getStackTrace(); - onChanged(); return this; } - void setStackTrace(com.google.protobuf.ByteString value) { + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse.Builder getExceptionBuilder() { bitField0_ |= 0x00000002; - stackTrace_ = value; onChanged(); + return getExceptionFieldBuilder().getBuilder(); + } + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponseOrBuilder getExceptionOrBuilder() { + if (exceptionBuilder_ != null) { + return exceptionBuilder_.getMessageOrBuilder(); + } else { + return exception_; + } + } + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponseOrBuilder> + getExceptionFieldBuilder() { + if (exceptionBuilder_ == null) { + exceptionBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponseOrBuilder>( + exception_, + getParentForChildren(), + isClean()); + exception_ = null; + } + return exceptionBuilder_; + } + + // optional .CellBlockMeta cellBlockMeta = 3; + private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta cellBlockMeta_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMetaOrBuilder> cellBlockMetaBuilder_; + public boolean hasCellBlockMeta() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta getCellBlockMeta() { + if (cellBlockMetaBuilder_ == null) { + return cellBlockMeta_; + } else { + return cellBlockMetaBuilder_.getMessage(); + } + } + public Builder setCellBlockMeta(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta value) { + if (cellBlockMetaBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + cellBlockMeta_ = value; + onChanged(); + } else { + cellBlockMetaBuilder_.setMessage(value); + } + bitField0_ |= 0x00000004; + return this; + } + public Builder setCellBlockMeta( + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.Builder builderForValue) { + if (cellBlockMetaBuilder_ == null) { + cellBlockMeta_ = builderForValue.build(); + onChanged(); + } else { + cellBlockMetaBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000004; + return this; + } + public Builder mergeCellBlockMeta(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta value) { + if (cellBlockMetaBuilder_ == null) { + if (((bitField0_ & 0x00000004) == 0x00000004) && + cellBlockMeta_ != org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.getDefaultInstance()) { + cellBlockMeta_ = + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.newBuilder(cellBlockMeta_).mergeFrom(value).buildPartial(); + } else { + cellBlockMeta_ = value; + } + onChanged(); + } else { + cellBlockMetaBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000004; + return this; + } + public Builder clearCellBlockMeta() { + if (cellBlockMetaBuilder_ == null) { + cellBlockMeta_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.getDefaultInstance(); + onChanged(); + } else { + cellBlockMetaBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.Builder getCellBlockMetaBuilder() { + bitField0_ |= 0x00000004; + onChanged(); + return getCellBlockMetaFieldBuilder().getBuilder(); + } + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMetaOrBuilder getCellBlockMetaOrBuilder() { + if (cellBlockMetaBuilder_ != null) { + return cellBlockMetaBuilder_.getMessageOrBuilder(); + } else { + return cellBlockMeta_; + } + } + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMetaOrBuilder> + getCellBlockMetaFieldBuilder() { + if (cellBlockMetaBuilder_ == null) { + cellBlockMetaBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMetaOrBuilder>( + cellBlockMeta_, + getParentForChildren(), + isClean()); + cellBlockMeta_ = null; + } + return cellBlockMetaBuilder_; } - // @@protoc_insertion_point(builder_scope:RpcException) + // @@protoc_insertion_point(builder_scope:ResponseHeader) } static { - defaultInstance = new RpcException(true); + defaultInstance = new ResponseHeader(true); defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:RpcException) + // @@protoc_insertion_point(class_scope:ResponseHeader) } private static com.google.protobuf.Descriptors.Descriptor @@ -3656,30 +3952,25 @@ public final class RPCProtos { com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_ConnectionHeader_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_RpcRequestHeader_descriptor; + internal_static_CellBlockMeta_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_RpcRequestHeader_fieldAccessorTable; + internal_static_CellBlockMeta_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_RpcRequestBody_descriptor; + internal_static_ExceptionResponse_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_RpcRequestBody_fieldAccessorTable; + internal_static_ExceptionResponse_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_RpcResponseHeader_descriptor; + internal_static_RequestHeader_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_RpcResponseHeader_fieldAccessorTable; + internal_static_RequestHeader_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_RpcResponseBody_descriptor; + internal_static_ResponseHeader_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_RpcResponseBody_fieldAccessorTable; - private static com.google.protobuf.Descriptors.Descriptor - internal_static_RpcException_descriptor; - private static - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_RpcException_fieldAccessorTable; + internal_static_ResponseHeader_fieldAccessorTable; public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -3689,22 +3980,27 @@ public final class RPCProtos { descriptor; static { java.lang.String[] descriptorData = { - "\n\tRPC.proto\032\rTracing.proto\":\n\017UserInform" + - "ation\022\025\n\reffectiveUser\030\001 \002(\t\022\020\n\010realUser" + - "\030\002 \001(\t\"w\n\020ConnectionHeader\022\"\n\010userInfo\030\001" + - " \001(\0132\020.UserInformation\022?\n\010protocol\030\002 \001(\t" + - ":-org.apache.hadoop.hbase.client.ClientP" + - "rotocol\"<\n\020RpcRequestHeader\022\016\n\006callId\030\001 " + - "\002(\r\022\030\n\005tinfo\030\002 \001(\0132\t.RPCTInfo\"O\n\016RpcRequ" + - "estBody\022\022\n\nmethodName\030\001 \002(\t\022\017\n\007request\030\002" + - " \001(\014\022\030\n\020requestClassName\030\004 \001(\t\"{\n\021RpcRes" + - "ponseHeader\022\016\n\006callId\030\001 \002(\r\022)\n\006status\030\002 ", - "\002(\0162\031.RpcResponseHeader.Status\"+\n\006Status" + - "\022\013\n\007SUCCESS\020\000\022\t\n\005ERROR\020\001\022\t\n\005FATAL\020\002\"#\n\017R" + - "pcResponseBody\022\020\n\010response\030\001 \001(\014\"9\n\014RpcE" + - "xception\022\025\n\rexceptionName\030\001 \002(\t\022\022\n\nstack" + - "Trace\030\002 \001(\tB<\n*org.apache.hadoop.hbase.p" + - "rotobuf.generatedB\tRPCProtosH\001\240\001\001" + "\n\tRPC.proto\032\rTracing.proto\032\013hbase.proto\"" + + ":\n\017UserInformation\022\025\n\reffectiveUser\030\001 \002(" + + "\t\022\020\n\010realUser\030\002 \001(\t\"\343\001\n\020ConnectionHeader" + + "\022\"\n\010userInfo\030\001 \001(\0132\020.UserInformation\022?\n\010" + + "protocol\030\002 \001(\t:-org.apache.hadoop.hbase." + + "client.ClientProtocol\022H\n\023cellBlockCodecC" + + "lass\030\003 \001(\t:+org.apache.hadoop.hbase.code" + + "c.KeyValueCodec\022 \n\030cellBlockCompressorCl" + + "ass\030\004 \001(\t\"\037\n\rCellBlockMeta\022\016\n\006length\030\001 \001" + + "(\r\"w\n\021ExceptionResponse\022\032\n\022exceptionClas", + "sName\030\001 \001(\t\022\022\n\nstackTrace\030\002 \001(\t\022\020\n\010hostn" + + "ame\030\003 \001(\t\022\014\n\004port\030\004 \001(\005\022\022\n\ndoNotRetry\030\005 " + + "\001(\010\"\216\001\n\rRequestHeader\022\016\n\006callId\030\001 \001(\r\022\034\n" + + "\ttraceInfo\030\002 \001(\0132\t.RPCTInfo\022\022\n\nmethodNam" + + "e\030\003 \001(\t\022\024\n\014requestParam\030\004 \001(\010\022%\n\rcellBlo" + + "ckMeta\030\005 \001(\0132\016.CellBlockMeta\"n\n\016Response" + + "Header\022\016\n\006callId\030\001 \001(\r\022%\n\texception\030\002 \001(" + + "\0132\022.ExceptionResponse\022%\n\rcellBlockMeta\030\003" + + " \001(\0132\016.CellBlockMetaB<\n*org.apache.hadoo" + + "p.hbase.protobuf.generatedB\tRPCProtosH\001\240", + "\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -3724,49 +4020,41 @@ public final class RPCProtos { internal_static_ConnectionHeader_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ConnectionHeader_descriptor, - new java.lang.String[] { "UserInfo", "Protocol", }, + new java.lang.String[] { "UserInfo", "Protocol", "CellBlockCodecClass", "CellBlockCompressorClass", }, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader.class, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader.Builder.class); - internal_static_RpcRequestHeader_descriptor = + internal_static_CellBlockMeta_descriptor = getDescriptor().getMessageTypes().get(2); - internal_static_RpcRequestHeader_fieldAccessorTable = new + internal_static_CellBlockMeta_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_RpcRequestHeader_descriptor, - new java.lang.String[] { "CallId", "Tinfo", }, - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader.class, - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader.Builder.class); - internal_static_RpcRequestBody_descriptor = + internal_static_CellBlockMeta_descriptor, + new java.lang.String[] { "Length", }, + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.class, + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.Builder.class); + internal_static_ExceptionResponse_descriptor = getDescriptor().getMessageTypes().get(3); - internal_static_RpcRequestBody_fieldAccessorTable = new + internal_static_ExceptionResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_RpcRequestBody_descriptor, - new java.lang.String[] { "MethodName", "Request", "RequestClassName", }, - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody.class, - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody.Builder.class); - internal_static_RpcResponseHeader_descriptor = + internal_static_ExceptionResponse_descriptor, + new java.lang.String[] { "ExceptionClassName", "StackTrace", "Hostname", "Port", "DoNotRetry", }, + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse.class, + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse.Builder.class); + internal_static_RequestHeader_descriptor = getDescriptor().getMessageTypes().get(4); - internal_static_RpcResponseHeader_fieldAccessorTable = new + internal_static_RequestHeader_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_RpcResponseHeader_descriptor, - new java.lang.String[] { "CallId", "Status", }, - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.class, - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Builder.class); - internal_static_RpcResponseBody_descriptor = + internal_static_RequestHeader_descriptor, + new java.lang.String[] { "CallId", "TraceInfo", "MethodName", "RequestParam", "CellBlockMeta", }, + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader.class, + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader.Builder.class); + internal_static_ResponseHeader_descriptor = getDescriptor().getMessageTypes().get(5); - internal_static_RpcResponseBody_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_RpcResponseBody_descriptor, - new java.lang.String[] { "Response", }, - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody.class, - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody.Builder.class); - internal_static_RpcException_descriptor = - getDescriptor().getMessageTypes().get(6); - internal_static_RpcException_fieldAccessorTable = new + internal_static_ResponseHeader_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_RpcException_descriptor, - new java.lang.String[] { "ExceptionName", "StackTrace", }, - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.class, - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.Builder.class); + internal_static_ResponseHeader_descriptor, + new java.lang.String[] { "CallId", "Exception", "CellBlockMeta", }, + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader.class, + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader.Builder.class); return null; } }; @@ -3774,6 +4062,7 @@ public final class RPCProtos { .internalBuildGeneratedFileFrom(descriptorData, new com.google.protobuf.Descriptors.FileDescriptor[] { org.apache.hadoop.hbase.protobuf.generated.Tracing.getDescriptor(), + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.getDescriptor(), }, assigner); } diff --git a/hbase-protocol/src/main/protobuf/Client.proto b/hbase-protocol/src/main/protobuf/Client.proto index d725b38..5a8e1d0 100644 --- a/hbase-protocol/src/main/protobuf/Client.proto +++ b/hbase-protocol/src/main/protobuf/Client.proto @@ -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; - - enum MutateType { + 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 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; } diff --git a/hbase-protocol/src/main/protobuf/MultiRowMutation.proto b/hbase-protocol/src/main/protobuf/MultiRowMutation.proto index 1aaefee..02b038b 100644 --- a/hbase-protocol/src/main/protobuf/MultiRowMutation.proto +++ b/hbase-protocol/src/main/protobuf/MultiRowMutation.proto @@ -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 { diff --git a/hbase-protocol/src/main/protobuf/RPC.proto b/hbase-protocol/src/main/protobuf/RPC.proto index 90ec7ce..e396370 100644 --- a/hbase-protocol/src/main/protobuf/RPC.proto +++ b/hbase-protocol/src/main/protobuf/RPC.proto @@ -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]> - * - * For every RPC that the client makes it needs to send the following - * RpcRequestHeader and the RpcRequestBody. At the data level this looks like: - * - * - * - * - * On a success, the server's protobuf response looks like - * - * - * On a failure, the server's protobuf response looks like - * - * - * - * 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: +// +// +// +// +// +// +// ...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: +// +// +// +// +// +// +// ...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; +// 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 request body - */ -message RpcRequestBody { - /** Name of the RPC method */ - required string methodName = 1; - - /** Bytes corresponding to the client protobuf request. This is the actual - * bytes corresponding to the RPC request argument. - */ - optional bytes request = 2; - /** Some metainfo about the request. Helps us to treat RPCs with - * different priorities. For now this is just the classname of the request - * proto object. - */ - optional string requestClassName = 4; +// At the RPC layer, this message is used to carry +// the 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; } -/** - * 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; +// 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 } -/** - * At the RPC layer, this message is used to indicate - * the server side exception to the RPC client. - * - * HBase RPC client throws an exception indicated - * by exceptionName with the stackTrace. - */ -message RpcException { - /** Class name of the exception thrown from the server */ - required string exceptionName = 1; - /** Exception stack trace from the server side */ - optional string stackTrace = 2; +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; } diff --git a/hbase-protocol/src/main/protobuf/hbase.proto b/hbase-protocol/src/main/protobuf/hbase.proto index 6719b36..cd04872 100644 --- a/hbase-protocol/src/main/protobuf/hbase.proto +++ b/hbase-protocol/src/main/protobuf/hbase.proto @@ -24,6 +24,33 @@ 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 */ @@ -201,21 +228,6 @@ enum CompareType { } /** - * 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; -} \ No newline at end of file +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/MetaEditor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/MetaEditor.java index ae19bbc..5e03c0e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/MetaEditor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/MetaEditor.java @@ -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 = diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java new file mode 100644 index 0000000..ddb095c --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java @@ -0,0 +1,75 @@ +package org.apache.hadoop.hbase.codec; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +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; + +/** + * 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. + */ +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); + } +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MultiRowMutationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MultiRowMutationEndpoint.java index cc004e4..784cd30 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MultiRowMutationEndpoint.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MultiRowMutationEndpoint.java @@ -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 rowsToLock = new TreeSet(Bytes.BYTES_COMPARATOR); - List mutateRequestList = request.getMutationRequestList(); + List mutateRequestList = request.getMutationRequestList(); List mutations = new ArrayList(mutateRequestList.size()); - for (Mutate m : mutateRequestList) { + for (MutationProto m : mutateRequestList) { mutations.add(ProtobufUtil.toMutation(m)); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java index b06960a..8926d88 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java @@ -23,9 +23,9 @@ import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_SECURITY_AUTHO import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; +import java.lang.reflect.Method; import java.net.BindException; import java.net.InetAddress; import java.net.InetSocketAddress; @@ -66,32 +66,37 @@ 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.codec.Codec; import org.apache.hadoop.hbase.exceptions.CallerDisconnectedException; +import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException; +import org.apache.hadoop.hbase.exceptions.RegionMovedException; import org.apache.hadoop.hbase.exceptions.ServerNotRunningYetException; +import org.apache.hadoop.hbase.io.ByteBufferOutputStream; import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler; import org.apache.hadoop.hbase.monitoring.TaskMonitor; +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.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.security.HBaseSaslRpcServer; import org.apache.hadoop.hbase.security.AuthMethod; +import org.apache.hadoop.hbase.security.HBaseSaslRpcServer; import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.SaslDigestCallbackHandler; import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.SaslGssCallbackHandler; import org.apache.hadoop.hbase.security.SaslStatus; import org.apache.hadoop.hbase.security.SaslUtil; import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.io.ByteBufferOutputStream; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableUtils; -import org.apache.hadoop.ipc.RPC.VersionMismatch; +import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.ipc.Server; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.UserGroupInformation; @@ -112,7 +117,10 @@ import org.cloudera.htrace.impl.NullSpan; import com.google.common.base.Function; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import com.google.protobuf.CodedInputStream; import com.google.protobuf.Message; +import com.google.protobuf.Message.Builder; +import com.google.protobuf.TextFormat; // Uses Writables doing sasl /** A client for an IPC service. IPC calls take a single Protobuf message as a @@ -126,9 +134,12 @@ import com.google.protobuf.Message; */ @InterfaceAudience.Private public abstract class HBaseServer implements RpcServer { + public static final Log LOG = LogFactory.getLog("org.apache.hadoop.ipc.HBaseServer"); private final boolean authorize; protected boolean isSecurityEnabled; + public static final byte CURRENT_VERSION = 0; + /** * How many calls/handler are allowed in the queue. */ @@ -150,11 +161,7 @@ public abstract class HBaseServer implements RpcServer { private final int warnDelayedCalls; private AtomicInteger delayedCalls; - - public static final Log LOG = - LogFactory.getLog("org.apache.hadoop.ipc.HBaseServer"); - protected static final Log TRACELOG = - LogFactory.getLog("org.apache.hadoop.ipc.HBaseServer.trace"); + private final IPCUtil ipcUtil; private static final String AUTH_FAILED_FOR = "Auth failed for "; private static final String AUTH_SUCCESSFUL_FOR = "Auth successful for "; @@ -166,6 +173,7 @@ public abstract class HBaseServer implements RpcServer { protected static final ThreadLocal SERVER = new ThreadLocal(); private volatile boolean started = false; + private static final ReflectionCache methodCache = new ReflectionCache(); private static final Map> PROTOCOL_CACHE = new ConcurrentHashMap>(); @@ -307,7 +315,10 @@ public abstract class HBaseServer implements RpcServer { /** A call queued for handling. */ protected class Call implements RpcCallContext { protected int id; // the client's call id - protected RpcRequestBody rpcRequestBody; // the parameter passed + protected Method method; + protected Message param; // the parameter passed + // Optional cell data passed outside of protobufs. + protected CellScanner cellScanner; protected Connection connection; // connection to client protected long timestamp; // the time received when response is null // the time served when response is not null @@ -320,10 +331,12 @@ public abstract class HBaseServer implements RpcServer { protected boolean isError; protected TraceInfo tinfo; - public Call(int id, RpcRequestBody rpcRequestBody, Connection connection, - Responder responder, long size, TraceInfo tinfo) { + public Call(int id, Method method, Message param, CellScanner cellScanner, + Connection connection, Responder responder, long size, TraceInfo tinfo) { this.id = id; - this.rpcRequestBody = rpcRequestBody; + this.method = method; + this.param = param; + this.cellScanner = cellScanner; this.connection = connection; this.timestamp = System.currentTimeMillis(); this.response = null; @@ -336,57 +349,77 @@ public abstract class HBaseServer implements RpcServer { @Override public String toString() { - return rpcRequestBody.toString() + " from " + connection.toString(); + return "callId: " + this.id + " methodName: " + this.method.getName() + " param: " + + (this.param != null? TextFormat.shortDebugString(this.param): "") + + " from " + connection.toString(); } protected synchronized void setSaslTokenResponse(ByteBuffer response) { this.response = response; } - protected synchronized void setResponse(Object value, Status status, - String errorClass, String error) { - if (this.isError) - return; - if (errorClass != null) { - this.isError = true; - } - - ByteBufferOutputStream buf = null; - if (value != null) { - buf = new ByteBufferOutputStream(((Message)value).getSerializedSize()); - } else { - buf = new ByteBufferOutputStream(BUFFER_INITIAL_SIZE); - } - DataOutputStream out = new DataOutputStream(buf); + protected synchronized void setResponse(Object m, final CellScanner cells, + Throwable t, String errorMsg) { + if (this.isError) return; + if (t != null) this.isError = true; + ByteBufferOutputStream bbos = null; try { - RpcResponseHeader.Builder builder = RpcResponseHeader.newBuilder(); + ResponseHeader.Builder headerBuilder = ResponseHeader.newBuilder(); + // Presume it a pb Message. Could be null. + Message result = (Message)m; // Call id. - builder.setCallId(this.id); - builder.setStatus(status); - builder.build().writeDelimitedTo(out); - if (error != null) { - RpcException.Builder b = RpcException.newBuilder(); - b.setExceptionName(errorClass); - b.setStackTrace(error); - b.build().writeDelimitedTo(out); - } else { - if (value != null) { - ((Message)value).writeDelimitedTo(out); + headerBuilder.setCallId(this.id); + if (t != null) { + ExceptionResponse.Builder exceptionBuilder = ExceptionResponse.newBuilder(); + exceptionBuilder.setExceptionClassName(t.getClass().getName()); + exceptionBuilder.setStackTrace(errorMsg); + exceptionBuilder.setDoNotRetry(t instanceof DoNotRetryIOException); + if (t instanceof RegionMovedException) { + // Special casing for this exception. This is only one carrying a payload. + // Do this instead of build a generic system for allowing exceptions carry + // any kind of payload. + RegionMovedException rme = (RegionMovedException)t; + exceptionBuilder.setHostname(rme.getHostname()); + exceptionBuilder.setPort(rme.getPort()); } + // Set the exception as the result of the method invocation. + headerBuilder.setException(exceptionBuilder.build()); } + + // TODO: Fix this hardcoding of Encoder/Decoder. Consult the Connection to see what + // the client will accept. + ByteBuffer cellBlock = + ipcUtil.buildCellBlock(this.connection.codec, this.connection.compressor, cells); + if (cellBlock != null) { + CellBlockMeta.Builder cellBlockBuilder = CellBlockMeta.newBuilder(); + // Presumes the cellBlock bytebuffer has been flipped so limit has total size in it. + cellBlockBuilder.setLength(cellBlock.limit()); + headerBuilder.setCellBlockMeta(cellBlockBuilder.build()); + } + Message header = headerBuilder.build(); + bbos = IPCUtil.write(header, result, cellBlock); + // TODO: If SASL, we have written the response at this stage? if (connection.useWrap) { - wrapWithSasl(buf); + wrapWithSasl(bbos); + } + if (LOG.isDebugEnabled()) { + LOG.debug("header " + TextFormat.shortDebugString(header) + + ", result " + (result != null? TextFormat.shortDebugString(result): "null")); } } catch (IOException e) { LOG.warn("Exception while creating response " + e); } - ByteBuffer bb = buf.getByteBuffer(); - bb.position(0); + ByteBuffer bb = null; + if (bbos != null) { + // TODO: If SASL, maybe buffer already been flipped and written? + bb = bbos.getByteBuffer(); + bb.position(0); + } this.response = bb; } private void wrapWithSasl(ByteBufferOutputStream response) - throws IOException { + throws IOException { if (connection.useSasl) { // getByteBuffer calls flip() ByteBuffer buf = response.getByteBuffer(); @@ -413,8 +446,9 @@ public abstract class HBaseServer implements RpcServer { assert this.delayReturnValue || result == null; this.delayResponse = false; delayedCalls.decrementAndGet(); - if (this.delayReturnValue) - this.setResponse(result, Status.SUCCESS, null, null); + if (this.delayReturnValue) { + this.setResponse(result, null, null, null); + } this.responder.doRespond(this); } @@ -437,8 +471,7 @@ public abstract class HBaseServer implements RpcServer { @Override public synchronized void endDelayThrowing(Throwable t) throws IOException { - this.setResponse(null, Status.ERROR, t.getClass().toString(), - StringUtils.stringifyException(t)); + this.setResponse(null, null, t, StringUtils.stringifyException(t)); this.delayResponse = false; this.sendResponseIfReady(); } @@ -540,7 +573,7 @@ public abstract class HBaseServer implements RpcServer { try { readSelector.close(); } catch (IOException ioe) { - LOG.error("Error closing read selector in " + getName(), ioe); + LOG.error("Error closing read selector", ioe); } } } @@ -567,8 +600,7 @@ public abstract class HBaseServer implements RpcServer { } } catch (InterruptedException e) { if (running) { // unexpected -- log it - LOG.info(getName() + " unexpectedly interrupted: " + - StringUtils.stringifyException(e)); + LOG.info("Unexpectedly interrupted: " + StringUtils.stringifyException(e)); } } catch (IOException ex) { LOG.error("Error in Reader", ex); @@ -635,7 +667,7 @@ public abstract class HBaseServer implements RpcServer { } if (c.timedOut(currentTime)) { if (LOG.isDebugEnabled()) - LOG.debug(getName() + ": disconnecting client " + c.getHostAddress()); + LOG.debug("Disconnecting client " + c.getHostAddress()); closeConnection(c); numNuked++; end--; @@ -651,7 +683,7 @@ public abstract class HBaseServer implements RpcServer { @Override public void run() { - LOG.info(getName() + ": starting"); + LOG.info("Starting"); SERVER.set(HBaseServer.this); while (running) { @@ -674,7 +706,7 @@ public abstract class HBaseServer implements RpcServer { } catch (OutOfMemoryError e) { if (errorHandler != null) { if (errorHandler.checkOOME(e)) { - LOG.info(getName() + ": exiting on OOME"); + LOG.info("Exiting on OOME"); closeCurrentConnection(key, e); cleanupConnections(true); return; @@ -693,7 +725,7 @@ public abstract class HBaseServer implements RpcServer { } cleanupConnections(false); } - LOG.info("Stopping " + this.getName()); + LOG.info("Stopping"); synchronized (this) { try { @@ -716,7 +748,7 @@ public abstract class HBaseServer implements RpcServer { Connection c = (Connection)key.attachment(); if (c != null) { if (LOG.isDebugEnabled()) { - LOG.debug(getName() + ": disconnecting client " + c.getHostAddress() + + LOG.debug("Disconnecting client " + c.getHostAddress() + (e != null ? " on error " + e.getMessage() : "")); } closeConnection(c); @@ -750,7 +782,7 @@ public abstract class HBaseServer implements RpcServer { numConnections++; } if (LOG.isDebugEnabled()) - LOG.debug("Server connection from " + c.toString() + + LOG.debug("Connection from " + c.toString() + "; # active connections: " + numConnections + "; # queued calls: " + callQueue.size()); } finally { @@ -766,24 +798,23 @@ public abstract class HBaseServer implements RpcServer { return; } c.setLastContact(System.currentTimeMillis()); - try { count = c.readAndProcess(); } catch (InterruptedException ieo) { throw ieo; } catch (Exception e) { - LOG.warn(getName() + ": readAndProcess threw exception " + e + ". Count of bytes read: " + count, e); + LOG.warn("ReadAndProcess threw exception " + e + ". Count of bytes read: " + + count, e); count = -1; //so that the (count < 0) block is executed } if (count < 0) { - if (LOG.isDebugEnabled()) - LOG.debug(getName() + ": disconnecting client " + - c.getHostAddress() + ". Number of active connections: "+ - numConnections); + if (LOG.isDebugEnabled()) { + LOG.debug("Disconnecting client " + c.getHostAddress() + + ". Number of active connections: " + numConnections); + } closeConnection(c); // c = null; - } - else { + } else { c.setLastContact(System.currentTimeMillis()); } } @@ -797,7 +828,7 @@ public abstract class HBaseServer implements RpcServer { try { acceptChannel.socket().close(); } catch (IOException e) { - LOG.info(getName() + ":Exception in closing listener socket. " + e); + LOG.info("Exception in closing listener socket. " + e); } } readPool.shutdownNow(); @@ -825,16 +856,16 @@ public abstract class HBaseServer implements RpcServer { @Override public void run() { - LOG.info(getName() + ": starting"); + LOG.info("Starting"); SERVER.set(HBaseServer.this); try { doRunLoop(); } finally { - LOG.info("Stopping " + this.getName()); + LOG.info("Stopping"); try { writeSelector.close(); } catch (IOException ioe) { - LOG.error("Couldn't close write selector in " + this.getName(), ioe); + LOG.error("Couldn't close write selector", ioe); } } } @@ -855,7 +886,7 @@ public abstract class HBaseServer implements RpcServer { doAsyncWrite(key); } } catch (IOException e) { - LOG.info(getName() + ": doAsyncWrite threw exception " + e); + LOG.info("asyncWrite", e); } } long now = System.currentTimeMillis(); @@ -893,7 +924,7 @@ public abstract class HBaseServer implements RpcServer { } catch (OutOfMemoryError e) { if (errorHandler != null) { if (errorHandler.checkOOME(e)) { - LOG.info(getName() + ": exiting on OOME"); + LOG.info("Exiting on OOME"); return; } } else { @@ -910,7 +941,7 @@ public abstract class HBaseServer implements RpcServer { StringUtils.stringifyException(e)); } } - LOG.info("Stopping " + this.getName()); + LOG.info("Stopped"); } private void doAsyncWrite(SelectionKey key) throws IOException { @@ -958,8 +989,8 @@ public abstract class HBaseServer implements RpcServer { // Processes one response. Returns true if there are no more pending // data for this channel. // - private boolean processResponse(final LinkedList responseQueue, - boolean inHandler) throws IOException { + private boolean processResponse(final LinkedList responseQueue, boolean inHandler) + throws IOException { boolean error = true; boolean done = false; // there is more data for this channel. int numElements; @@ -980,10 +1011,6 @@ public abstract class HBaseServer implements RpcServer { // call = responseQueue.removeFirst(); SocketChannel channel = call.connection.channel; - if (LOG.isDebugEnabled()) { - LOG.debug(getName() + ": responding to #" + call.id + " from " + - call.connection); - } // // Send as much data as we can in the non-blocking fashion // @@ -1000,8 +1027,7 @@ public abstract class HBaseServer implements RpcServer { done = false; // more calls pending to be sent. } if (LOG.isDebugEnabled()) { - LOG.debug(getName() + ": responding to #" + call.id + " from " + - call.connection + " Wrote " + numBytes + " bytes."); + LOG.debug("callId: " + call.id + " sent, wrote " + numBytes + " bytes."); } } else { // @@ -1017,16 +1043,14 @@ public abstract class HBaseServer implements RpcServer { done = true; } if (LOG.isDebugEnabled()) { - LOG.debug(getName() + ": responding to #" + call.id + " from " + - call.connection + " Wrote partial " + numBytes + - " bytes."); + LOG.debug(call.toString() + " partially sent, wrote " + numBytes + " bytes."); } } error = false; // everything went off well } } finally { if (error && call != null) { - LOG.warn(getName()+", call " + call + ": output error"); + LOG.warn(call.toString() + ": output error"); done = true; // error. no more data for this channel. closeConnection(call.connection); } @@ -1090,12 +1114,38 @@ public abstract class HBaseServer implements RpcServer { } } + @SuppressWarnings("serial") + public static class CallQueueTooBigException extends IOException { + CallQueueTooBigException() { + super(); + } + } + + private Function, Integer> qosFunction = null; + + /** + * Gets the QOS level for this call. If it is higher than the highPriorityLevel and there + * are priorityHandlers available it will be processed in it's own thread set. + * + * @param newFunc + */ + @Override + public void setQosFunction(Function, Integer> newFunc) { + qosFunction = newFunc; + } + + protected int getQosLevel(Pair headerAndParam) { + if (qosFunction == null) return 0; + Integer res = qosFunction.apply(headerAndParam); + return res == null? 0: res; + } + /** Reads calls from a connection and queues them for handling. */ public class Connection { - private boolean rpcHeaderRead = false; //if initial signature and - //version are read - private boolean headerRead = false; //if the connection header that - //follows version is read. + // If initial preamble with version and magic has been read or not. + private boolean connectionPreambleRead = false; + // If the connection header has been read or not. + private boolean connectionHeaderReader = false; protected SocketChannel channel; private ByteBuffer data; private ByteBuffer dataLengthBuffer; @@ -1109,7 +1159,15 @@ public abstract class HBaseServer implements RpcServer { // disconnected, we can say where it used to connect to. protected String hostAddress; protected int remotePort; - ConnectionHeader header; + ConnectionHeader connectionHeader; + /** + * Codec the client asked use. + */ + private Codec codec; + /** + * Compression codec the client asked us use. + */ + private CompressionCodec compressor; Class protocol; protected UserGroupInformation user = null; private AuthMethod authMethod; @@ -1117,29 +1175,31 @@ public abstract class HBaseServer implements RpcServer { private boolean skipInitialSaslHandshake; private ByteBuffer rpcHeaderBuffer; private ByteBuffer unwrappedData; + // When is this set? FindBugs wants to know! Says NP private ByteBuffer unwrappedDataLengthBuffer; boolean useSasl; SaslServer saslServer; private boolean useWrap = false; // Fake 'call' for failed authorization response private static final int AUTHROIZATION_FAILED_CALLID = -1; - private final Call authFailedCall = new Call(AUTHROIZATION_FAILED_CALLID, - null, this, null, 0, null); + private final Call authFailedCall = + new Call(AUTHROIZATION_FAILED_CALLID, null, null, null, this, null, 0, null); private ByteArrayOutputStream authFailedResponse = new ByteArrayOutputStream(); // Fake 'call' for SASL context setup private static final int SASL_CALLID = -33; - private final Call saslCall = new Call(SASL_CALLID, null, this, null, 0, - null); + private final Call saslCall = + new Call(SASL_CALLID, null, null, null, this, null, 0, null); public UserGroupInformation attemptingUser = null; // user name before auth + public Connection(SocketChannel channel, long lastContact) { this.channel = channel; this.lastContact = lastContact; this.data = null; this.dataLengthBuffer = ByteBuffer.allocate(4); this.socket = channel.socket(); - InetAddress addr = socket.getInetAddress(); + this.addr = socket.getInetAddress(); if (addr == null) { this.hostAddress = "*Unknown*"; } else { @@ -1352,49 +1412,56 @@ public abstract class HBaseServer implements RpcServer { } } + /** + * Read off the wire. + * @return Returns -1 if failure (and caller will close connection) else return how many + * bytes were read and processed. + * @throws IOException + * @throws InterruptedException + */ public int readAndProcess() throws IOException, InterruptedException { while (true) { - /* Read at most one RPC. If the header is not read completely yet - * then iterate until we read first RPC or until there is no data left. - */ + // Try and read in an int. If new connection, the int will hold the 'HBas' HEADER. If it + // does, read in the rest of the connection preamble, the version and the auth method. + // Else it will be length of the data to read (or -1 if a ping). We catch the integer + // length into the 4-byte this.dataLengthBuffer. int count = -1; if (dataLengthBuffer.remaining() > 0) { count = channelRead(channel, dataLengthBuffer); - if (count < 0 || dataLengthBuffer.remaining() > 0) + if (count < 0 || dataLengthBuffer.remaining() > 0) { return count; + } } - if (!rpcHeaderRead) { - //Every connection is expected to send the header. + // If we have not read the connection setup preamble, look to see if that is on the wire. + if (!connectionPreambleRead) { + this.dataLengthBuffer.flip(); + if (!HConstants.RPC_HEADER.equals(this.dataLengthBuffer)) { + return doBadPreambleHandling("Expected HEADER=" + + Bytes.toStringBinary(HConstants.RPC_HEADER.array()) + + " but received HEADER=" + Bytes.toStringBinary(this.dataLengthBuffer.array())); + } + // Read the next two bytes, the version and the auth to use. if (rpcHeaderBuffer == null) { + // Allocate two bytes to hold version and auth type. rpcHeaderBuffer = ByteBuffer.allocate(2); } count = channelRead(channel, rpcHeaderBuffer); if (count < 0 || rpcHeaderBuffer.remaining() > 0) { return count; } - int version = rpcHeaderBuffer.get(0); - byte[] method = new byte[] {rpcHeaderBuffer.get(1)}; - authMethod = AuthMethod.read(new DataInputStream( - new ByteArrayInputStream(method))); - dataLengthBuffer.flip(); - if (!HConstants.RPC_HEADER.equals(dataLengthBuffer) || version != HConstants.CURRENT_VERSION) { - LOG.warn("Incorrect header or version mismatch from " + - hostAddress + ":" + remotePort + - " got version " + version + - " expected version " + HConstants.CURRENT_VERSION); - setupBadVersionResponse(version); - return -1; + int version = this.rpcHeaderBuffer.get(0); + byte authbyte = this.rpcHeaderBuffer.get(1); + this.authMethod = AuthMethod.valueOf(authbyte); + if (version != CURRENT_VERSION || authMethod == null) { + return doBadPreambleHandling("serverVersion=" + CURRENT_VERSION + + ", clientVersion=" + version + ", authMethod=" + authbyte + + ", authSupported=" + (authMethod != null)); } dataLengthBuffer.clear(); - if (authMethod == null) { - throw new IOException("Unable to read authentication method"); - } if (isSecurityEnabled && authMethod == AuthMethod.SIMPLE) { - AccessControlException ae = new AccessControlException( - "Authentication is required"); - setupResponse(authFailedResponse, authFailedCall, Status.FATAL, - ae.getClass().getName(), ae.getMessage()); + AccessControlException ae = new AccessControlException("Authentication is required"); + setupResponse(authFailedResponse, authFailedCall, ae, ae.getMessage()); responder.doRespond(authFailedCall); throw ae; } @@ -1410,18 +1477,18 @@ public abstract class HBaseServer implements RpcServer { if (authMethod != AuthMethod.SIMPLE) { useSasl = true; } - rpcHeaderBuffer = null; - rpcHeaderRead = true; + connectionPreambleRead = true; + // Preamble checks out. continue; } - + // We have read a length and we have read the preamble. It is either the connection header + // or it is a request. if (data == null) { - dataLengthBuffer.flip(); - dataLength = dataLengthBuffer.getInt(); - + this.dataLengthBuffer.flip(); + dataLength = this.dataLengthBuffer.getInt(); if (dataLength == HBaseClient.PING_CALL_ID) { - if(!useWrap) { //covers the !useSasl too + if (!useWrap) { //covers the !useSasl too dataLengthBuffer.clear(); return 0; //ping message } @@ -1433,9 +1500,7 @@ public abstract class HBaseServer implements RpcServer { data = ByteBuffer.allocate(dataLength); incRpcCount(); // Increment the rpc count } - count = channelRead(channel, data); - if (data.remaining() == 0) { dataLengthBuffer.clear(); data.flip(); @@ -1444,13 +1509,13 @@ public abstract class HBaseServer implements RpcServer { skipInitialSaslHandshake = false; continue; } - boolean isHeaderRead = headerRead; + boolean isHeaderRead = connectionHeaderReader; if (useSasl) { saslReadAndProcess(data.array()); } else { processOneRpc(data.array()); } - data = null; + this.data = null; if (!isHeaderRead) { continue; } @@ -1459,50 +1524,30 @@ public abstract class HBaseServer implements RpcServer { } } - /** - * Try to set up the response to indicate that the client version - * is incompatible with the server. This can contain special-case - * code to speak enough of past IPC protocols to pass back - * an exception to the caller. - * @param clientVersion the version the caller is using - * @throws IOException - */ - private void setupBadVersionResponse(int clientVersion) throws IOException { - String errMsg = "Server IPC version " + HConstants.CURRENT_VERSION + - " cannot communicate with client version " + clientVersion; - ByteArrayOutputStream buffer = new ByteArrayOutputStream(); - - if (clientVersion >= 3) { - // We used to return an id of -1 which caused server to close the - // connection without telling the client what the problem was. Now - // we return 0 which will keep the socket up -- bad clients, unless - // they switch to suit the running server -- will fail later doing - // getProtocolVersion. - Call fakeCall = new Call(0, null, this, responder, 0, null); - // Versions 3 and greater can interpret this exception - // response in the same manner - setupResponse(buffer, fakeCall, Status.FATAL, - VersionMismatch.class.getName(), errMsg); - - responder.doRespond(fakeCall); - } - } - - /// Reads the connection header following version - private void processHeader(byte[] buf) throws IOException { - DataInputStream in = - new DataInputStream(new ByteArrayInputStream(buf)); - header = ConnectionHeader.parseFrom(in); + private int doBadPreambleHandling(final String errMsg) throws IOException { + String msg = errMsg + "; cannot communicate with client at " + hostAddress + ":" + port; + LOG.warn(msg); + Call fakeCall = new Call(-1, null, null, null, this, responder, -1, null); + setupResponse(null, fakeCall, new FatalConnectionException(msg), msg); + responder.doRespond(fakeCall); + // Returning -1 closes out the connection. + return -1; + } + + // Reads the connection header following version + private void processConnectionHeader(byte[] buf) throws IOException { + this.connectionHeader = ConnectionHeader.parseFrom(buf); try { - String protocolClassName = header.getProtocol(); + String protocolClassName = connectionHeader.getProtocol(); if (protocolClassName != null) { - protocol = getProtocolClass(header.getProtocol(), conf); + protocol = getProtocolClass(connectionHeader.getProtocol(), conf); } } catch (ClassNotFoundException cnfe) { - throw new IOException("Unknown protocol: " + header.getProtocol()); + throw new IOException("Unknown protocol: " + connectionHeader.getProtocol()); } + setupCellBlockCodecs(this.connectionHeader); - UserGroupInformation protocolUser = createUser(header); + UserGroupInformation protocolUser = createUser(connectionHeader); if (!useSasl) { user = protocolUser; if (user != null) { @@ -1535,6 +1580,30 @@ public abstract class HBaseServer implements RpcServer { } } + /** + * Set up cell block codecs + * @param header + * @throws FatalConnectionException + */ + private void setupCellBlockCodecs(final ConnectionHeader header) + throws FatalConnectionException { + // TODO: Plug in other supported decoders. + if (!header.hasCellBlockCodecClass()) throw new FatalConnectionException("No codec"); + String className = header.getCellBlockCodecClass(); + try { + this.codec = (Codec)Class.forName(className).newInstance(); + } catch (Exception e) { + throw new FatalConnectionException("Unsupported codec " + className, e); + } + if (!header.hasCellBlockCompressorClass()) return; + className = header.getCellBlockCompressorClass(); + try { + this.compressor = (CompressionCodec)Class.forName(className).newInstance(); + } catch (Exception e) { + throw new FatalConnectionException("Unsupported codec " + className, e); + } + } + private void processUnwrappedData(byte[] inBuf) throws IOException, InterruptedException { ReadableByteChannel ch = Channels.newChannel(new ByteArrayInputStream( @@ -1576,74 +1645,103 @@ public abstract class HBaseServer implements RpcServer { private void processOneRpc(byte[] buf) throws IOException, InterruptedException { - if (headerRead) { - processData(buf); + if (connectionHeaderReader) { + processRequest(buf); } else { - processHeader(buf); - headerRead = true; + processConnectionHeader(buf); + this.connectionHeaderReader = true; if (!authorizeConnection()) { throw new AccessControlException("Connection from " + this - + " for protocol " + header.getProtocol() + + " for protocol " + connectionHeader.getProtocol() + " is unauthorized for user " + user); } } } - protected void processData(byte[] buf) throws IOException, InterruptedException { - DataInputStream dis = - new DataInputStream(new ByteArrayInputStream(buf)); - RpcRequestHeader request = RpcRequestHeader.parseDelimitedFrom(dis); - - int id = request.getCallId(); - long callSize = buf.length; - + /** + * @param buf Has the request header and the request param and optionally encoded data buffer + * all in this one array. + * @throws IOException + * @throws InterruptedException + */ + protected void processRequest(byte[] buf) throws IOException, InterruptedException { + long totalRequestSize = buf.length; + int offset = 0; + // Here we read in the header. We avoid having pb + // do its default 4k allocation for CodedInputStream. We force it to use backing array. + CodedInputStream cis = CodedInputStream.newInstance(buf, offset, buf.length); + int headerSize = cis.readRawVarint32(); + offset = cis.getTotalBytesRead(); + RequestHeader header = + RequestHeader.newBuilder().mergeFrom(buf, offset, headerSize).build(); + offset += headerSize; + int id = header.getCallId(); if (LOG.isDebugEnabled()) { - LOG.debug(" got call #" + id + ", " + callSize + " bytes"); + LOG.debug(" requestHeader " + TextFormat.shortDebugString(header) + + " totalRequestSize: " + totalRequestSize + " bytes"); } // Enforcing the call queue size, this triggers a retry in the client - if ((callSize + callQueueSize.get()) > maxQueueSize) { - final Call callTooBig = new Call(id, null, this, responder, callSize, - null); + // This is a bit late to be doing this check - we have already read in the total request. + if ((totalRequestSize + callQueueSize.get()) > maxQueueSize) { + final Call callTooBig = + new Call(id, null, null, null, this, responder, totalRequestSize, null); ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream(); - setupResponse(responseBuffer, callTooBig, Status.FATAL, - IOException.class.getName(), - "Call queue is full, is ipc.server.max.callqueue.size too small?"); + setupResponse(responseBuffer, callTooBig, new CallQueueTooBigException(), + "Call queue is full, is ipc.server.max.callqueue.size too small?"); responder.doRespond(callTooBig); return; } - - RpcRequestBody rpcRequestBody; + Method method = null; + Message param = null; + CellScanner cellScanner = null; try { - rpcRequestBody = RpcRequestBody.parseDelimitedFrom(dis); + if (header.hasRequestParam() && header.getRequestParam()) { + method = methodCache.getMethod(this.protocol, header.getMethodName()); + Message m = methodCache.getMethodArgType(method); + // Check that there is a param to deserialize. + if (m != null) { + Builder builder = null; + builder = m.newBuilderForType(); + // To read the varint, I need an inputstream; might as well be a CIS. + cis = CodedInputStream.newInstance(buf, offset, buf.length); + int paramSize = cis.readRawVarint32(); + offset += cis.getTotalBytesRead(); + if (builder != null) { + builder.mergeFrom(buf, offset, paramSize); + param = builder.build(); + } + offset += paramSize; + } + } + if (header.hasCellBlockMeta()) { + cellScanner = ipcUtil.createCellScanner(this.codec, this.compressor, + buf, offset, buf.length); + } } catch (Throwable t) { - LOG.warn("Unable to read call parameters for client " + - getHostAddress(), t); - final Call readParamsFailedCall = new Call(id, null, this, responder, - callSize, null); + String msg = "Unable to read call parameter from client " + getHostAddress(); + LOG.warn(msg, t); + final Call readParamsFailedCall = + new Call(id, null, null, null, this, responder, totalRequestSize, null); ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream(); - - setupResponse(responseBuffer, readParamsFailedCall, Status.FATAL, - t.getClass().getName(), - "IPC server unable to read call parameters: " + t.getMessage()); + setupResponse(responseBuffer, readParamsFailedCall, t, + msg + "; " + t.getMessage()); responder.doRespond(readParamsFailedCall); return; } - Call call; - if (request.hasTinfo()) { - call = new Call(id, rpcRequestBody, this, responder, callSize, - new TraceInfo(request.getTinfo().getTraceId(), request.getTinfo() - .getParentId())); + Call call = null; + if (header.hasTraceInfo()) { + call = new Call(id, method, param, cellScanner, this, responder, totalRequestSize, + new TraceInfo(header.getTraceInfo().getTraceId(), header.getTraceInfo().getParentId())); } else { - call = new Call(id, rpcRequestBody, this, responder, callSize, null); + call = new Call(id, method, param, cellScanner, this, responder, totalRequestSize, null); } - - callQueueSize.add(callSize); - - if (priorityCallQueue != null && getQosLevel(rpcRequestBody) > highPriorityLevel) { + callQueueSize.add(totalRequestSize); + Pair headerAndParam = new Pair(header, param); + if (priorityCallQueue != null && getQosLevel(headerAndParam) > highPriorityLevel) { priorityCallQueue.put(call); - } else if (replicationQueue != null - && getQosLevel(rpcRequestBody) == HConstants.REPLICATION_QOS) { + } else if (replicationQueue != null && + getQosLevel(headerAndParam) == HConstants.REPLICATION_QOS) { replicationQueue.put(call); } else { callQueue.put(call); // queue the call; maybe blocked here @@ -1660,16 +1758,15 @@ public abstract class HBaseServer implements RpcServer { && (authMethod != AuthMethod.DIGEST)) { ProxyUsers.authorize(user, this.getHostAddress(), conf); } - authorize(user, header, getHostInetAddress()); + authorize(user, connectionHeader, getHostInetAddress()); if (LOG.isDebugEnabled()) { - LOG.debug("Successfully authorized " + header); + LOG.debug("Successfully authorized " + TextFormat.shortDebugString(connectionHeader)); } metrics.authorizationSuccess(); } catch (AuthorizationException ae) { LOG.debug("Connection authorization failed: "+ae.getMessage(), ae); metrics.authorizationFailure(); - setupResponse(authFailedResponse, authFailedCall, Status.FATAL, - ae.getClass().getName(), ae.getMessage()); + setupResponse(authFailedResponse, authFailedCall, ae, ae.getMessage()); responder.doRespond(authFailedCall); return false; } @@ -1739,53 +1836,42 @@ public abstract class HBaseServer implements RpcServer { @Override public void run() { - LOG.info(getName() + ": starting"); + LOG.info("Starting"); status.setStatus("starting"); SERVER.set(HBaseServer.this); while (running) { - try { status.pause("Waiting for a call"); Call call = myCallQueue.take(); // pop the queue; maybe blocked here status.setStatus("Setting up call"); - status.setConnection(call.connection.getHostAddress(), - call.connection.getRemotePort()); - - if (LOG.isDebugEnabled()) - LOG.debug(getName() + ": has #" + call.id + " from " + - call.connection); - - String errorClass = null; + status.setConnection(call.connection.getHostAddress(), call.connection.getRemotePort()); + if (LOG.isDebugEnabled()) { + UserGroupInformation remoteUser = call.connection.user; + LOG.debug(call.toString() + " executing as " + + ((remoteUser == null)? "NULL principal": remoteUser.getUserName())); + } + Throwable errorThrowable = null; String error = null; - Message value = null; - + Pair resultPair = null; CurCall.set(call); Span currentRequestSpan = NullSpan.getInstance(); try { - if (!started) + if (!started) { throw new ServerNotRunningYetException("Server is not running yet"); - + } if (call.tinfo != null) { currentRequestSpan = Trace.startSpan( "handling " + call.toString(), call.tinfo, Sampler.ALWAYS); } - - if (LOG.isDebugEnabled()) { - UserGroupInformation remoteUser = call.connection.user; - LOG.debug(getName() + ": call #" + call.id + " executing as " - + (remoteUser == null ? "NULL principal" : - remoteUser.getUserName())); - } - RequestContext.set(User.create(call.connection.user), getRemoteIp(), - call.connection.protocol); + call.connection.protocol); // make the call - value = call(call.connection.protocol, call.rpcRequestBody, call.timestamp, - status); + resultPair = call(call.connection.protocol, call.method, call.param, call.cellScanner, + call.timestamp, status); } catch (Throwable e) { - LOG.debug(getName()+", call "+call+": error: " + e, e); - errorClass = e.getClass().getName(); + LOG.debug(call.toString() + " error: " + e, e); + errorThrowable = e; error = StringUtils.stringifyException(e); } finally { currentRequestSpan.stop(); @@ -1798,21 +1884,20 @@ public abstract class HBaseServer implements RpcServer { // Set the response for undelayed calls and delayed calls with // undelayed responses. if (!call.isDelayed() || !call.isReturnValueDelayed()) { - call.setResponse(value, - errorClass == null? Status.SUCCESS: Status.ERROR, - errorClass, error); + Message param = resultPair != null? resultPair.getFirst(): null; + CellScanner cells = resultPair != null? resultPair.getSecond(): null; + call.setResponse(param, cells, errorThrowable, error); } call.sendResponseIfReady(); status.markComplete("Sent response"); } catch (InterruptedException e) { if (running) { // unexpected -- log it - LOG.info(getName() + " caught: " + - StringUtils.stringifyException(e)); + LOG.info("Caught: " + StringUtils.stringifyException(e)); } } catch (OutOfMemoryError e) { if (errorHandler != null) { if (errorHandler.checkOOME(e)) { - LOG.info(getName() + ": exiting on OOME"); + LOG.info("Exiting on OOME"); return; } } else { @@ -1820,44 +1905,16 @@ public abstract class HBaseServer implements RpcServer { throw e; } } catch (ClosedChannelException cce) { - LOG.warn(getName() + " caught a ClosedChannelException, " + + LOG.warn("Caught a ClosedChannelException, " + "this means that the server was processing a " + "request but the client went away. The error message was: " + cce.getMessage()); } catch (Exception e) { - LOG.warn(getName() + " caught: " + - StringUtils.stringifyException(e)); + LOG.warn("Caught: " + StringUtils.stringifyException(e)); } } - LOG.info(getName() + ": exiting"); - } - - } - - - private Function qosFunction = null; - - /** - * Gets the QOS level for this call. If it is higher than the highPriorityLevel and there - * are priorityHandlers available it will be processed in it's own thread set. - * - * @param newFunc - */ - @Override - public void setQosFunction(Function newFunc) { - qosFunction = newFunc; - } - - protected int getQosLevel(RpcRequestBody rpcRequestBody) { - if (qosFunction == null) { - return 0; - } - - Integer res = qosFunction.apply(rpcRequestBody); - if (res == null) { - return 0; + LOG.info("Exiting"); } - return res; } /* Constructs a server listening on the named port and address. Parameters passed must @@ -1913,6 +1970,7 @@ public abstract class HBaseServer implements RpcServer { this.warnDelayedCalls = conf.getInt(WARN_DELAYED_CALLS, DEFAULT_WARN_DELAYED_CALLS); this.delayedCalls = new AtomicInteger(0); + this.ipcUtil = new IPCUtil(conf); // Create the responder here @@ -1943,12 +2001,10 @@ public abstract class HBaseServer implements RpcServer { * @param error error message, if the call failed * @throws IOException */ - private void setupResponse(ByteArrayOutputStream response, - Call call, Status status, - String errorClass, String error) + private void setupResponse(ByteArrayOutputStream response, Call call, Throwable t, String error) throws IOException { - response.reset(); - call.setResponse(null, status, errorClass, error); + if (response != null) response.reset(); + call.setResponse(null, null, t, error); } protected void closeConnection(Connection connection) { @@ -2088,6 +2144,7 @@ public abstract class HBaseServer implements RpcServer { * @param addr InetAddress of incoming connection * @throws org.apache.hadoop.security.authorize.AuthorizationException when the client isn't authorized to talk the protocol */ + @SuppressWarnings("static-access") public void authorize(UserGroupInformation user, ConnectionHeader connection, InetAddress addr diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ProtobufRpcServerEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ProtobufRpcServerEngine.java index dcb1ae8..3d38c1e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ProtobufRpcServerEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ProtobufRpcServerEngine.java @@ -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 methodArg = - new ConcurrentHashMap(); - private static final Map methodInstances = - new ConcurrentHashMap(); 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 protocol, - RpcRequestBody rpcRequest, long receiveTime, MonitoredRPCHandler status) + public Pair call(Class 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(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 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); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java index 6e244bd..972b9b1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java @@ -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 protocol, - RpcRequestBody param, long receiveTime, MonitoredRPCHandler status) - throws IOException; + Pair call(Class protocol, Method method, + Message param, CellScanner cellScanner, long receiveTime, MonitoredRPCHandler status) + throws IOException; void setErrorHandler(HBaseRPCErrorHandler handler); - void setQosFunction(Function 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, Integer> newFunc); } \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSerialization.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSerialization.java index 4223dbe..c6e5adc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSerialization.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSerialization.java @@ -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 { } public static class KeyValueDeserializer implements Deserializer { - 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 { - 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); } } } \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MutationSerialization.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MutationSerialization.java index 19ca984..b15b513 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MutationSerialization.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MutationSerialization.java @@ -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 { @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 { @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); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandler.java index 795b88c..031943c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandler.java @@ -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); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandlerImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandlerImpl.java index bb5b928..cb98034 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandlerImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandlerImpl.java @@ -18,19 +18,15 @@ */ package org.apache.hadoop.hbase.monitoring; +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.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 com.google.protobuf.Message; /** * A MonitoredTask implementation designed for use with RPC Handlers @@ -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(); } -} +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index c3078cd..a4ed139 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -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) @@ -567,152 +568,6 @@ public class HRegionServer implements ClientProtocol, } /** - * Utility used ensuring higher quality of service for priority rpcs; e.g. - * rpcs to .META., etc. - */ - class QosFunction implements Function { - private final Map 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[] 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> argumentToClassMap = - new HashMap>(); - private final Map, Method>> - methodMap = new HashMap, Method>>(); - - public QosFunction() { - Map qosMap = new HashMap(); - 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, Method>()); - } - if (methodMap.get("getRegion") == null) { - methodMap.put("getRegion", - new HashMap, Method>()); - } - for (Class 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 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. * * @throws IOException @@ -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 cellsToReturn = null; try { HRegion region = getRegion(request.getRegion()); MultiResponse.Builder builder = MultiResponse.newBuilder(); + List mutations = new ArrayList(request.getActionCount()); + // Do a bunch of mutations atomically. Mutations are Puts and Deletes. NOT Gets. if (request.hasAtomic() && request.getAtomic()) { - List mutates = new ArrayList(); + // 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 mutates = new ArrayList(); + cellsToReturn = new ArrayList(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 mutates) { + protected void doBatchOp(final MultiResponse.Builder builder, final HRegion region, + final List mutations, final CellScanner cells) { @SuppressWarnings("unchecked") - Pair[] mutationsWithLocks = new Pair[mutates.size()]; + Pair[] 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, 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 mutates) throws IOException { - Mutate firstMutate = mutates.get(0); + protected void mutateRows(final HRegion region, final List mutations, + final CellScanner cellScanner) + throws IOException { + MutationProto firstMutate = mutations.get(0); if (!region.getRegionInfo().isMetaTable()) { cacheFlusher.reclaimMemStoreMemory(); } 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( diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/QosFunction.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/QosFunction.java new file mode 100644 index 0000000..f8a16e6 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/QosFunction.java @@ -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, Integer> { + public static final Log LOG = LogFactory.getLog(QosFunction.class.getName()); + private final Map annotatedQos; + //We need to mock the regionserver instance for some unit tests (set via + //setRegionServer method. + private HRegionServer hRegionServer; + @SuppressWarnings("unchecked") + private final Class[] 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> argumentToClassMap = + new HashMap>(); + private final Map, Method>> methodMap = + new HashMap, Method>>(); + + QosFunction(final HRegionServer hrs) { + this.hRegionServer = hrs; + Map qosMap = new HashMap(); + 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, Method>()); + methodMap.put("getRegion", new HashMap, Method>()); + } + for (Class 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 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 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; + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java index 2e3fd20..187cf1d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java @@ -49,7 +49,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; 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; @@ -79,12 +78,13 @@ 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.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; @@ -4148,11 +4148,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); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/codec/CodecPerformance.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/codec/CodecPerformance.java new file mode 100644 index 0000000..991d898 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/codec/CodecPerformance.java @@ -0,0 +1,130 @@ +/** + * 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.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; + +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); + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/codec/TestCellMessageCodec.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/codec/TestCellMessageCodec.java new file mode 100644 index 0000000..97d08ac --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/codec/TestCellMessageCodec.java @@ -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()); + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java index 3803353..5bb58a3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java @@ -43,6 +43,8 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.mortbay.log.Log; +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 @@ -97,8 +99,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(); } @@ -179,7 +180,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 { @@ -198,7 +199,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) { @@ -240,9 +242,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); @@ -273,7 +274,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."); } @@ -281,7 +282,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( @@ -300,7 +301,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(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java index a120cd5..987b4df 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java @@ -19,27 +19,37 @@ package org.apache.hadoop.hbase.ipc; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; import static org.mockito.Matchers.anyInt; 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.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.SmallTests; import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler; -import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody; 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 +63,9 @@ 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); + static final int COUNT = 1; private static class TestRpcServer extends HBaseServer { TestRpcServer() throws IOException { @@ -60,10 +73,53 @@ public class TestIPC { } @Override - public Message call(Class protocol, - RpcRequestBody param, long receiveTime, MonitoredRPCHandler status) + public Pair call(Class protocol, Method method, + Message param, final CellScanner cells, long receiveTime, MonitoredRPCHandler status) throws IOException { - return param; + int i = 0; + List cellsOut = new ArrayList(); + while (cells.advance()) { + Cell cell = cells.current(); + Bytes.equals(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), + CELL_BYTES, 0, CELL_BYTES.length); + cellsOut.add(cell); + i++; + } + assertEquals(COUNT, i); + return new Pair(param, CellUtil.createCellScanner(cellsOut)); + } + } + + /** + * 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 cells = new ArrayList(); + 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", null); + client.call(m, null, CellUtil.createCellScanner(cells), address, NothingProtocol.class, + User.getCurrent(), 0); + } finally { + client.stop(); + rpcServer.stop(); } } @@ -81,18 +137,18 @@ 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(); } } -} \ No newline at end of file +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java index cdde6e1..f45bce4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java @@ -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(); } } -} \ No newline at end of file +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java index 7738ab4..53c6a5a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java @@ -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)); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java index 8ba27bb..ff92b40 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java @@ -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(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(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(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(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(header, scanRequest)) == + HConstants.NORMAL_QOS); } - }