Index: hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ClientCache.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ClientCache.java (revision 0) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ClientCache.java (revision 0) @@ -0,0 +1,93 @@ +/** + * 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.HashMap; +import java.util.Map; + +import javax.net.SocketFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.io.HbaseObjectWritable; +import org.apache.hadoop.io.Writable; + +/** + * Cache a client using its socket factory as the hash key. + * Enables reuse/sharing of clients on a per SocketFactory basis. A client + * establishes certain configuration dependent characteristics like timeouts, + * tcp-keepalive (true or false), etc. For more details on the characteristics, + * look at {@link HBaseClient#HBaseClient(Class, Configuration, SocketFactory)} + * Creation of dynamic proxies to protocols creates the clients (and increments + * reference count once created), and stopping of the proxies leads to clearing + * out of references and when the reference drops to zero, the cache mapping is + * cleared. + */ +class ClientCache { + private Map clients = + new HashMap(); + + protected ClientCache() {} + + /** + * Construct & cache an IPC client with the user-provided SocketFactory + * if no cached client exists. + * + * @param conf Configuration + * @param factory socket factory + * @return an IPC client + */ + protected synchronized HBaseClient getClient(Configuration conf, + SocketFactory factory) { + return getClient(conf, factory, HbaseObjectWritable.class); + } + protected synchronized HBaseClient getClient(Configuration conf, + SocketFactory factory, Class valueClass) { + // Construct & cache client. The configuration is only used for timeout, + // and Clients have connection pools. So we can either (a) lose some + // connection pooling and leak sockets, or (b) use the same timeout for + // all configurations. Since the IPC is usually intended globally, not + // per-job, we choose (a). + HBaseClient client = clients.get(factory); + if (client == null) { + // Make an hbase client instead of hadoop Client. + client = new HBaseClient(valueClass, conf, factory); + clients.put(factory, client); + } else { + client.incCount(); + } + return client; + } + + /** + * Stop a RPC client connection + * A RPC client is closed only when its reference count becomes zero. + * @param client client to stop + */ + protected void stopClient(HBaseClient client) { + synchronized (this) { + client.decCount(); + if (client.isZeroReference()) { + clients.remove(client.getSocketFactory()); + } + } + if (client.isZeroReference()) { + client.stop(); + } + } +} \ No newline at end of file Index: hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java (revision 1359337) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java (working copy) @@ -53,12 +53,12 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.io.DataOutputOutputStream; import org.apache.hadoop.hbase.protobuf.generated.RPCProtos; +import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException; +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.ConnectionHeader; -import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest; -import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse; -import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.Status; +import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader; import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation; import org.apache.hadoop.hbase.security.HBaseSaslRpcClient; import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.AuthMethod; @@ -82,7 +82,6 @@ import org.apache.hadoop.security.token.TokenSelector; import org.apache.hadoop.util.ReflectionUtils; -import com.google.protobuf.ByteString; /** A client for an IPC service. IPC calls take a single {@link Writable} as a * parameter, and return a {@link Writable} as their value. A service runs on @@ -826,17 +825,15 @@ try { if (LOG.isDebugEnabled()) LOG.debug(getName() + " sending #" + call.id); - RpcRequest.Builder builder = RPCProtos.RpcRequest.newBuilder(); + RpcRequestHeader.Builder builder = RPCProtos.RpcRequestHeader.newBuilder(); builder.setCallId(call.id); - Invocation invocation = (Invocation)call.param; DataOutputBuffer d = new DataOutputBuffer(); - invocation.write(d); - builder.setRequest(ByteString.copyFrom(d.getData())); + builder.build().writeDelimitedTo(d); + call.param.write(d); //noinspection SynchronizeOnNonFinalField synchronized (this.out) { // FindBugs IS2_INCONSISTENT_SYNC - RpcRequest obj = builder.build(); - this.out.writeInt(obj.getSerializedSize()); - obj.writeTo(DataOutputOutputStream.constructOutputStream(this.out)); + this.out.writeInt(d.getLength()); + this.out.write(d.getData(), 0, d.getLength()); this.out.flush(); } } catch(IOException e) { @@ -859,7 +856,7 @@ // so the exception name/trace), and the response bytes // Read the call id. - RpcResponse response = RpcResponse.parseDelimitedFrom(in); + 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. @@ -873,11 +870,8 @@ Status status = response.getStatus(); if (status == Status.SUCCESS) { - ByteString responseObj = response.getResponse(); - DataInputStream dis = - new DataInputStream(responseObj.newInput()); Writable value = ReflectionUtils.newInstance(valueClass, conf); - value.readFields(dis); // read value + value.readFields(in); // read value // 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) { @@ -885,18 +879,20 @@ } calls.remove(id); } else if (status == Status.ERROR) { + RpcException exceptionResponse = RpcException.parseDelimitedFrom(in); if (call != null) { //noinspection ThrowableInstanceNeverThrown call.setException(new RemoteException( - response.getException().getExceptionName(), - response.getException().getStackTrace())); + exceptionResponse.getExceptionName(), + exceptionResponse.getStackTrace())); calls.remove(id); } } else if (status == Status.FATAL) { + RpcException exceptionResponse = RpcException.parseDelimitedFrom(in); // Close the connection markClosed(new RemoteException( - response.getException().getExceptionName(), - response.getException().getStackTrace())); + exceptionResponse.getExceptionName(), + exceptionResponse.getStackTrace())); } } catch (IOException e) { if (e instanceof SocketTimeoutException && remoteId.rpcTimeout > 0) { Index: hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRPC.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRPC.java (revision 1359337) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRPC.java (working copy) @@ -34,6 +34,7 @@ import javax.net.SocketFactory; import java.io.IOException; +import java.lang.reflect.Field; import java.lang.reflect.Proxy; import java.net.ConnectException; import java.net.InetSocketAddress; @@ -101,6 +102,13 @@ } }; + static long getProtocolVersion(Class protocol) + throws NoSuchFieldException, IllegalAccessException { + Field versionField = protocol.getField("VERSION"); + versionField.setAccessible(true); + return versionField.getLong(protocol); + } + // set a protocol to use a non-default RpcEngine static void setProtocolEngine(Configuration conf, Class protocol, Class engine) { @@ -333,16 +341,21 @@ long clientVersion, InetSocketAddress addr, User ticket, Configuration conf, SocketFactory factory, int rpcTimeout) throws IOException { - VersionedProtocol proxy = - getProtocolEngine(protocol,conf) - .getProxy(protocol, clientVersion, addr, ticket, conf, factory, Math.min(rpcTimeout, HBaseRPC.getRpcTimeout())); - long serverVersion = proxy.getProtocolVersion(protocol.getName(), - clientVersion); - if (serverVersion == clientVersion) { - return proxy; + RpcEngine engine = getProtocolEngine(protocol,conf); + VersionedProtocol proxy = engine + .getProxy(protocol, clientVersion, addr, ticket, conf, factory, + Math.min(rpcTimeout, HBaseRPC.getRpcTimeout())); + if (engine instanceof WritableRpcEngine) { + long serverVersion = proxy.getProtocolVersion(protocol.getName(), + clientVersion); + if (serverVersion == clientVersion) { + return proxy; + } + + throw new VersionMismatch(protocol.getName(), clientVersion, + serverVersion); } - throw new VersionMismatch(protocol.getName(), clientVersion, - serverVersion); + return proxy; } /** Index: hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java (revision 1359337) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java (working copy) @@ -73,11 +73,11 @@ import org.apache.hadoop.hbase.io.HbaseObjectWritable; import org.apache.hadoop.hbase.io.WritableWithSize; import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader; -import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest; -import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse; import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException; -import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.Status; +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.UserInformation; +import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Status; import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler; import org.apache.hadoop.hbase.monitoring.TaskMonitor; import org.apache.hadoop.hbase.security.User; @@ -378,23 +378,21 @@ } ByteBufferOutputStream buf = new ByteBufferOutputStream(size); + DataOutputStream out = new DataOutputStream(buf); try { - RpcResponse.Builder builder = RpcResponse.newBuilder(); + RpcResponseHeader.Builder builder = RpcResponseHeader.newBuilder(); // 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); - builder.setException(b.build()); + b.build().writeDelimitedTo(out); } else { - DataOutputBuffer d = new DataOutputBuffer(size); - result.write(d); - byte[] response = d.getData(); - builder.setResponse(ByteString.copyFrom(response)); + result.write(out); } - builder.build().writeDelimitedTo(buf); if (connection.useWrap) { wrapWithSasl(buf); } @@ -1613,9 +1611,10 @@ } protected void processData(byte[] buf) throws IOException, InterruptedException { - RpcRequest request = RpcRequest.parseFrom(buf); + DataInputStream dis = + new DataInputStream(new ByteArrayInputStream(buf)); + RpcRequestHeader request = RpcRequestHeader.parseDelimitedFrom(dis); int id = request.getCallId(); - ByteString clientRequest = request.getRequest(); long callSize = buf.length; if (LOG.isDebugEnabled()) { @@ -1636,8 +1635,6 @@ Writable param; try { - DataInputStream dis = - new DataInputStream(clientRequest.newInput()); param = ReflectionUtils.newInstance(paramClass, conf);//read param param.readFields(dis); } catch (Throwable t) { Index: hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/Invocation.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/Invocation.java (revision 1359337) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/Invocation.java (working copy) @@ -57,7 +57,7 @@ // For generated protocol classes which don't have VERSION field, // such as protobuf interfaces. - private static final Map, Long> + static final Map, Long> PROTOCOL_VERSION = new HashMap, Long>(); static { Index: hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ProtobufRpcEngine.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ProtobufRpcEngine.java (revision 0) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ProtobufRpcEngine.java (revision 0) @@ -0,0 +1,511 @@ +/** + * 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.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.lang.reflect.InvocationHandler; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.lang.reflect.Proxy; +import java.net.InetSocketAddress; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import javax.net.SocketFactory; + +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.io.DataOutputOutputStream; +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.security.User; +import org.apache.hadoop.io.*; +import org.apache.hadoop.hbase.util.Objects; +import org.apache.hadoop.hbase.util.ProtoUtil; + +import com.google.protobuf.Message; +import com.google.protobuf.ServiceException; +/** + * The {@link RpcEngine} implementation for ProtoBuf-based RPCs. + */ +@InterfaceAudience.Private +class ProtobufRpcEngine implements RpcEngine { + private static final Log LOG = + LogFactory.getLog("org.apache.hadoop.hbase.ipc.ProtobufRpcEngine"); + protected final static ClientCache CLIENTS = new ClientCache(); + @Override + public VersionedProtocol getProxy( + Class protocol, long clientVersion, + InetSocketAddress addr, User ticket, Configuration conf, + SocketFactory factory, int rpcTimeout) throws IOException { + final Invoker invoker = new Invoker(protocol, addr, ticket, conf, factory, + rpcTimeout); + return (VersionedProtocol)Proxy.newProxyInstance( + protocol.getClassLoader(), new Class[]{protocol}, invoker); + } + + @Override + public void stopProxy(VersionedProtocol proxy) { + if (proxy!=null) { + ((Invoker)Proxy.getInvocationHandler(proxy)).close(); + } + } + + @Override + public Server getServer(Class protocol, + Object instance, Class[] ifaces, String bindAddress, int port, + int numHandlers, int metaHandlerCount, boolean verbose, + Configuration conf, int highPriorityLevel) throws IOException { + return new Server(instance, ifaces, conf, bindAddress, port, numHandlers, + metaHandlerCount, verbose, highPriorityLevel); + } + private static class Invoker implements InvocationHandler { + private final Map returnTypes = + new ConcurrentHashMap(); + private Class protocol; + private InetSocketAddress address; + private User ticket; + private HBaseClient client; + private boolean isClosed = false; + final private int rpcTimeout; + private final long clientProtocolVersion; + + public Invoker(Class protocol, + InetSocketAddress addr, User ticket, Configuration conf, + SocketFactory factory, int rpcTimeout) throws IOException { + this.protocol = protocol; + this.address = addr; + this.ticket = ticket; + this.client = CLIENTS.getClient(conf, factory, RpcResponseWritable.class); + this.rpcTimeout = rpcTimeout; + Long version = Invocation.PROTOCOL_VERSION.get(protocol); + if (version != null) { + this.clientProtocolVersion = version; + } else { + try { + this.clientProtocolVersion = HBaseRPC.getProtocolVersion(protocol); + } catch (NoSuchFieldException e) { + throw new RuntimeException("Exception encountered " + protocol, e); + } catch (IllegalAccessException e) { + throw new RuntimeException("Exception encountered " + protocol, e); + } + } + } + + 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.setRequest(param.toByteString()); + builder.setClientProtocolVersion(clientProtocolVersion); + rpcRequest = builder.build(); + return rpcRequest; + } + + /** + * This is the client side invoker of RPC method. It only throws + * ServiceException, since the invocation proxy expects only + * ServiceException to be thrown by the method in case protobuf service. + * + * ServiceException has the following causes: + *
    + *
  1. Exceptions encountered on the client side in this method are + * set as cause in ServiceException as is.
  2. + *
  3. Exceptions from the server are wrapped in RemoteException and are + * set as cause in ServiceException
  4. + *
+ * + * 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 { + long startTime = 0; + if (LOG.isDebugEnabled()) { + startTime = System.currentTimeMillis(); + } + + RpcRequestBody rpcRequest = constructRpcRequest(method, args); + RpcResponseWritable val = null; + try { + val = (RpcResponseWritable) client.call( + new RpcRequestWritable(rpcRequest), address, protocol, ticket, + rpcTimeout); + + if (LOG.isDebugEnabled()) { + long callTime = System.currentTimeMillis() - startTime; + LOG.debug("Call: " + method.getName() + " " + callTime); + } + + Message protoType = null; + protoType = getReturnProtoType(method); + Message returnMessage; + returnMessage = protoType.newBuilderForType() + .mergeFrom(val.responseMessage).build(); + return returnMessage; + } catch (Throwable e) { + throw new ServiceException(e); + } + } + + synchronized protected void close() { + if (!isClosed) { + isClosed = true; + CLIENTS.stopClient(client); + } + } + + private Message getReturnProtoType(Method method) throws Exception { + if (returnTypes.containsKey(method.getName())) { + return returnTypes.get(method.getName()); + } + + Class returnType = method.getReturnType(); + Method newInstMethod = returnType.getMethod("getDefaultInstance"); + newInstMethod.setAccessible(true); + Message protoType = (Message) newInstMethod.invoke(null, (Object[]) null); + returnTypes.put(method.getName(), protoType); + return protoType; + } + } + + /** + * Writable Wrapper for Protocol Buffer Requests + */ + private static class RpcRequestWritable implements Writable { + RpcRequestBody message; + + @SuppressWarnings("unused") + public RpcRequestWritable() { + } + + RpcRequestWritable(RpcRequestBody message) { + this.message = message; + } + + @Override + public void write(DataOutput out) throws IOException { + ((Message)message).writeDelimitedTo( + DataOutputOutputStream.constructOutputStream(out)); + } + + @Override + public void readFields(DataInput in) throws IOException { + int length = ProtoUtil.readRawVarint32(in); + byte[] bytes = new byte[length]; + in.readFully(bytes); + message = RpcRequestBody.parseFrom(bytes); + } + + public int getSerializedSize() { + return message.getSerializedSize(); + } + + @Override + public String toString() { + return " Client Protocol Version: " + + message.getClientProtocolVersion() + " MethodName: " + + message.getMethodName(); + } + } + + /** + * Writable Wrapper for Protocol Buffer Responses + */ + private static class RpcResponseWritable implements Writable { + byte[] responseMessage; + + @SuppressWarnings("unused") + public RpcResponseWritable() { + } + + public RpcResponseWritable(Message message) { + this.responseMessage = message.toByteArray(); + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeInt(responseMessage.length); + out.write(responseMessage); + } + + @Override + public void readFields(DataInput in) throws IOException { + int length = in.readInt(); + byte[] bytes = new byte[length]; + in.readFully(bytes); + responseMessage = bytes; + } + } + public static class Server extends WritableRpcEngine.Server { + boolean verbose; + Object instance; + Class implementation; + private static final String WARN_RESPONSE_TIME = + "hbase.ipc.warn.response.time"; + private static final String WARN_RESPONSE_SIZE = + "hbase.ipc.warn.response.size"; + + /** Default value for above params */ + private static final int DEFAULT_WARN_RESPONSE_TIME = 10000; // milliseconds + private static final int DEFAULT_WARN_RESPONSE_SIZE = 100 * 1024 * 1024; + + /** Names for suffixed metrics */ + private static final String ABOVE_ONE_SEC_METRIC = ".aboveOneSec."; + + private final int warnResponseTime; + private final int warnResponseSize; + /** + * Construct an RPC server. + * + * @param protocolClass the class of protocol + * @param protocolImpl the protocolImpl whose methods will be called + * @param conf the configuration to use + * @param bindAddress the address to bind on to listen for connection + * @param port the port to listen for connections on + * @param numHandlers the number of method handler threads to run + * @param verbose whether each call should be logged + * @param portRangeConfig A config parameter that can be used to restrict + * the range of ports used when port is 0 (an ephemeral port) + */ + public Server(Object instance, final Class[] ifaces, + Configuration conf, String bindAddress, int port, + int numHandlers, int metaHandlerCount, boolean verbose, + int highPriorityLevel) + throws IOException { + super(instance, ifaces, RpcRequestWritable.class, conf, bindAddress, port, + numHandlers, metaHandlerCount, verbose, highPriorityLevel); + this.verbose = verbose; + this.instance = instance; + this.implementation = instance.getClass(); + // create metrics for the advertised interfaces this server implements. + String [] metricSuffixes = new String [] {ABOVE_ONE_SEC_METRIC}; + this.rpcMetrics.createMetrics(ifaces, false, metricSuffixes); + + this.warnResponseTime = conf.getInt(WARN_RESPONSE_TIME, + DEFAULT_WARN_RESPONSE_TIME); + this.warnResponseSize = conf.getInt(WARN_RESPONSE_SIZE, + DEFAULT_WARN_RESPONSE_SIZE); + } + private final Map methodArg = + new ConcurrentHashMap(); + private final Map methodInstances = + new ConcurrentHashMap(); + @Override + /** + * This is a server side method, which is invoked over RPC. On success + * the return response has protobuf response payload. On failure, the + * exception name and the stack trace are returned in the protobuf response. + */ + public Writable call(Class protocol, + Writable writableRequest, long receiveTime, MonitoredRPCHandler status) + throws IOException { + try { + RpcRequestWritable request = (RpcRequestWritable) writableRequest; + RpcRequestBody rpcRequest = request.message; + String methodName = rpcRequest.getMethodName(); + Method method = getMethod(protocol, methodName); + if (method == null) { + throw new HBaseRPC.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. + */ + //TODO: use the clientVersion to do protocol compatibility checks, and + //this could be used here to handle complex use cases like deciding + //which implementation of the protocol should be used to service the + //current request, etc. Ideally, we shouldn't land up in a situation + //where we need to support such a use case. + //For now the clientVersion field is simply ignored + long clientVersion = rpcRequest.getClientProtocolVersion(); + + if (verbose) { + LOG.info("Call: protocol name=" + protocol.getName() + + ", method=" + methodName); + } + + status.setRPC(rpcRequest.getMethodName(), receiveTime); + status.setRPCPacket(writableRequest); + 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)) { + impl = this.instance; + } else { + throw new HBaseRPC.UnknownProtocolException(protocol); + } + + 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); + } else { + throw new ServiceException("Too many parameters for method: [" + + method.getName() + "]" + ", allowed (at most): 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 + + " contents=" + Objects.describeQuantity(param)); + } + rpcMetrics.rpcQueueTime.inc(qTime); + rpcMetrics.rpcProcessingTime.inc(processingTime); + rpcMetrics.inc(method.getName(), processingTime); + if (verbose) { + WritableRpcEngine.log("Return: "+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); + if (tooSlow || tooLarge) { + // when tagging, we let TooLarge trump TooSmall to keep output simple + // note that large responses will often also be slow. + StringBuilder buffer = new StringBuilder(256); + buffer.append(methodName); + buffer.append("("); + buffer.append(param.getClass().getName()); + buffer.append(")"); + buffer.append(", client version="+clientVersion); + logResponse(new Object[]{rpcRequest.getRequest()}, + methodName, buffer.toString(), (tooLarge ? "TooLarge" : "TooSlow"), + status.getClient(), startTime, processingTime, qTime, + responseSize); + // provides a count of log-reported slow responses + if (tooSlow) { + rpcMetrics.rpcSlowResponseTime.inc(processingTime); + } + } + if (processingTime > 1000) { + // we use a hard-coded one second period so that we can clearly + // indicate the time period we're warning about in the name of the + // metric itself + rpcMetrics.inc(method.getName() + ABOVE_ONE_SEC_METRIC, + processingTime); + } + return new RpcResponseWritable(result); + } catch (InvocationTargetException e) { + Throwable target = e.getTargetException(); + if (target instanceof IOException) { + throw (IOException)target; + } + if (target instanceof ServiceException) { + throw ProtobufUtil.getRemoteException((ServiceException)target); + } + IOException ioe = new IOException(target.toString()); + ioe.setStackTrace(target.getStackTrace()); + throw ioe; + } catch (Throwable e) { + if (!(e instanceof IOException)) { + LOG.error("Unexpected throwable object ", e); + } + IOException ioe = new IOException(e.toString()); + ioe.setStackTrace(e.getStackTrace()); + throw ioe; + } + } + + private Method getMethod(Class protocol, + String methodName) { + Method method = methodInstances.get(methodName); + if (method != null) { + return method; + } + Method[] methods = protocol.getMethods(); + LOG.warn("Methods length : " + methods.length); + for (Method m : methods) { + if (m.getName().equals(methodName)) { + m.setAccessible(true); + methodInstances.put(methodName, m); + return m; + } + } + return null; + } + + private 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; + } + } +} \ No newline at end of file Index: hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/WritableRpcEngine.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/WritableRpcEngine.java (revision 1359337) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/WritableRpcEngine.java (working copy) @@ -28,18 +28,14 @@ import java.net.InetSocketAddress; import java.io.*; -import java.util.HashSet; import java.util.Map; import java.util.HashMap; -import java.util.Set; import javax.net.SocketFactory; import org.apache.commons.logging.*; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.client.AdminProtocol; -import org.apache.hadoop.hbase.client.ClientProtocol; import org.apache.hadoop.hbase.client.Operation; import org.apache.hadoop.hbase.io.HbaseObjectWritable; import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler; @@ -69,57 +65,6 @@ // DEBUG log level does NOT emit RPC-level logging. private static final Log LOG = LogFactory.getLog("org.apache.hadoop.ipc.RPCEngine"); - /* Cache a client using its socket factory as the hash key */ - static private class ClientCache { - private Map clients = - new HashMap(); - - protected ClientCache() {} - - /** - * Construct & cache an IPC client with the user-provided SocketFactory - * if no cached client exists. - * - * @param conf Configuration - * @param factory socket factory - * @return an IPC client - */ - protected synchronized HBaseClient getClient(Configuration conf, - SocketFactory factory) { - // Construct & cache client. The configuration is only used for timeout, - // and Clients have connection pools. So we can either (a) lose some - // connection pooling and leak sockets, or (b) use the same timeout for - // all configurations. Since the IPC is usually intended globally, not - // per-job, we choose (a). - HBaseClient client = clients.get(factory); - if (client == null) { - // Make an hbase client instead of hadoop Client. - client = new HBaseClient(HbaseObjectWritable.class, conf, factory); - clients.put(factory, client); - } else { - client.incCount(); - } - return client; - } - - /** - * Stop a RPC client connection - * A RPC client is closed only when its reference count becomes zero. - * @param client client to stop - */ - protected void stopClient(HBaseClient client) { - synchronized (this) { - client.decCount(); - if (client.isZeroReference()) { - clients.remove(client.getSocketFactory()); - } - } - if (client.isZeroReference()) { - client.stop(); - } - } - } - protected final static ClientCache CLIENTS = new ClientCache(); private static class Invoker implements InvocationHandler { @@ -150,8 +95,8 @@ try { HbaseObjectWritable value = (HbaseObjectWritable) - client.call(new Invocation(method, args), address, - protocol, ticket, rpcTimeout); + client.call(new Invocation(method, args), address, protocol, ticket, + rpcTimeout); if (logDebug) { // FIGURE HOW TO TURN THIS OFF! long callTime = System.currentTimeMillis() - startTime; @@ -279,10 +224,11 @@ * @throws IOException e */ public Server(Object instance, final Class[] ifaces, + Class paramClass, Configuration conf, String bindAddress, int port, int numHandlers, int metaHandlerCount, boolean verbose, int highPriorityLevel) throws IOException { - super(bindAddress, port, Invocation.class, numHandlers, metaHandlerCount, + super(bindAddress, port, paramClass, numHandlers, metaHandlerCount, conf, classNameBase(instance.getClass().getName()), highPriorityLevel); this.instance = instance; @@ -301,6 +247,14 @@ DEFAULT_WARN_RESPONSE_SIZE); } + public Server(Object instance, final Class[] ifaces, + Configuration conf, String bindAddress, int port, + int numHandlers, int metaHandlerCount, boolean verbose, + int highPriorityLevel) throws IOException { + this(instance, ifaces, Invocation.class, conf, bindAddress, port, + numHandlers, metaHandlerCount, verbose, highPriorityLevel); + } + public AuthenticationTokenSecretManager createSecretManager(){ if (!User.isSecurityEnabled() || !(instance instanceof org.apache.hadoop.hbase.Server)) { @@ -341,7 +295,7 @@ throw new IOException("Could not find requested method, the usual " + "cause is a version mismatch between client and server."); } - if (verbose) log("Call: " + call); + if (verbose) log("Call: " + call, LOG); status.setRPC(call.getMethodName(), call.getParameters(), receivedTime); status.setRPCPacket(param); status.resume("Servicing call"); @@ -389,7 +343,7 @@ rpcMetrics.rpcQueueTime.inc(qTime); rpcMetrics.rpcProcessingTime.inc(processingTime); rpcMetrics.inc(call.getMethodName(), processingTime); - if (verbose) log("Return: "+value); + if (verbose) log("Return: "+value, LOG); HbaseObjectWritable retVal = new HbaseObjectWritable(method.getReturnType(), value); @@ -403,7 +357,8 @@ if (tooSlow || tooLarge) { // when tagging, we let TooLarge trump TooSmall to keep output simple // note that large responses will often also be slow. - logResponse(call, (tooLarge ? "TooLarge" : "TooSlow"), + logResponse(call.getParameters(), call.getMethodName(), + call.toString(), (tooLarge ? "TooLarge" : "TooSlow"), status.getClient(), startTime, processingTime, qTime, responseSize); // provides a count of log-reported slow responses @@ -444,7 +399,9 @@ /** * Logs an RPC response to the LOG file, producing valid JSON objects for * client Operations. - * @param call The call to log. + * @param params The parameters received in the call. + * @param methodName The name of the method invoked + * @param call The string representation of the call * @param tag The tag that will be used to indicate this event in the log. * @param client The address of the client who made this call. * @param startTime The time that the call was initiated, in ms. @@ -453,10 +410,10 @@ * prior to being initiated, in ms. * @param responseSize The size in bytes of the response buffer. */ - private void logResponse(Invocation call, String tag, String clientAddress, - long startTime, int processingTime, int qTime, long responseSize) + void logResponse(Object[] params, String methodName, String call, String tag, + String clientAddress, long startTime, int processingTime, int qTime, + long responseSize) throws IOException { - Object params[] = call.getParameters(); // for JSON encoding ObjectMapper mapper = new ObjectMapper(); // base information that is reported regardless of type of call @@ -467,7 +424,7 @@ responseInfo.put("responsesize", responseSize); responseInfo.put("client", clientAddress); responseInfo.put("class", instance.getClass().getSimpleName()); - responseInfo.put("method", call.getMethodName()); + responseInfo.put("method", methodName); if (params.length == 2 && instance instanceof HRegionServer && params[0] instanceof byte[] && params[1] instanceof Operation) { @@ -491,14 +448,14 @@ } else { // can't get JSON details, so just report call.toString() along with // a more generic tag. - responseInfo.put("call", call.toString()); + responseInfo.put("call", call); LOG.warn("(response" + tag + "): " + mapper.writeValueAsString(responseInfo)); } } } - protected static void log(String value) { + protected static void log(String value, Log LOG) { String v = value; if (v != null && v.length() > 55) v = v.substring(0, 55)+"..."; Index: hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (revision 1359337) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (working copy) @@ -29,7 +29,6 @@ import java.util.Collections; import java.util.Comparator; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -80,7 +79,6 @@ import org.apache.hadoop.hbase.MasterAdminProtocol; import org.apache.hadoop.hbase.RegionServerStatusProtocol; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.ipc.ProtocolSignature; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory; @@ -316,7 +314,7 @@ } int numHandlers = conf.getInt("hbase.master.handler.count", conf.getInt("hbase.regionserver.handler.count", 25)); - this.rpcServer = HBaseRPC.getServer(this, + this.rpcServer = HBaseRPC.getServer(MasterMonitorProtocol.class, this, new Class[]{MasterMonitorProtocol.class, MasterAdminProtocol.class, RegionServerStatusProtocol.class}, initialIsa.getHostName(), // BindAddress is IP we got for this server. Index: hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandler.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandler.java (revision 1359337) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandler.java (working copy) @@ -41,6 +41,8 @@ public abstract void setRPC(String methodName, Object [] params, long queueTime); + public abstract void setRPC(String methodName, + long queueTime); public abstract void setRPCPacket(Writable param); public abstract void setConnection(String clientAddress, int remotePort); } Index: hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandlerImpl.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandlerImpl.java (revision 1359337) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandlerImpl.java (working copy) @@ -226,6 +226,14 @@ this.packet = null; } + @Override + public void setRPC(String methodName, long queueTime) { + this.methodName = methodName; + this.rpcStartTime = System.currentTimeMillis(); + this.rpcQueueTime = queueTime; + this.state = State.RUNNING; + } + public synchronized Map toMap() { // only include RPC info if the Handler is actively servicing an RPC call Map map = super.toMap(); @@ -261,5 +269,4 @@ } return super.toString() + ", rpcMethod=" + getRPC(); } - } Index: hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java (revision 1359337) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java (working copy) @@ -1099,47 +1099,43 @@ // @@protoc_insertion_point(class_scope:ConnectionHeader) } - public interface RpcRequestOrBuilder + public interface RpcRequestHeaderOrBuilder extends com.google.protobuf.MessageOrBuilder { - // required int32 callId = 1; + // required uint32 callId = 1; boolean hasCallId(); int getCallId(); - - // optional bytes request = 2; - boolean hasRequest(); - com.google.protobuf.ByteString getRequest(); } - public static final class RpcRequest extends + public static final class RpcRequestHeader extends com.google.protobuf.GeneratedMessage - implements RpcRequestOrBuilder { - // Use RpcRequest.newBuilder() to construct. - private RpcRequest(Builder builder) { + implements RpcRequestHeaderOrBuilder { + // Use RpcRequestHeader.newBuilder() to construct. + private RpcRequestHeader(Builder builder) { super(builder); } - private RpcRequest(boolean noInit) {} + private RpcRequestHeader(boolean noInit) {} - private static final RpcRequest defaultInstance; - public static RpcRequest getDefaultInstance() { + private static final RpcRequestHeader defaultInstance; + public static RpcRequestHeader getDefaultInstance() { return defaultInstance; } - public RpcRequest getDefaultInstanceForType() { + public RpcRequestHeader getDefaultInstanceForType() { return defaultInstance; } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcRequest_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcRequestHeader_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcRequest_fieldAccessorTable; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcRequestHeader_fieldAccessorTable; } private int bitField0_; - // required int32 callId = 1; + // required uint32 callId = 1; public static final int CALLID_FIELD_NUMBER = 1; private int callId_; public boolean hasCallId() { @@ -1149,19 +1145,8 @@ return callId_; } - // optional bytes request = 2; - public static final int REQUEST_FIELD_NUMBER = 2; - private com.google.protobuf.ByteString request_; - public boolean hasRequest() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public com.google.protobuf.ByteString getRequest() { - return request_; - } - private void initFields() { callId_ = 0; - request_ = com.google.protobuf.ByteString.EMPTY; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -1180,11 +1165,8 @@ throws java.io.IOException { getSerializedSize(); if (((bitField0_ & 0x00000001) == 0x00000001)) { - output.writeInt32(1, callId_); + output.writeUInt32(1, callId_); } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeBytes(2, request_); - } getUnknownFields().writeTo(output); } @@ -1196,12 +1178,8 @@ size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeInt32Size(1, callId_); + .computeUInt32Size(1, callId_); } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += com.google.protobuf.CodedOutputStream - .computeBytesSize(2, request_); - } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -1219,10 +1197,10 @@ if (obj == this) { return true; } - if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest)) { + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader)) { return super.equals(obj); } - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest other = (org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest) obj; + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader other = (org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader) obj; boolean result = true; result = result && (hasCallId() == other.hasCallId()); @@ -1230,11 +1208,6 @@ result = result && (getCallId() == other.getCallId()); } - result = result && (hasRequest() == other.hasRequest()); - if (hasRequest()) { - result = result && getRequest() - .equals(other.getRequest()); - } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -1248,49 +1221,45 @@ hash = (37 * hash) + CALLID_FIELD_NUMBER; hash = (53 * hash) + getCallId(); } - if (hasRequest()) { - hash = (37 * hash) + REQUEST_FIELD_NUMBER; - hash = (53 * hash) + getRequest().hashCode(); - } hash = (29 * hash) + getUnknownFields().hashCode(); return hash; } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader 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.RpcRequest parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader 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.RpcRequest parseFrom(byte[] data) + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader 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.RpcRequest parseFrom(java.io.InputStream input) + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader parseFrom(java.io.InputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader 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.RpcRequest parseDelimitedFrom(java.io.InputStream input) + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -1299,7 +1268,7 @@ return null; } } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest parseDelimitedFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -1310,12 +1279,12 @@ return null; } } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -1325,7 +1294,7 @@ public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest prototype) { + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } @@ -1338,18 +1307,18 @@ } public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder - implements org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestOrBuilder { + implements org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeaderOrBuilder { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcRequest_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcRequestHeader_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcRequest_fieldAccessorTable; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcRequestHeader_fieldAccessorTable; } - // Construct using org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest.newBuilder() + // Construct using org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -1370,8 +1339,6 @@ super.clear(); callId_ = 0; bitField0_ = (bitField0_ & ~0x00000001); - request_ = com.google.protobuf.ByteString.EMPTY; - bitField0_ = (bitField0_ & ~0x00000002); return this; } @@ -1381,24 +1348,24 @@ public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest.getDescriptor(); + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader.getDescriptor(); } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest getDefaultInstanceForType() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest.getDefaultInstance(); + 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.RpcRequest build() { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest result = buildPartial(); + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader build() { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } return result; } - private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest buildParsed() + private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest result = buildPartial(); + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); @@ -1406,40 +1373,33 @@ return result; } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest buildPartial() { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest result = new org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest(this); + 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); 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; - } - result.request_ = request_; 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.RpcRequest) { - return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest)other); + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader)other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest other) { - if (other == org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest.getDefaultInstance()) 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.hasRequest()) { - setRequest(other.getRequest()); - } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -1477,21 +1437,16 @@ } case 8: { bitField0_ |= 0x00000001; - callId_ = input.readInt32(); + callId_ = input.readUInt32(); break; } - case 18: { - bitField0_ |= 0x00000002; - request_ = input.readBytes(); - break; - } } } } private int bitField0_; - // required int32 callId = 1; + // required uint32 callId = 1; private int callId_ ; public boolean hasCallId() { return ((bitField0_ & 0x00000001) == 0x00000001); @@ -1512,89 +1467,69 @@ return this; } - // optional bytes request = 2; - private com.google.protobuf.ByteString request_ = com.google.protobuf.ByteString.EMPTY; - public boolean hasRequest() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public com.google.protobuf.ByteString getRequest() { - return request_; - } - public Builder setRequest(com.google.protobuf.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000002; - request_ = value; - onChanged(); - return this; - } - public Builder clearRequest() { - bitField0_ = (bitField0_ & ~0x00000002); - request_ = getDefaultInstance().getRequest(); - onChanged(); - return this; - } - - // @@protoc_insertion_point(builder_scope:RpcRequest) + // @@protoc_insertion_point(builder_scope:RpcRequestHeader) } static { - defaultInstance = new RpcRequest(true); + defaultInstance = new RpcRequestHeader(true); defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:RpcRequest) + // @@protoc_insertion_point(class_scope:RpcRequestHeader) } - public interface RpcExceptionOrBuilder + public interface RpcRequestBodyOrBuilder extends com.google.protobuf.MessageOrBuilder { - // required string exceptionName = 1; - boolean hasExceptionName(); - String getExceptionName(); + // required string methodName = 1; + boolean hasMethodName(); + String getMethodName(); - // optional string stackTrace = 2; - boolean hasStackTrace(); - String getStackTrace(); + // optional uint64 clientProtocolVersion = 2; + boolean hasClientProtocolVersion(); + long getClientProtocolVersion(); + + // optional bytes request = 3; + boolean hasRequest(); + com.google.protobuf.ByteString getRequest(); } - public static final class RpcException extends + public static final class RpcRequestBody extends com.google.protobuf.GeneratedMessage - implements RpcExceptionOrBuilder { - // Use RpcException.newBuilder() to construct. - private RpcException(Builder builder) { + implements RpcRequestBodyOrBuilder { + // Use RpcRequestBody.newBuilder() to construct. + private RpcRequestBody(Builder builder) { super(builder); } - private RpcException(boolean noInit) {} + private RpcRequestBody(boolean noInit) {} - private static final RpcException defaultInstance; - public static RpcException getDefaultInstance() { + private static final RpcRequestBody defaultInstance; + public static RpcRequestBody getDefaultInstance() { return defaultInstance; } - public RpcException getDefaultInstanceForType() { + public RpcRequestBody 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_RpcRequestBody_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_RpcRequestBody_fieldAccessorTable; } private int bitField0_; - // required string exceptionName = 1; - public static final int EXCEPTIONNAME_FIELD_NUMBER = 1; - private java.lang.Object exceptionName_; - public boolean hasExceptionName() { + // required string methodName = 1; + public static final int METHODNAME_FIELD_NUMBER = 1; + private java.lang.Object methodName_; + public boolean hasMethodName() { return ((bitField0_ & 0x00000001) == 0x00000001); } - public String getExceptionName() { - java.lang.Object ref = exceptionName_; + public String getMethodName() { + java.lang.Object ref = methodName_; if (ref instanceof String) { return (String) ref; } else { @@ -1602,65 +1537,54 @@ (com.google.protobuf.ByteString) ref; String s = bs.toStringUtf8(); if (com.google.protobuf.Internal.isValidUtf8(bs)) { - exceptionName_ = s; + methodName_ = s; } return s; } } - private com.google.protobuf.ByteString getExceptionNameBytes() { - java.lang.Object ref = exceptionName_; + 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); - exceptionName_ = b; + methodName_ = b; return b; } else { return (com.google.protobuf.ByteString) ref; } } - // optional string stackTrace = 2; - public static final int STACKTRACE_FIELD_NUMBER = 2; - private java.lang.Object stackTrace_; - public boolean hasStackTrace() { + // optional uint64 clientProtocolVersion = 2; + public static final int CLIENTPROTOCOLVERSION_FIELD_NUMBER = 2; + private long clientProtocolVersion_; + public boolean hasClientProtocolVersion() { 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 long getClientProtocolVersion() { + return clientProtocolVersion_; } - 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 bytes request = 3; + public static final int REQUEST_FIELD_NUMBER = 3; + private com.google.protobuf.ByteString request_; + public boolean hasRequest() { + return ((bitField0_ & 0x00000004) == 0x00000004); } + public com.google.protobuf.ByteString getRequest() { + return request_; + } private void initFields() { - exceptionName_ = ""; - stackTrace_ = ""; + methodName_ = ""; + clientProtocolVersion_ = 0L; + request_ = com.google.protobuf.ByteString.EMPTY; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - if (!hasExceptionName()) { + if (!hasMethodName()) { memoizedIsInitialized = 0; return false; } @@ -1672,11 +1596,14 @@ throws java.io.IOException { getSerializedSize(); if (((bitField0_ & 0x00000001) == 0x00000001)) { - output.writeBytes(1, getExceptionNameBytes()); + output.writeBytes(1, getMethodNameBytes()); } if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeBytes(2, getStackTraceBytes()); + output.writeUInt64(2, clientProtocolVersion_); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, request_); + } getUnknownFields().writeTo(output); } @@ -1688,12 +1615,16 @@ size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeBytesSize(1, getExceptionNameBytes()); + .computeBytesSize(1, getMethodNameBytes()); } if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeBytesSize(2, getStackTraceBytes()); + .computeUInt64Size(2, clientProtocolVersion_); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(3, request_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -1711,22 +1642,27 @@ 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.RpcRequestBody)) { 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.RpcRequestBody other = (org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody) obj; boolean result = true; - result = result && (hasExceptionName() == other.hasExceptionName()); - if (hasExceptionName()) { - result = result && getExceptionName() - .equals(other.getExceptionName()); + result = result && (hasMethodName() == other.hasMethodName()); + if (hasMethodName()) { + result = result && getMethodName() + .equals(other.getMethodName()); } - result = result && (hasStackTrace() == other.hasStackTrace()); - if (hasStackTrace()) { - result = result && getStackTrace() - .equals(other.getStackTrace()); + result = result && (hasClientProtocolVersion() == other.hasClientProtocolVersion()); + if (hasClientProtocolVersion()) { + result = result && (getClientProtocolVersion() + == other.getClientProtocolVersion()); } + result = result && (hasRequest() == other.hasRequest()); + if (hasRequest()) { + result = result && getRequest() + .equals(other.getRequest()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -1736,53 +1672,57 @@ 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 (hasMethodName()) { + hash = (37 * hash) + METHODNAME_FIELD_NUMBER; + hash = (53 * hash) + getMethodName().hashCode(); } - if (hasStackTrace()) { - hash = (37 * hash) + STACKTRACE_FIELD_NUMBER; - hash = (53 * hash) + getStackTrace().hashCode(); + if (hasClientProtocolVersion()) { + hash = (37 * hash) + CLIENTPROTOCOLVERSION_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getClientProtocolVersion()); } + if (hasRequest()) { + hash = (37 * hash) + REQUEST_FIELD_NUMBER; + hash = (53 * hash) + getRequest().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.RpcRequestBody 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.RpcRequestBody 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.RpcRequestBody 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.RpcRequestBody 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.RpcRequestBody 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.RpcRequestBody 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.RpcRequestBody parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -1791,7 +1731,7 @@ return null; } } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException parseDelimitedFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -1802,12 +1742,12 @@ return null; } } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody 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.RpcRequestBody parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -1817,7 +1757,7 @@ 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.RpcRequestBody prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } @@ -1830,18 +1770,18 @@ } 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.RpcRequestBodyOrBuilder { 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_RpcRequestBody_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_RpcRequestBody_fieldAccessorTable; } - // Construct using org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.newBuilder() + // Construct using org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -1860,10 +1800,12 @@ public Builder clear() { super.clear(); - exceptionName_ = ""; + methodName_ = ""; bitField0_ = (bitField0_ & ~0x00000001); - stackTrace_ = ""; + clientProtocolVersion_ = 0L; bitField0_ = (bitField0_ & ~0x00000002); + request_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000004); return this; } @@ -1873,24 +1815,24 @@ 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.RpcRequestBody.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.RpcRequestBody getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody.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.RpcRequestBody build() { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody 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.RpcRequestBody buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException result = buildPartial(); + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); @@ -1898,46 +1840,53 @@ 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.RpcRequestBody buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody result = new org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody(this); int from_bitField0_ = bitField0_; int to_bitField0_ = 0; if (((from_bitField0_ & 0x00000001) == 0x00000001)) { to_bitField0_ |= 0x00000001; } - result.exceptionName_ = exceptionName_; + result.methodName_ = methodName_; if (((from_bitField0_ & 0x00000002) == 0x00000002)) { to_bitField0_ |= 0x00000002; } - result.stackTrace_ = stackTrace_; + result.clientProtocolVersion_ = clientProtocolVersion_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.request_ = request_; 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.RpcRequestBody) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody)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.RpcRequestBody other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody.getDefaultInstance()) return this; + if (other.hasMethodName()) { + setMethodName(other.getMethodName()); } - if (other.hasStackTrace()) { - setStackTrace(other.getStackTrace()); + if (other.hasClientProtocolVersion()) { + setClientProtocolVersion(other.getClientProtocolVersion()); } + if (other.hasRequest()) { + setRequest(other.getRequest()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } public final boolean isInitialized() { - if (!hasExceptionName()) { + if (!hasMethodName()) { return false; } @@ -1969,149 +1918,154 @@ } case 10: { bitField0_ |= 0x00000001; - exceptionName_ = input.readBytes(); + methodName_ = input.readBytes(); break; } - case 18: { + case 16: { bitField0_ |= 0x00000002; - stackTrace_ = input.readBytes(); + clientProtocolVersion_ = input.readUInt64(); break; } + case 26: { + bitField0_ |= 0x00000004; + request_ = input.readBytes(); + break; + } } } } private int bitField0_; - // required string exceptionName = 1; - private java.lang.Object exceptionName_ = ""; - public boolean hasExceptionName() { + // required string methodName = 1; + private java.lang.Object methodName_ = ""; + public boolean hasMethodName() { return ((bitField0_ & 0x00000001) == 0x00000001); } - public String getExceptionName() { - java.lang.Object ref = exceptionName_; + public String getMethodName() { + java.lang.Object ref = methodName_; if (!(ref instanceof String)) { String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); - exceptionName_ = s; + methodName_ = s; return s; } else { return (String) ref; } } - public Builder setExceptionName(String value) { + public Builder setMethodName(String value) { if (value == null) { throw new NullPointerException(); } bitField0_ |= 0x00000001; - exceptionName_ = value; + methodName_ = value; onChanged(); return this; } - public Builder clearExceptionName() { + public Builder clearMethodName() { bitField0_ = (bitField0_ & ~0x00000001); - exceptionName_ = getDefaultInstance().getExceptionName(); + methodName_ = getDefaultInstance().getMethodName(); onChanged(); return this; } - void setExceptionName(com.google.protobuf.ByteString value) { + void setMethodName(com.google.protobuf.ByteString value) { bitField0_ |= 0x00000001; - exceptionName_ = value; + methodName_ = value; onChanged(); } - // optional string stackTrace = 2; - private java.lang.Object stackTrace_ = ""; - public boolean hasStackTrace() { + // optional uint64 clientProtocolVersion = 2; + private long clientProtocolVersion_ ; + public boolean hasClientProtocolVersion() { 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; - } else { - return (String) ref; - } + public long getClientProtocolVersion() { + return clientProtocolVersion_; } - public Builder setStackTrace(String value) { + public Builder setClientProtocolVersion(long value) { + bitField0_ |= 0x00000002; + clientProtocolVersion_ = value; + onChanged(); + return this; + } + public Builder clearClientProtocolVersion() { + bitField0_ = (bitField0_ & ~0x00000002); + clientProtocolVersion_ = 0L; + onChanged(); + return this; + } + + // optional bytes request = 3; + private com.google.protobuf.ByteString request_ = com.google.protobuf.ByteString.EMPTY; + public boolean hasRequest() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public com.google.protobuf.ByteString getRequest() { + return request_; + } + public Builder setRequest(com.google.protobuf.ByteString value) { if (value == null) { throw new NullPointerException(); } - bitField0_ |= 0x00000002; - stackTrace_ = value; + bitField0_ |= 0x00000004; + request_ = value; onChanged(); return this; } - public Builder clearStackTrace() { - bitField0_ = (bitField0_ & ~0x00000002); - stackTrace_ = getDefaultInstance().getStackTrace(); + public Builder clearRequest() { + bitField0_ = (bitField0_ & ~0x00000004); + request_ = getDefaultInstance().getRequest(); onChanged(); return this; } - void setStackTrace(com.google.protobuf.ByteString value) { - bitField0_ |= 0x00000002; - stackTrace_ = value; - onChanged(); - } - // @@protoc_insertion_point(builder_scope:RpcException) + // @@protoc_insertion_point(builder_scope:RpcRequestBody) } static { - defaultInstance = new RpcException(true); + defaultInstance = new RpcRequestBody(true); defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:RpcException) + // @@protoc_insertion_point(class_scope:RpcRequestBody) } - public interface RpcResponseOrBuilder + public interface RpcResponseHeaderOrBuilder extends com.google.protobuf.MessageOrBuilder { // required int32 callId = 1; boolean hasCallId(); int getCallId(); - // required .RpcResponse.Status status = 2; + // required .RpcResponseHeader.Status status = 2; boolean hasStatus(); - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.Status getStatus(); - - // optional bytes response = 3; - boolean hasResponse(); - com.google.protobuf.ByteString getResponse(); - - // optional .RpcException exception = 4; - boolean hasException(); - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException getException(); - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcExceptionOrBuilder getExceptionOrBuilder(); + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Status getStatus(); } - public static final class RpcResponse extends + public static final class RpcResponseHeader extends com.google.protobuf.GeneratedMessage - implements RpcResponseOrBuilder { - // Use RpcResponse.newBuilder() to construct. - private RpcResponse(Builder builder) { + implements RpcResponseHeaderOrBuilder { + // Use RpcResponseHeader.newBuilder() to construct. + private RpcResponseHeader(Builder builder) { super(builder); } - private RpcResponse(boolean noInit) {} + private RpcResponseHeader(boolean noInit) {} - private static final RpcResponse defaultInstance; - public static RpcResponse getDefaultInstance() { + private static final RpcResponseHeader defaultInstance; + public static RpcResponseHeader getDefaultInstance() { return defaultInstance; } - public RpcResponse getDefaultInstanceForType() { + public RpcResponseHeader getDefaultInstanceForType() { return defaultInstance; } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcResponse_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcResponseHeader_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcResponse_fieldAccessorTable; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcResponseHeader_fieldAccessorTable; } public enum Status @@ -2159,7 +2113,7 @@ } public static final com.google.protobuf.Descriptors.EnumDescriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.getDescriptor().getEnumTypes().get(0); + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.getDescriptor().getEnumTypes().get(0); } private static final Status[] VALUES = { @@ -2183,7 +2137,7 @@ this.value = value; } - // @@protoc_insertion_point(enum_scope:RpcResponse.Status) + // @@protoc_insertion_point(enum_scope:RpcResponseHeader.Status) } private int bitField0_; @@ -2197,44 +2151,19 @@ return callId_; } - // required .RpcResponse.Status status = 2; + // required .RpcResponseHeader.Status status = 2; public static final int STATUS_FIELD_NUMBER = 2; - private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.Status status_; + private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Status status_; public boolean hasStatus() { return ((bitField0_ & 0x00000002) == 0x00000002); } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.Status getStatus() { + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Status getStatus() { return status_; } - // optional bytes response = 3; - public static final int RESPONSE_FIELD_NUMBER = 3; - private com.google.protobuf.ByteString response_; - public boolean hasResponse() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public com.google.protobuf.ByteString getResponse() { - return response_; - } - - // optional .RpcException exception = 4; - public static final int EXCEPTION_FIELD_NUMBER = 4; - private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException exception_; - public boolean hasException() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException getException() { - return exception_; - } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcExceptionOrBuilder getExceptionOrBuilder() { - return exception_; - } - private void initFields() { callId_ = 0; - status_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.Status.SUCCESS; - response_ = com.google.protobuf.ByteString.EMPTY; - exception_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.getDefaultInstance(); + status_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Status.SUCCESS; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -2249,12 +2178,6 @@ memoizedIsInitialized = 0; return false; } - if (hasException()) { - if (!getException().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } memoizedIsInitialized = 1; return true; } @@ -2268,12 +2191,6 @@ if (((bitField0_ & 0x00000002) == 0x00000002)) { output.writeEnum(2, status_.getNumber()); } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeBytes(3, response_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeMessage(4, exception_); - } getUnknownFields().writeTo(output); } @@ -2291,14 +2208,6 @@ size += com.google.protobuf.CodedOutputStream .computeEnumSize(2, status_.getNumber()); } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += com.google.protobuf.CodedOutputStream - .computeBytesSize(3, response_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(4, exception_); - } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -2316,10 +2225,10 @@ if (obj == this) { return true; } - if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse)) { + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader)) { return super.equals(obj); } - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse other = (org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse) obj; + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader other = (org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader) obj; boolean result = true; result = result && (hasCallId() == other.hasCallId()); @@ -2332,16 +2241,6 @@ result = result && (getStatus() == other.getStatus()); } - result = result && (hasResponse() == other.hasResponse()); - if (hasResponse()) { - result = result && getResponse() - .equals(other.getResponse()); - } - result = result && (hasException() == other.hasException()); - if (hasException()) { - result = result && getException() - .equals(other.getException()); - } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -2359,53 +2258,45 @@ hash = (37 * hash) + STATUS_FIELD_NUMBER; hash = (53 * hash) + hashEnum(getStatus()); } - if (hasResponse()) { - hash = (37 * hash) + RESPONSE_FIELD_NUMBER; - hash = (53 * hash) + getResponse().hashCode(); - } - if (hasException()) { - hash = (37 * hash) + EXCEPTION_FIELD_NUMBER; - hash = (53 * hash) + getException().hashCode(); - } hash = (29 * hash) + getUnknownFields().hashCode(); return hash; } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader 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.RpcResponse parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader 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.RpcResponse parseFrom(byte[] data) + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader 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.RpcResponse parseFrom(java.io.InputStream input) + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader parseFrom(java.io.InputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader 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.RpcResponse parseDelimitedFrom(java.io.InputStream input) + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -2414,7 +2305,7 @@ return null; } } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse parseDelimitedFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -2425,12 +2316,12 @@ return null; } } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -2440,7 +2331,7 @@ public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse prototype) { + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } @@ -2453,18 +2344,18 @@ } public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder - implements org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseOrBuilder { + implements org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeaderOrBuilder { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcResponse_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcResponseHeader_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcResponse_fieldAccessorTable; + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcResponseHeader_fieldAccessorTable; } - // Construct using org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.newBuilder() + // Construct using org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -2475,7 +2366,6 @@ } private void maybeForceBuilderInitialization() { if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { - getExceptionFieldBuilder(); } } private static Builder create() { @@ -2486,16 +2376,8 @@ super.clear(); callId_ = 0; bitField0_ = (bitField0_ & ~0x00000001); - status_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.Status.SUCCESS; + status_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Status.SUCCESS; bitField0_ = (bitField0_ & ~0x00000002); - response_ = com.google.protobuf.ByteString.EMPTY; - bitField0_ = (bitField0_ & ~0x00000004); - if (exceptionBuilder_ == null) { - exception_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.getDefaultInstance(); - } else { - exceptionBuilder_.clear(); - } - bitField0_ = (bitField0_ & ~0x00000008); return this; } @@ -2505,24 +2387,24 @@ public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.getDescriptor(); + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.getDescriptor(); } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse getDefaultInstanceForType() { - return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.getDefaultInstance(); + 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.RpcResponse build() { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse result = buildPartial(); + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader build() { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } return result; } - private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse buildParsed() + private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse result = buildPartial(); + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); @@ -2530,8 +2412,8 @@ return result; } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse buildPartial() { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse result = new org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse(this); + 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); int from_bitField0_ = bitField0_; int to_bitField0_ = 0; if (((from_bitField0_ & 0x00000001) == 0x00000001)) { @@ -2542,46 +2424,28 @@ to_bitField0_ |= 0x00000002; } result.status_ = status_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.response_ = response_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - if (exceptionBuilder_ == null) { - result.exception_ = exception_; - } else { - result.exception_ = exceptionBuilder_.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.RpcResponse) { - return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse)other); + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader)other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse other) { - if (other == org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.getDefaultInstance()) 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; if (other.hasCallId()) { setCallId(other.getCallId()); } if (other.hasStatus()) { setStatus(other.getStatus()); } - if (other.hasResponse()) { - setResponse(other.getResponse()); - } - if (other.hasException()) { - mergeException(other.getException()); - } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -2595,12 +2459,6 @@ return false; } - if (hasException()) { - if (!getException().isInitialized()) { - - return false; - } - } return true; } @@ -2634,7 +2492,7 @@ } case 16: { int rawValue = input.readEnum(); - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.Status value = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.Status.valueOf(rawValue); + 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 { @@ -2643,20 +2501,6 @@ } break; } - case 26: { - bitField0_ |= 0x00000004; - response_ = input.readBytes(); - break; - } - case 34: { - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.newBuilder(); - if (hasException()) { - subBuilder.mergeFrom(getException()); - } - input.readMessage(subBuilder, extensionRegistry); - setException(subBuilder.buildPartial()); - break; - } } } } @@ -2684,15 +2528,15 @@ return this; } - // required .RpcResponse.Status status = 2; - private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.Status status_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.Status.SUCCESS; + // 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() { return ((bitField0_ & 0x00000002) == 0x00000002); } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.Status getStatus() { + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Status getStatus() { return status_; } - public Builder setStatus(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.Status value) { + public Builder setStatus(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Status value) { if (value == null) { throw new NullPointerException(); } @@ -2703,15 +2547,365 @@ } public Builder clearStatus() { bitField0_ = (bitField0_ & ~0x00000002); - status_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.Status.SUCCESS; + status_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Status.SUCCESS; onChanged(); return this; } - // optional bytes response = 3; + // @@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 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 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; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcResponseBody_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody.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(); + response_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody.getDefaultInstance(); + } + + 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; + } + + 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 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; + } + + 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); + } else { + super.mergeFrom(other); + return this; + } + } + + 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()); + } + 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; + response_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // optional bytes response = 1; private com.google.protobuf.ByteString response_ = com.google.protobuf.ByteString.EMPTY; public boolean hasResponse() { - return ((bitField0_ & 0x00000004) == 0x00000004); + return ((bitField0_ & 0x00000001) == 0x00000001); } public com.google.protobuf.ByteString getResponse() { return response_; @@ -2720,117 +2914,546 @@ if (value == null) { throw new NullPointerException(); } - bitField0_ |= 0x00000004; + bitField0_ |= 0x00000001; response_ = value; onChanged(); return this; } public Builder clearResponse() { - bitField0_ = (bitField0_ & ~0x00000004); + bitField0_ = (bitField0_ & ~0x00000001); response_ = getDefaultInstance().getResponse(); onChanged(); return this; } - // optional .RpcException exception = 4; - private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException exception_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.getDefaultInstance(); - private com.google.protobuf.SingleFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcExceptionOrBuilder> exceptionBuilder_; - public boolean hasException() { - return ((bitField0_ & 0x00000008) == 0x00000008); + // @@protoc_insertion_point(builder_scope:RpcResponseBody) + } + + static { + defaultInstance = new RpcResponseBody(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:RpcResponseBody) + } + + public interface RpcExceptionOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string exceptionName = 1; + boolean hasExceptionName(); + String getExceptionName(); + + // optional string stackTrace = 2; + boolean hasStackTrace(); + String getStackTrace(); + } + public static final class RpcException extends + com.google.protobuf.GeneratedMessage + implements RpcExceptionOrBuilder { + // Use RpcException.newBuilder() to construct. + private RpcException(Builder builder) { + super(builder); + } + private RpcException(boolean noInit) {} + + private static final RpcException defaultInstance; + public static RpcException getDefaultInstance() { + return defaultInstance; + } + + public RpcException getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcException_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcException_fieldAccessorTable; + } + + private int bitField0_; + // required string exceptionName = 1; + public static final int EXCEPTIONNAME_FIELD_NUMBER = 1; + private java.lang.Object exceptionName_; + public boolean hasExceptionName() { + 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; } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException getException() { - if (exceptionBuilder_ == null) { - return exception_; - } else { - return exceptionBuilder_.getMessage(); + } + 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; + } + } + + // 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 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 Builder setException(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException value) { - if (exceptionBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - exception_ = value; - onChanged(); - } else { - exceptionBuilder_.setMessage(value); + } + 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; + } + } + + private void initFields() { + exceptionName_ = ""; + stackTrace_ = ""; + } + 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; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getExceptionNameBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getStackTraceBytes()); + } + 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, getExceptionNameBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(2, getStackTraceBytes()); + } + 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.RpcException)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException other = (org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException) obj; + + boolean result = true; + result = result && (hasExceptionName() == other.hasExceptionName()); + if (hasExceptionName()) { + result = result && getExceptionName() + .equals(other.getExceptionName()); + } + result = result && (hasStackTrace() == other.hasStackTrace()); + if (hasStackTrace()) { + result = result && getStackTrace() + .equals(other.getStackTrace()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + 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 (hasStackTrace()) { + hash = (37 * hash) + STACKTRACE_FIELD_NUMBER; + hash = (53 * hash) + getStackTrace().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException 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( + 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) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException 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) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException 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) + 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.RPCProtos.RpcException 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.RPCProtos.RpcException 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( + 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.RpcException 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.RpcExceptionOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcException_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcException_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { } - bitField0_ |= 0x00000008; + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + exceptionName_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + stackTrace_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); return this; } - public Builder setException( - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.Builder builderForValue) { - if (exceptionBuilder_ == null) { - exception_ = builderForValue.build(); - onChanged(); + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.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.RpcException build() { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + 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); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.exceptionName_ = exceptionName_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.stackTrace_ = stackTrace_; + 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); } else { - exceptionBuilder_.setMessage(builderForValue.build()); + super.mergeFrom(other); + return this; } - bitField0_ |= 0x00000008; + } + + 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()); + } + if (other.hasStackTrace()) { + setStackTrace(other.getStackTrace()); + } + this.mergeUnknownFields(other.getUnknownFields()); return this; } - public Builder mergeException(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException value) { - if (exceptionBuilder_ == null) { - if (((bitField0_ & 0x00000008) == 0x00000008) && - exception_ != org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.getDefaultInstance()) { - exception_ = - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.newBuilder(exception_).mergeFrom(value).buildPartial(); - } else { - exception_ = value; + + public final boolean isInitialized() { + if (!hasExceptionName()) { + + return false; + } + 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; + exceptionName_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + stackTrace_ = input.readBytes(); + break; + } } - onChanged(); - } else { - exceptionBuilder_.mergeFrom(value); } - bitField0_ |= 0x00000008; - return this; } - public Builder clearException() { - if (exceptionBuilder_ == null) { - exception_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.getDefaultInstance(); - onChanged(); + + private int bitField0_; + + // required string exceptionName = 1; + private java.lang.Object exceptionName_ = ""; + public boolean hasExceptionName() { + 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 { - exceptionBuilder_.clear(); + return (String) ref; } - bitField0_ = (bitField0_ & ~0x00000008); + } + public Builder setExceptionName(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + exceptionName_ = value; + onChanged(); return this; } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.Builder getExceptionBuilder() { - bitField0_ |= 0x00000008; + public Builder clearExceptionName() { + bitField0_ = (bitField0_ & ~0x00000001); + exceptionName_ = getDefaultInstance().getExceptionName(); onChanged(); - return getExceptionFieldBuilder().getBuilder(); + return this; } - public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcExceptionOrBuilder getExceptionOrBuilder() { - if (exceptionBuilder_ != null) { - return exceptionBuilder_.getMessageOrBuilder(); + void setExceptionName(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + exceptionName_ = value; + onChanged(); + } + + // optional string stackTrace = 2; + private java.lang.Object stackTrace_ = ""; + public boolean hasStackTrace() { + 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; } else { - return exception_; + return (String) ref; } } - private com.google.protobuf.SingleFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcExceptionOrBuilder> - getExceptionFieldBuilder() { - if (exceptionBuilder_ == null) { - exceptionBuilder_ = new com.google.protobuf.SingleFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcExceptionOrBuilder>( - exception_, - getParentForChildren(), - isClean()); - exception_ = null; - } - return exceptionBuilder_; + public Builder setStackTrace(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + stackTrace_ = value; + onChanged(); + return this; } + public Builder clearStackTrace() { + bitField0_ = (bitField0_ & ~0x00000002); + stackTrace_ = getDefaultInstance().getStackTrace(); + onChanged(); + return this; + } + void setStackTrace(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000002; + stackTrace_ = value; + onChanged(); + } - // @@protoc_insertion_point(builder_scope:RpcResponse) + // @@protoc_insertion_point(builder_scope:RpcException) } static { - defaultInstance = new RpcResponse(true); + defaultInstance = new RpcException(true); defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:RpcResponse) + // @@protoc_insertion_point(class_scope:RpcException) } private static com.google.protobuf.Descriptors.Descriptor @@ -2844,20 +3467,30 @@ com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_ConnectionHeader_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_RpcRequest_descriptor; + internal_static_RpcRequestHeader_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_RpcRequest_fieldAccessorTable; + internal_static_RpcRequestHeader_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_RpcException_descriptor; + internal_static_RpcRequestBody_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_RpcException_fieldAccessorTable; + internal_static_RpcRequestBody_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_RpcResponse_descriptor; + internal_static_RpcResponseHeader_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_RpcResponse_fieldAccessorTable; + internal_static_RpcResponseHeader_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_RpcResponseBody_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; public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -2871,16 +3504,17 @@ "iveUser\030\001 \002(\t\022\020\n\010realUser\030\002 \001(\t\"w\n\020Conne" + "ctionHeader\022\"\n\010userInfo\030\001 \001(\0132\020.UserInfo" + "rmation\022?\n\010protocol\030\002 \001(\t:-org.apache.ha" + - "doop.hbase.client.ClientProtocol\"-\n\nRpcR" + - "equest\022\016\n\006callId\030\001 \002(\005\022\017\n\007request\030\002 \001(\014\"" + - "9\n\014RpcException\022\025\n\rexceptionName\030\001 \002(\t\022\022" + - "\n\nstackTrace\030\002 \001(\t\"\243\001\n\013RpcResponse\022\016\n\006ca" + - "llId\030\001 \002(\005\022#\n\006status\030\002 \002(\0162\023.RpcResponse" + - ".Status\022\020\n\010response\030\003 \001(\014\022 \n\texception\030\004", - " \001(\0132\r.RpcException\"+\n\006Status\022\013\n\007SUCCESS" + - "\020\000\022\t\n\005ERROR\020\001\022\t\n\005FATAL\020\002B<\n*org.apache.h" + - "adoop.hbase.protobuf.generatedB\tRPCProto" + - "sH\001\240\001\001" + "doop.hbase.client.ClientProtocol\"\"\n\020RpcR" + + "equestHeader\022\016\n\006callId\030\001 \002(\r\"T\n\016RpcReque" + + "stBody\022\022\n\nmethodName\030\001 \002(\t\022\035\n\025clientProt" + + "ocolVersion\030\002 \001(\004\022\017\n\007request\030\003 \001(\014\"{\n\021Rp" + + "cResponseHeader\022\016\n\006callId\030\001 \002(\005\022)\n\006statu" + + "s\030\002 \002(\0162\031.RpcResponseHeader.Status\"+\n\006St", + "atus\022\013\n\007SUCCESS\020\000\022\t\n\005ERROR\020\001\022\t\n\005FATAL\020\002\"" + + "#\n\017RpcResponseBody\022\020\n\010response\030\001 \001(\014\"9\n\014" + + "RpcException\022\025\n\rexceptionName\030\001 \002(\t\022\022\n\ns" + + "tackTrace\030\002 \001(\tB<\n*org.apache.hadoop.hba" + + "se.protobuf.generatedB\tRPCProtosH\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -2903,30 +3537,46 @@ new java.lang.String[] { "UserInfo", "Protocol", }, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader.class, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader.Builder.class); - internal_static_RpcRequest_descriptor = + internal_static_RpcRequestHeader_descriptor = getDescriptor().getMessageTypes().get(2); - internal_static_RpcRequest_fieldAccessorTable = new + internal_static_RpcRequestHeader_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_RpcRequest_descriptor, - new java.lang.String[] { "CallId", "Request", }, - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest.class, - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest.Builder.class); + internal_static_RpcRequestHeader_descriptor, + new java.lang.String[] { "CallId", }, + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader.class, + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader.Builder.class); + internal_static_RpcRequestBody_descriptor = + getDescriptor().getMessageTypes().get(3); + internal_static_RpcRequestBody_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_RpcRequestBody_descriptor, + new java.lang.String[] { "MethodName", "ClientProtocolVersion", "Request", }, + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody.class, + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody.Builder.class); + internal_static_RpcResponseHeader_descriptor = + getDescriptor().getMessageTypes().get(4); + internal_static_RpcResponseHeader_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 = + 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(3); + getDescriptor().getMessageTypes().get(6); internal_static_RpcException_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_RpcResponse_descriptor = - getDescriptor().getMessageTypes().get(4); - internal_static_RpcResponse_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_RpcResponse_descriptor, - new java.lang.String[] { "CallId", "Status", "Response", "Exception", }, - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.class, - org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.Builder.class); return null; } }; Index: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (revision 1359337) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (working copy) @@ -496,7 +496,7 @@ throw new IllegalArgumentException("Failed resolve of " + initialIsa); } - this.rpcServer = HBaseRPC.getServer(this, + this.rpcServer = HBaseRPC.getServer(AdminProtocol.class, this, new Class[]{ClientProtocol.class, AdminProtocol.class, HBaseRPCErrorHandler.class, OnlineRegions.class}, Index: hbase-server/src/main/protobuf/RPC.proto =================================================================== --- hbase-server/src/main/protobuf/RPC.proto (revision 1359337) +++ hbase-server/src/main/protobuf/RPC.proto (working copy) @@ -26,15 +26,20 @@ * * 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> + * <"hrpc"-bytearray><'5'-byte> * - * For every RPC that the client makes it needs to send the - * RpcRequest. At the data level this looks like - * + * For every RPC that the client makes it needs to send the following + * RpcRequestHeader and the RpcRequestBody. At the data level this looks like: + * + * + * * - * The server sends back a RpcResponse object as response. - * 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 @@ -63,34 +68,30 @@ /** - * The complete RPC request message + * The RPC request header */ -message RpcRequest { +message RpcRequestHeader { /** Monotonically increasing callId, mostly to keep track of RPCs */ - required int32 callId = 1; - /** The request bytes */ - optional bytes request = 2; + required uint32 callId = 1; } - /** - * At the RPC layer, this message is used to indicate - * the server side exception to the RPC client. - * - * HBase RPC client throws an exception indicated - * by exceptionName with the stackTrace. + * The RPC request body */ -message RpcException { - /** Class name of the exception thrown from the server */ - required string exceptionName = 1; +message RpcRequestBody { + /** Name of the RPC method */ + required string methodName = 1; - /** Exception stack trace from the server side */ - optional string stackTrace = 2; + /** protocol version of class declaring the called method */ + optional uint64 clientProtocolVersion = 2; + + /** Bytes corresponding to the client protobuf request */ + optional bytes request = 3; } /** - * The complete RPC response message + * The RPC response header */ -message RpcResponse { +message RpcResponseHeader { /** Echo back the callId the client sent */ required int32 callId = 1; /** Did the RPC execution encounter an error at the server */ @@ -100,8 +101,25 @@ FATAL = 2; } required Status status = 2; - /** Optional response bytes */ - optional bytes response = 3; - /** Optional exception when error is true*/ - optional RpcException exception = 4; +} +/** + * The RPC response body + */ +message RpcResponseBody { + /** Optional response bytes */ + optional bytes response = 1; +} +/** + * At the RPC layer, this message is used to indicate + * the server side exception to the RPC client. + * + * HBase RPC client throws an exception indicated + * by exceptionName with the stackTrace. + */ +message RpcException { + /** Class name of the exception thrown from the server */ + required string exceptionName = 1; + + /** Exception stack trace from the server side */ + optional string stackTrace = 2; } \ No newline at end of file Index: hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java (revision 0) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java (revision 0) @@ -0,0 +1,150 @@ +/** + * 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.IOException; +import java.net.InetSocketAddress; + +import org.apache.hadoop.conf.Configuration; +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.junit.Assert; +import org.junit.Test; +import org.junit.Before; +import org.junit.After; +import org.junit.experimental.categories.Category; + +import com.google.protobuf.RpcController; +import com.google.protobuf.ServiceException; + +/** + * Test for testing protocol buffer based RPC mechanism. + * This test depends on test.proto definition of types in + * hbase-server/src/test/protobuf/test.proto + * and protobuf service definition from + * hbase-server/src/test/protobuf/test_rpc_service.proto + */ +@Category(MediumTests.class) +public class TestProtoBufRpc { + public final static String ADDRESS = "0.0.0.0"; + public final static int PORT = 0; + private static InetSocketAddress addr; + private static Configuration conf; + private static RpcServer server; + + public interface TestRpcService + extends TestProtobufRpcProto.BlockingInterface, VersionedProtocol { + public long VERSION = 1; + } + + public static class PBServerImpl implements TestRpcService { + + @Override + public EmptyResponseProto ping(RpcController unused, + EmptyRequestProto request) throws ServiceException { + return EmptyResponseProto.newBuilder().build(); + } + + @Override + public EchoResponseProto echo(RpcController unused, EchoRequestProto request) + throws ServiceException { + return EchoResponseProto.newBuilder().setMessage(request.getMessage()) + .build(); + } + + @Override + public EmptyResponseProto error(RpcController unused, + EmptyRequestProto request) throws ServiceException { + throw new ServiceException("error", new IOException("error")); + } + + @Override + public long getProtocolVersion(String protocol, long clientVersion) + throws IOException { + // TODO Auto-generated method stub + return 0; + } + + @Override + public ProtocolSignature getProtocolSignature(String protocol, + long clientVersion, int clientMethodsHash) throws IOException { + // TODO Auto-generated method stub + return null; + } + } + + @Before + public void setUp() throws IOException { // Setup server for both protocols + conf = new Configuration(); + // Set RPC engine to protobuf RPC engine + HBaseRPC.setProtocolEngine(conf, TestRpcService.class, ProtobufRpcEngine.class); + + // Create server side implementation + PBServerImpl serverImpl = new PBServerImpl(); + // Get RPC server for server side implementation + server = HBaseRPC.getServer(TestRpcService.class,serverImpl, + new Class[]{TestRpcService.class}, + ADDRESS, PORT, 10, 10, true, conf, 0); + addr = server.getListenerAddress(); + server.start(); + } + + + @After + public void tearDown() throws Exception { + server.stop(); + } + + private static TestRpcService getClient() throws IOException { + // Set RPC engine to protobuf RPC engine + HBaseRPC.setProtocolEngine(conf, TestRpcService.class, + ProtobufRpcEngine.class); + return (TestRpcService) HBaseRPC.getProxy(TestRpcService.class, 0, + addr, conf, 10000); + } + + @Test + public void testProtoBufRpc() throws Exception { + TestRpcService client = getClient(); + testProtoBufRpc(client); + } + + // separated test out so that other tests can call it. + public static void testProtoBufRpc(TestRpcService client) throws Exception { + // Test ping method + EmptyRequestProto emptyRequest = EmptyRequestProto.newBuilder().build(); + client.ping(null, emptyRequest); + + // Test echo method + EchoRequestProto echoRequest = EchoRequestProto.newBuilder() + .setMessage("hello").build(); + EchoResponseProto echoResponse = client.echo(null, echoRequest); + Assert.assertEquals(echoResponse.getMessage(), "hello"); + + // Test error method - error should be thrown as RemoteException + try { + client.error(null, emptyRequest); + Assert.fail("Expected exception is not thrown"); + } catch (ServiceException e) { + } + } +} \ No newline at end of file Index: hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/protobuf/generated/TestProtos.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/protobuf/generated/TestProtos.java (revision 0) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/protobuf/generated/TestProtos.java (revision 0) @@ -0,0 +1,1525 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: test.proto + +package org.apache.hadoop.hbase.ipc.protobuf.generated; + +public final class TestProtos { + private TestProtos() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + } + public interface EmptyRequestProtoOrBuilder + extends com.google.protobuf.MessageOrBuilder { + } + public static final class EmptyRequestProto extends + com.google.protobuf.GeneratedMessage + implements EmptyRequestProtoOrBuilder { + // Use EmptyRequestProto.newBuilder() to construct. + private EmptyRequestProto(Builder builder) { + super(builder); + } + private EmptyRequestProto(boolean noInit) {} + + private static final EmptyRequestProto defaultInstance; + public static EmptyRequestProto getDefaultInstance() { + return defaultInstance; + } + + public EmptyRequestProto getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_EmptyRequestProto_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_EmptyRequestProto_fieldAccessorTable; + } + + private void initFields() { + } + 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(); + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + 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.ipc.protobuf.generated.TestProtos.EmptyRequestProto)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto other = (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto) obj; + + boolean result = true; + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto 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.ipc.protobuf.generated.TestProtos.EmptyRequestProto parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto 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.ipc.protobuf.generated.TestProtos.EmptyRequestProto parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto 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.ipc.protobuf.generated.TestProtos.EmptyRequestProto 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.ipc.protobuf.generated.TestProtos.EmptyRequestProto 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.ipc.protobuf.generated.TestProtos.EmptyRequestProto parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto 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.ipc.protobuf.generated.TestProtos.EmptyRequestProto 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.ipc.protobuf.generated.TestProtos.EmptyRequestProtoOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_EmptyRequestProto_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_EmptyRequestProto_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto.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(); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto.getDescriptor(); + } + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto getDefaultInstanceForType() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto build() { + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto buildPartial() { + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto result = new org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto(this); + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto) { + return mergeFrom((org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto other) { + if (other == org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto.getDefaultInstance()) return this; + 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; + } + } + } + } + + + // @@protoc_insertion_point(builder_scope:EmptyRequestProto) + } + + static { + defaultInstance = new EmptyRequestProto(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:EmptyRequestProto) + } + + public interface EmptyResponseProtoOrBuilder + extends com.google.protobuf.MessageOrBuilder { + } + public static final class EmptyResponseProto extends + com.google.protobuf.GeneratedMessage + implements EmptyResponseProtoOrBuilder { + // Use EmptyResponseProto.newBuilder() to construct. + private EmptyResponseProto(Builder builder) { + super(builder); + } + private EmptyResponseProto(boolean noInit) {} + + private static final EmptyResponseProto defaultInstance; + public static EmptyResponseProto getDefaultInstance() { + return defaultInstance; + } + + public EmptyResponseProto getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_EmptyResponseProto_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_EmptyResponseProto_fieldAccessorTable; + } + + private void initFields() { + } + 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(); + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + 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.ipc.protobuf.generated.TestProtos.EmptyResponseProto)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto other = (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto) obj; + + boolean result = true; + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto 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.ipc.protobuf.generated.TestProtos.EmptyResponseProto parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto 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.ipc.protobuf.generated.TestProtos.EmptyResponseProto parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto 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.ipc.protobuf.generated.TestProtos.EmptyResponseProto 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.ipc.protobuf.generated.TestProtos.EmptyResponseProto 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.ipc.protobuf.generated.TestProtos.EmptyResponseProto parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto 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.ipc.protobuf.generated.TestProtos.EmptyResponseProto 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.ipc.protobuf.generated.TestProtos.EmptyResponseProtoOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_EmptyResponseProto_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_EmptyResponseProto_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.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(); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDescriptor(); + } + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto getDefaultInstanceForType() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto build() { + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto buildPartial() { + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto result = new org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto(this); + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto) { + return mergeFrom((org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto other) { + if (other == org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance()) return this; + 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; + } + } + } + } + + + // @@protoc_insertion_point(builder_scope:EmptyResponseProto) + } + + static { + defaultInstance = new EmptyResponseProto(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:EmptyResponseProto) + } + + public interface EchoRequestProtoOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string message = 1; + boolean hasMessage(); + String getMessage(); + } + public static final class EchoRequestProto extends + com.google.protobuf.GeneratedMessage + implements EchoRequestProtoOrBuilder { + // Use EchoRequestProto.newBuilder() to construct. + private EchoRequestProto(Builder builder) { + super(builder); + } + private EchoRequestProto(boolean noInit) {} + + private static final EchoRequestProto defaultInstance; + public static EchoRequestProto getDefaultInstance() { + return defaultInstance; + } + + public EchoRequestProto getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_EchoRequestProto_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_EchoRequestProto_fieldAccessorTable; + } + + private int bitField0_; + // required string message = 1; + public static final int MESSAGE_FIELD_NUMBER = 1; + private java.lang.Object message_; + public boolean hasMessage() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getMessage() { + java.lang.Object ref = message_; + 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)) { + message_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getMessageBytes() { + java.lang.Object ref = message_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + message_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private void initFields() { + message_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasMessage()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getMessageBytes()); + } + 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, getMessageBytes()); + } + 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.ipc.protobuf.generated.TestProtos.EchoRequestProto)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto other = (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto) obj; + + boolean result = true; + result = result && (hasMessage() == other.hasMessage()); + if (hasMessage()) { + result = result && getMessage() + .equals(other.getMessage()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasMessage()) { + hash = (37 * hash) + MESSAGE_FIELD_NUMBER; + hash = (53 * hash) + getMessage().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto 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.ipc.protobuf.generated.TestProtos.EchoRequestProto parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto 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.ipc.protobuf.generated.TestProtos.EchoRequestProto parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto 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.ipc.protobuf.generated.TestProtos.EchoRequestProto 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.ipc.protobuf.generated.TestProtos.EchoRequestProto 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.ipc.protobuf.generated.TestProtos.EchoRequestProto parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto 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.ipc.protobuf.generated.TestProtos.EchoRequestProto 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.ipc.protobuf.generated.TestProtos.EchoRequestProtoOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_EchoRequestProto_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_EchoRequestProto_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto.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(); + message_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto.getDescriptor(); + } + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto getDefaultInstanceForType() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto build() { + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto buildPartial() { + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto result = new org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.message_ = message_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto) { + return mergeFrom((org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto other) { + if (other == org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto.getDefaultInstance()) return this; + if (other.hasMessage()) { + setMessage(other.getMessage()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasMessage()) { + + return false; + } + 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; + message_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required string message = 1; + private java.lang.Object message_ = ""; + public boolean hasMessage() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getMessage() { + java.lang.Object ref = message_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + message_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setMessage(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + message_ = value; + onChanged(); + return this; + } + public Builder clearMessage() { + bitField0_ = (bitField0_ & ~0x00000001); + message_ = getDefaultInstance().getMessage(); + onChanged(); + return this; + } + void setMessage(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + message_ = value; + onChanged(); + } + + // @@protoc_insertion_point(builder_scope:EchoRequestProto) + } + + static { + defaultInstance = new EchoRequestProto(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:EchoRequestProto) + } + + public interface EchoResponseProtoOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string message = 1; + boolean hasMessage(); + String getMessage(); + } + public static final class EchoResponseProto extends + com.google.protobuf.GeneratedMessage + implements EchoResponseProtoOrBuilder { + // Use EchoResponseProto.newBuilder() to construct. + private EchoResponseProto(Builder builder) { + super(builder); + } + private EchoResponseProto(boolean noInit) {} + + private static final EchoResponseProto defaultInstance; + public static EchoResponseProto getDefaultInstance() { + return defaultInstance; + } + + public EchoResponseProto getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_EchoResponseProto_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_EchoResponseProto_fieldAccessorTable; + } + + private int bitField0_; + // required string message = 1; + public static final int MESSAGE_FIELD_NUMBER = 1; + private java.lang.Object message_; + public boolean hasMessage() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getMessage() { + java.lang.Object ref = message_; + 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)) { + message_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getMessageBytes() { + java.lang.Object ref = message_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + message_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private void initFields() { + message_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasMessage()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getMessageBytes()); + } + 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, getMessageBytes()); + } + 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.ipc.protobuf.generated.TestProtos.EchoResponseProto)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto other = (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto) obj; + + boolean result = true; + result = result && (hasMessage() == other.hasMessage()); + if (hasMessage()) { + result = result && getMessage() + .equals(other.getMessage()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasMessage()) { + hash = (37 * hash) + MESSAGE_FIELD_NUMBER; + hash = (53 * hash) + getMessage().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto 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.ipc.protobuf.generated.TestProtos.EchoResponseProto parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto 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.ipc.protobuf.generated.TestProtos.EchoResponseProto parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto 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.ipc.protobuf.generated.TestProtos.EchoResponseProto 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.ipc.protobuf.generated.TestProtos.EchoResponseProto 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.ipc.protobuf.generated.TestProtos.EchoResponseProto parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto 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.ipc.protobuf.generated.TestProtos.EchoResponseProto 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.ipc.protobuf.generated.TestProtos.EchoResponseProtoOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_EchoResponseProto_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.internal_static_EchoResponseProto_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto.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(); + message_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto.getDescriptor(); + } + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto getDefaultInstanceForType() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto build() { + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto buildPartial() { + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto result = new org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.message_ = message_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto) { + return mergeFrom((org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto other) { + if (other == org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto.getDefaultInstance()) return this; + if (other.hasMessage()) { + setMessage(other.getMessage()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasMessage()) { + + return false; + } + 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; + message_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required string message = 1; + private java.lang.Object message_ = ""; + public boolean hasMessage() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getMessage() { + java.lang.Object ref = message_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + message_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setMessage(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + message_ = value; + onChanged(); + return this; + } + public Builder clearMessage() { + bitField0_ = (bitField0_ & ~0x00000001); + message_ = getDefaultInstance().getMessage(); + onChanged(); + return this; + } + void setMessage(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + message_ = value; + onChanged(); + } + + // @@protoc_insertion_point(builder_scope:EchoResponseProto) + } + + static { + defaultInstance = new EchoResponseProto(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:EchoResponseProto) + } + + private static com.google.protobuf.Descriptors.Descriptor + internal_static_EmptyRequestProto_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_EmptyRequestProto_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_EmptyResponseProto_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_EmptyResponseProto_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_EchoRequestProto_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_EchoRequestProto_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_EchoResponseProto_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_EchoResponseProto_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\ntest.proto\"\023\n\021EmptyRequestProto\"\024\n\022Emp" + + "tyResponseProto\"#\n\020EchoRequestProto\022\017\n\007m" + + "essage\030\001 \002(\t\"$\n\021EchoResponseProto\022\017\n\007mes" + + "sage\030\001 \002(\tB?\n.org.apache.hadoop.hbase.ip" + + "c.protobuf.generatedB\nTestProtos\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_EmptyRequestProto_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_EmptyRequestProto_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_EmptyRequestProto_descriptor, + new java.lang.String[] { }, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto.class, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto.Builder.class); + internal_static_EmptyResponseProto_descriptor = + getDescriptor().getMessageTypes().get(1); + internal_static_EmptyResponseProto_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_EmptyResponseProto_descriptor, + new java.lang.String[] { }, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.class, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.Builder.class); + internal_static_EchoRequestProto_descriptor = + getDescriptor().getMessageTypes().get(2); + internal_static_EchoRequestProto_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_EchoRequestProto_descriptor, + new java.lang.String[] { "Message", }, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto.class, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto.Builder.class); + internal_static_EchoResponseProto_descriptor = + getDescriptor().getMessageTypes().get(3); + internal_static_EchoResponseProto_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_EchoResponseProto_descriptor, + new java.lang.String[] { "Message", }, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto.class, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto.Builder.class); + return null; + } + }; + com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }, assigner); + } + + // @@protoc_insertion_point(outer_class_scope) +} Index: hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/protobuf/generated/TestRpcServiceProtos.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/protobuf/generated/TestRpcServiceProtos.java (revision 0) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/protobuf/generated/TestRpcServiceProtos.java (revision 0) @@ -0,0 +1,396 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: test_rpc_service.proto + +package org.apache.hadoop.hbase.ipc.protobuf.generated; + +public final class TestRpcServiceProtos { + private TestRpcServiceProtos() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + } + public static abstract class TestProtobufRpcProto + implements com.google.protobuf.Service { + protected TestProtobufRpcProto() {} + + public interface Interface { + public abstract void ping( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto request, + com.google.protobuf.RpcCallback done); + + public abstract void echo( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto request, + com.google.protobuf.RpcCallback done); + + public abstract void error( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto request, + com.google.protobuf.RpcCallback done); + + } + + public static com.google.protobuf.Service newReflectiveService( + final Interface impl) { + return new TestProtobufRpcProto() { + @java.lang.Override + public void ping( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto request, + com.google.protobuf.RpcCallback done) { + impl.ping(controller, request, done); + } + + @java.lang.Override + public void echo( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto request, + com.google.protobuf.RpcCallback done) { + impl.echo(controller, request, done); + } + + @java.lang.Override + public void error( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto request, + com.google.protobuf.RpcCallback done) { + impl.error(controller, request, done); + } + + }; + } + + public static com.google.protobuf.BlockingService + newReflectiveBlockingService(final BlockingInterface impl) { + return new com.google.protobuf.BlockingService() { + public final com.google.protobuf.Descriptors.ServiceDescriptor + getDescriptorForType() { + return getDescriptor(); + } + + public final com.google.protobuf.Message callBlockingMethod( + com.google.protobuf.Descriptors.MethodDescriptor method, + com.google.protobuf.RpcController controller, + com.google.protobuf.Message request) + throws com.google.protobuf.ServiceException { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.callBlockingMethod() given method descriptor for " + + "wrong service type."); + } + switch(method.getIndex()) { + case 0: + return impl.ping(controller, (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto)request); + case 1: + return impl.echo(controller, (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto)request); + case 2: + return impl.error(controller, (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto)request); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public final com.google.protobuf.Message + getRequestPrototype( + com.google.protobuf.Descriptors.MethodDescriptor method) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.getRequestPrototype() given method " + + "descriptor for wrong service type."); + } + switch(method.getIndex()) { + case 0: + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto.getDefaultInstance(); + case 1: + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto.getDefaultInstance(); + case 2: + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto.getDefaultInstance(); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public final com.google.protobuf.Message + getResponsePrototype( + com.google.protobuf.Descriptors.MethodDescriptor method) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.getResponsePrototype() given method " + + "descriptor for wrong service type."); + } + switch(method.getIndex()) { + case 0: + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance(); + case 1: + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto.getDefaultInstance(); + case 2: + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance(); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + }; + } + + public abstract void ping( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto request, + com.google.protobuf.RpcCallback done); + + public abstract void echo( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto request, + com.google.protobuf.RpcCallback done); + + public abstract void error( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto request, + com.google.protobuf.RpcCallback done); + + public static final + com.google.protobuf.Descriptors.ServiceDescriptor + getDescriptor() { + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos.getDescriptor().getServices().get(0); + } + public final com.google.protobuf.Descriptors.ServiceDescriptor + getDescriptorForType() { + return getDescriptor(); + } + + public final void callMethod( + com.google.protobuf.Descriptors.MethodDescriptor method, + com.google.protobuf.RpcController controller, + com.google.protobuf.Message request, + com.google.protobuf.RpcCallback< + com.google.protobuf.Message> done) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.callMethod() given method descriptor for wrong " + + "service type."); + } + switch(method.getIndex()) { + case 0: + this.ping(controller, (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto)request, + com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + case 1: + this.echo(controller, (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto)request, + com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + case 2: + this.error(controller, (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto)request, + com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public final com.google.protobuf.Message + getRequestPrototype( + com.google.protobuf.Descriptors.MethodDescriptor method) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.getRequestPrototype() given method " + + "descriptor for wrong service type."); + } + switch(method.getIndex()) { + case 0: + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto.getDefaultInstance(); + case 1: + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto.getDefaultInstance(); + case 2: + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto.getDefaultInstance(); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public final com.google.protobuf.Message + getResponsePrototype( + com.google.protobuf.Descriptors.MethodDescriptor method) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.getResponsePrototype() given method " + + "descriptor for wrong service type."); + } + switch(method.getIndex()) { + case 0: + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance(); + case 1: + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto.getDefaultInstance(); + case 2: + return org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance(); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public static Stub newStub( + com.google.protobuf.RpcChannel channel) { + return new Stub(channel); + } + + public static final class Stub extends org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto implements Interface { + private Stub(com.google.protobuf.RpcChannel channel) { + this.channel = channel; + } + + private final com.google.protobuf.RpcChannel channel; + + public com.google.protobuf.RpcChannel getChannel() { + return channel; + } + + public void ping( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(0), + controller, + request, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.class, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance())); + } + + public void echo( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(1), + controller, + request, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto.class, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto.getDefaultInstance())); + } + + public void error( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(2), + controller, + request, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.class, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance())); + } + } + + public static BlockingInterface newBlockingStub( + com.google.protobuf.BlockingRpcChannel channel) { + return new BlockingStub(channel); + } + + public interface BlockingInterface { + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto ping( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto request) + throws com.google.protobuf.ServiceException; + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto echo( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto request) + throws com.google.protobuf.ServiceException; + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto error( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto request) + throws com.google.protobuf.ServiceException; + } + + private static final class BlockingStub implements BlockingInterface { + private BlockingStub(com.google.protobuf.BlockingRpcChannel channel) { + this.channel = channel; + } + + private final com.google.protobuf.BlockingRpcChannel channel; + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto ping( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto) channel.callBlockingMethod( + getDescriptor().getMethods().get(0), + controller, + request, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance()); + } + + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto echo( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto) channel.callBlockingMethod( + getDescriptor().getMethods().get(1), + controller, + request, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto.getDefaultInstance()); + } + + + public org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto error( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto) channel.callBlockingMethod( + getDescriptor().getMethods().get(2), + controller, + request, + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance()); + } + + } + } + + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\026test_rpc_service.proto\032\ntest.proto2\250\001\n" + + "\024TestProtobufRpcProto\022/\n\004ping\022\022.EmptyReq" + + "uestProto\032\023.EmptyResponseProto\022-\n\004echo\022\021" + + ".EchoRequestProto\032\022.EchoResponseProto\0220\n" + + "\005error\022\022.EmptyRequestProto\032\023.EmptyRespon" + + "seProtoBL\n.org.apache.hadoop.hbase.ipc.p" + + "rotobuf.generatedB\024TestRpcServiceProtos\210" + + "\001\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; + return null; + } + }; + com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.getDescriptor(), + }, assigner); + } + + // @@protoc_insertion_point(outer_class_scope) +} Index: hbase-server/src/test/protobuf/test.proto =================================================================== --- hbase-server/src/test/protobuf/test.proto (revision 0) +++ hbase-server/src/test/protobuf/test.proto (revision 0) @@ -0,0 +1,35 @@ +/** + * 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. + */ + +option java_package = "org.apache.hadoop.hbase.ipc.protobuf.generated"; +option java_outer_classname = "TestProtos"; +option java_generate_equals_and_hash = true; + +message EmptyRequestProto { +} + +message EmptyResponseProto { +} + +message EchoRequestProto { + required string message = 1; +} + +message EchoResponseProto { + required string message = 1; +} Index: hbase-server/src/test/protobuf/test_rpc_service.proto =================================================================== --- hbase-server/src/test/protobuf/test_rpc_service.proto (revision 0) +++ hbase-server/src/test/protobuf/test_rpc_service.proto (revision 0) @@ -0,0 +1,33 @@ +/** + * 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. + */ +option java_package = "org.apache.hadoop.hbase.ipc.protobuf.generated"; +option java_outer_classname = "TestRpcServiceProtos"; +option java_generic_services = true; +option java_generate_equals_and_hash = true; + +import "test.proto"; + + +/** + * A protobuf service for use in tests + */ +service TestProtobufRpcProto { + rpc ping(EmptyRequestProto) returns (EmptyResponseProto); + rpc echo(EchoRequestProto) returns (EchoResponseProto); + rpc error(EmptyRequestProto) returns (EmptyResponseProto); +} \ No newline at end of file