diff --git a/src/main/java/org/apache/hadoop/hbase/io/DataOutputOutputStream.java b/src/main/java/org/apache/hadoop/hbase/io/DataOutputOutputStream.java index 82123a868ba..fe7044d040d 100644 --- a/src/main/java/org/apache/hadoop/hbase/io/DataOutputOutputStream.java +++ b/src/main/java/org/apache/hadoop/hbase/io/DataOutputOutputStream.java @@ -27,7 +27,7 @@ import org.apache.hadoop.classification.InterfaceAudience; * OutputStream implementation that wraps a DataOutput. */ @InterfaceAudience.Private -class DataOutputOutputStream extends OutputStream { +public class DataOutputOutputStream extends OutputStream { private final DataOutput out; diff --git a/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java b/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java index 2602461c58e..9c36edbfb9a 100644 --- a/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java +++ b/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java @@ -46,18 +46,23 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.protobuf.generated.RPCProtos; +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.security.User; -import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.PoolMap; import org.apache.hadoop.hbase.util.PoolMap.PoolType; import org.apache.hadoop.io.DataOutputBuffer; +import org.apache.hadoop.hbase.io.DataOutputOutputStream; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.Writable; -import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.net.NetUtils; 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 * a port and is defined by a parameter class and a value class. @@ -233,8 +238,9 @@ public class HBaseClient { User ticket = remoteId.getTicket(); Class protocol = remoteId.getProtocol(); - header = new ConnectionHeader( - protocol == null ? null : protocol.getName(), ticket); + ConnectionHeader.Builder builder = ConnectionHeader.newBuilder(); + builder.setProtocol(protocol == null ? "" : protocol.getName()); + this.header = builder.build(); this.setName("IPC Client (" + socketFactory.hashCode() +") connection to " + remoteId.getAddress().toString() + @@ -436,13 +442,8 @@ public class HBaseClient { private void writeHeader() throws IOException { out.write(HBaseServer.HEADER.array()); out.write(HBaseServer.CURRENT_VERSION); - //When there are more fields we can have ConnectionHeader Writable. - DataOutputBuffer buf = new DataOutputBuffer(); - header.write(buf); - - int bufLen = buf.getLength(); - out.writeInt(bufLen); - out.write(buf.getData(), 0, bufLen); + out.writeInt(header.getSerializedSize()); + header.writeTo(out); } /* wait till someone signals us to start reading RPC response or @@ -451,7 +452,6 @@ public class HBaseClient { * * Return true if it is time to read a response; false otherwise. */ - @SuppressWarnings({"ThrowableInstanceNeverThrown"}) protected synchronized boolean waitForWork() { if (calls.isEmpty() && !shouldCloseConnection.get() && running.get()) { long timeout = maxIdleTime- @@ -526,32 +526,24 @@ public class HBaseClient { if (shouldCloseConnection.get()) { return; } - - // For serializing the data to be written. - - final DataOutputBuffer d = new DataOutputBuffer(); try { if (LOG.isDebugEnabled()) LOG.debug(getName() + " sending #" + call.id); - - d.writeInt(0xdeadbeef); // placeholder for data length - d.writeInt(call.id); - call.param.write(d); - byte[] data = d.getData(); - int dataLength = d.getLength(); - // fill in the placeholder - Bytes.putInt(data, 0, dataLength - 4); + RpcRequest.Builder builder = RPCProtos.RpcRequest.newBuilder(); + builder.setCallId(call.id); + Invocation invocation = (Invocation)call.param; + DataOutputBuffer d = new DataOutputBuffer(); + invocation.write(d); + builder.setRequest(ByteString.copyFrom(d.getData())); //noinspection SynchronizeOnNonFinalField synchronized (this.out) { // FindBugs IS2_INCONSISTENT_SYNC - out.write(data, 0, dataLength); - out.flush(); + RpcRequest obj = builder.build(); + this.out.writeInt(obj.getSerializedSize()); + obj.writeTo(DataOutputOutputStream.constructOutputStream(this.out)); + this.out.flush(); } } catch(IOException e) { markClosed(e); - } finally { - //the buffer is just an in-memory buffer, but it is still polite to - // close early - IOUtils.closeStream(d); } } @@ -566,33 +558,31 @@ public class HBaseClient { try { // See HBaseServer.Call.setResponse for where we write out the response. - // It writes the call.id (int), a flag byte, then optionally the length - // of the response (int) followed by data. + // It writes the call.id (int), a boolean signifying any error (and if + // so the exception name/trace), and the response bytes // Read the call id. - int id = in.readInt(); + RpcResponse response = RpcResponse.parseDelimitedFrom(in); + int id = response.getCallId(); if (LOG.isDebugEnabled()) LOG.debug(getName() + " got value #" + id); Call call = calls.remove(id); - // Read the flag byte - byte flag = in.readByte(); - boolean isError = ResponseFlag.isError(flag); - if (ResponseFlag.isLength(flag)) { - // Currently length if present is unused. - in.readInt(); - } - int state = in.readInt(); // Read the state. Currently unused. + boolean isError = response.getError(); if (isError) { if (call != null) { //noinspection ThrowableInstanceNeverThrown - call.setException(new RemoteException(WritableUtils.readString(in), - WritableUtils.readString(in))); + call.setException(new RemoteException( + response.getException().getExceptionName(), + response.getException().getStackTrace())); } } else { + ByteString responseObj = response.getResponse(); + DataInputStream dis = + new DataInputStream(responseObj.newInput()); Writable value = ReflectionUtils.newInstance(valueClass, conf); - value.readFields(in); // read value + value.readFields(dis); // 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) { diff --git a/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java b/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java index 70c5666a741..973c7cbeec5 100644 --- a/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java +++ b/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java @@ -59,22 +59,28 @@ 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.HConstants; 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.monitoring.MonitoredRPCHandler; import org.apache.hadoop.hbase.monitoring.TaskMonitor; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.ByteBufferOutputStream; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.Writable; -import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.ipc.RPC.VersionMismatch; import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.StringUtils; import com.google.common.base.Function; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import com.google.protobuf.ByteString; import org.cliffc.high_scale_lib.Counter; @@ -94,7 +100,7 @@ public abstract class HBaseServer implements RpcServer { * The first four bytes of Hadoop RPC connections */ public static final ByteBuffer HEADER = ByteBuffer.wrap("hrpc".getBytes()); - public static final byte CURRENT_VERSION = 3; + public static final byte CURRENT_VERSION = 5; /** * How many calls/handler are allowed in the queue. @@ -333,40 +339,27 @@ public abstract class HBaseServer implements RpcServer { ByteBufferOutputStream buf = new ByteBufferOutputStream(size); DataOutputStream out = new DataOutputStream(buf); try { + RpcResponse.Builder builder = RpcResponse.newBuilder(); // Call id. - out.writeInt(this.id); - // Write flag. - byte flag = (error != null)? - ResponseFlag.getErrorAndLengthSet(): ResponseFlag.getLengthSetOnly(); - out.writeByte(flag); - // Place holder for length set later below after we - // fill the buffer with data. - out.writeInt(0xdeadbeef); - out.writeInt(status.state); - } catch (IOException e) { - errorClass = e.getClass().getName(); - error = StringUtils.stringifyException(e); - } - - try { - if (error == null) { - result.write(out); + builder.setCallId(this.id); + builder.setError(error != null); + if (error != null) { + RpcException.Builder b = RpcException.newBuilder(); + b.setExceptionName(errorClass); + b.setStackTrace(error); + builder.setException(b.build()); } else { - WritableUtils.writeString(out, errorClass); - WritableUtils.writeString(out, error); + DataOutputBuffer d = new DataOutputBuffer(size); + result.write(d); + byte[] response = d.getData(); + builder.setResponse(ByteString.copyFrom(response)); } + builder.build().writeDelimitedTo( + DataOutputOutputStream.constructOutputStream(out)); } catch (IOException e) { - LOG.warn("Error sending response to call: ", e); + LOG.warn("Exception while creating response " + e); } - - // Set the length into the ByteBuffer after call id and after - // byte flag. ByteBuffer bb = buf.getByteBuffer(); - int bufSiz = bb.remaining(); - // Move to the size location in our ByteBuffer past call.id - // and past the byte flag. - bb.position(Bytes.SIZEOF_INT + Bytes.SIZEOF_BYTE); - bb.putInt(bufSiz); bb.position(0); this.response = bb; } @@ -1065,9 +1058,9 @@ public abstract class HBaseServer implements RpcServer { // disconnected, we can say where it used to connect to. protected String hostAddress; protected int remotePort; - ConnectionHeader header = new ConnectionHeader(); + ConnectionHeader header; Class protocol; - protected User ticket = null; + protected User user = null; public Connection(SocketChannel channel, long lastContact) { this.channel = channel; @@ -1231,26 +1224,21 @@ public abstract class HBaseServer implements RpcServer { /// Reads the connection header following version private void processHeader() throws IOException { - DataInputStream in = - new DataInputStream(new ByteArrayInputStream(data.array())); - header.readFields(in); + header = ConnectionHeader.parseFrom(new ByteArrayInputStream(data.array())); try { String protocolClassName = header.getProtocol(); - if (protocolClassName == null) { - protocolClassName = "org.apache.hadoop.hbase.ipc.HRegionInterface"; - } protocol = getProtocolClass(protocolClassName, conf); } catch (ClassNotFoundException cnfe) { throw new IOException("Unknown protocol: " + header.getProtocol()); } - ticket = header.getUser(); + user = User.createUser(header); } protected void processData(byte[] buf) throws IOException, InterruptedException { - DataInputStream dis = - new DataInputStream(new ByteArrayInputStream(buf)); - int id = dis.readInt(); // try to read an id + RpcRequest request = RpcRequest.parseFrom(buf); + int id = request.getCallId(); + ByteString clientRequest = request.getRequest(); long callSize = buf.length; if (LOG.isDebugEnabled()) { @@ -1271,6 +1259,8 @@ public abstract class HBaseServer implements RpcServer { Writable param; try { + DataInputStream dis = + new DataInputStream(clientRequest.newInput()); param = ReflectionUtils.newInstance(paramClass, conf);//read param param.readFields(dis); } catch (Throwable t) { @@ -1372,12 +1362,12 @@ public abstract class HBaseServer implements RpcServer { throw new ServerNotRunningYetException("Server is not running yet"); if (LOG.isDebugEnabled()) { - User remoteUser = call.connection.ticket; + User remoteUser = call.connection.user; LOG.debug(getName() + ": call #" + call.id + " executing as " + (remoteUser == null ? "NULL principal" : remoteUser.getName())); } - RequestContext.set(call.connection.ticket, getRemoteIp(), + RequestContext.set(call.connection.user, getRemoteIp(), call.connection.protocol); // make the call value = call(call.connection.protocol, call.param, call.timestamp, diff --git a/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java b/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java new file mode 100644 index 00000000000..9c416fa0a1f --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java @@ -0,0 +1,2865 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: RPC.proto + +package org.apache.hadoop.hbase.protobuf.generated; + +public final class RPCProtos { + private RPCProtos() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + } + public interface UserInformationOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string effectiveUser = 1; + boolean hasEffectiveUser(); + String getEffectiveUser(); + + // required string realUser = 2; + boolean hasRealUser(); + String getRealUser(); + } + public static final class UserInformation extends + com.google.protobuf.GeneratedMessage + implements UserInformationOrBuilder { + // Use UserInformation.newBuilder() to construct. + private UserInformation(Builder builder) { + super(builder); + } + private UserInformation(boolean noInit) {} + + private static final UserInformation defaultInstance; + public static UserInformation getDefaultInstance() { + return defaultInstance; + } + + public UserInformation getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_UserInformation_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_UserInformation_fieldAccessorTable; + } + + private int bitField0_; + // required string effectiveUser = 1; + public static final int EFFECTIVEUSER_FIELD_NUMBER = 1; + private java.lang.Object effectiveUser_; + public boolean hasEffectiveUser() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getEffectiveUser() { + java.lang.Object ref = effectiveUser_; + 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)) { + effectiveUser_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getEffectiveUserBytes() { + java.lang.Object ref = effectiveUser_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + effectiveUser_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // required string realUser = 2; + public static final int REALUSER_FIELD_NUMBER = 2; + private java.lang.Object realUser_; + public boolean hasRealUser() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getRealUser() { + java.lang.Object ref = realUser_; + 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)) { + realUser_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getRealUserBytes() { + java.lang.Object ref = realUser_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + realUser_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private void initFields() { + effectiveUser_ = ""; + realUser_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasEffectiveUser()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasRealUser()) { + 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, getEffectiveUserBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getRealUserBytes()); + } + 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, getEffectiveUserBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(2, getRealUserBytes()); + } + 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.UserInformation)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation other = (org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation) obj; + + boolean result = true; + result = result && (hasEffectiveUser() == other.hasEffectiveUser()); + if (hasEffectiveUser()) { + result = result && getEffectiveUser() + .equals(other.getEffectiveUser()); + } + result = result && (hasRealUser() == other.hasRealUser()); + if (hasRealUser()) { + result = result && getRealUser() + .equals(other.getRealUser()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasEffectiveUser()) { + hash = (37 * hash) + EFFECTIVEUSER_FIELD_NUMBER; + hash = (53 * hash) + getEffectiveUser().hashCode(); + } + if (hasRealUser()) { + hash = (37 * hash) + REALUSER_FIELD_NUMBER; + hash = (53 * hash) + getRealUser().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation 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.UserInformation 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.UserInformation parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation 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.UserInformation parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation 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.UserInformation 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.UserInformation 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.UserInformation parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation 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.UserInformation 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.UserInformationOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_UserInformation_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_UserInformation_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.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(); + effectiveUser_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + realUser_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation build() { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation result = new org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.effectiveUser_ = effectiveUser_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.realUser_ = realUser_; + 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.UserInformation) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance()) return this; + if (other.hasEffectiveUser()) { + setEffectiveUser(other.getEffectiveUser()); + } + if (other.hasRealUser()) { + setRealUser(other.getRealUser()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasEffectiveUser()) { + + return false; + } + if (!hasRealUser()) { + + 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; + effectiveUser_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + realUser_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required string effectiveUser = 1; + private java.lang.Object effectiveUser_ = ""; + public boolean hasEffectiveUser() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getEffectiveUser() { + java.lang.Object ref = effectiveUser_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + effectiveUser_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setEffectiveUser(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + effectiveUser_ = value; + onChanged(); + return this; + } + public Builder clearEffectiveUser() { + bitField0_ = (bitField0_ & ~0x00000001); + effectiveUser_ = getDefaultInstance().getEffectiveUser(); + onChanged(); + return this; + } + void setEffectiveUser(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + effectiveUser_ = value; + onChanged(); + } + + // required string realUser = 2; + private java.lang.Object realUser_ = ""; + public boolean hasRealUser() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getRealUser() { + java.lang.Object ref = realUser_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + realUser_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setRealUser(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + realUser_ = value; + onChanged(); + return this; + } + public Builder clearRealUser() { + bitField0_ = (bitField0_ & ~0x00000002); + realUser_ = getDefaultInstance().getRealUser(); + onChanged(); + return this; + } + void setRealUser(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000002; + realUser_ = value; + onChanged(); + } + + // @@protoc_insertion_point(builder_scope:UserInformation) + } + + static { + defaultInstance = new UserInformation(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:UserInformation) + } + + public interface ConnectionHeaderOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // optional .UserInformation userInfo = 1; + boolean hasUserInfo(); + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo(); + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder(); + + // optional string protocol = 2 [default = "org.apache.hadoop.hbase.ipc.HRegionInterface"]; + boolean hasProtocol(); + String getProtocol(); + } + public static final class ConnectionHeader extends + com.google.protobuf.GeneratedMessage + implements ConnectionHeaderOrBuilder { + // Use ConnectionHeader.newBuilder() to construct. + private ConnectionHeader(Builder builder) { + super(builder); + } + private ConnectionHeader(boolean noInit) {} + + private static final ConnectionHeader defaultInstance; + public static ConnectionHeader getDefaultInstance() { + return defaultInstance; + } + + public ConnectionHeader getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_ConnectionHeader_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_ConnectionHeader_fieldAccessorTable; + } + + private int bitField0_; + // optional .UserInformation userInfo = 1; + public static final int USERINFO_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation userInfo_; + public boolean hasUserInfo() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo() { + return userInfo_; + } + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder() { + return userInfo_; + } + + // optional string protocol = 2 [default = "org.apache.hadoop.hbase.ipc.HRegionInterface"]; + public static final int PROTOCOL_FIELD_NUMBER = 2; + private java.lang.Object protocol_; + public boolean hasProtocol() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getProtocol() { + java.lang.Object ref = protocol_; + 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)) { + protocol_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getProtocolBytes() { + java.lang.Object ref = protocol_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + protocol_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private void initFields() { + userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance(); + protocol_ = "org.apache.hadoop.hbase.ipc.HRegionInterface"; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (hasUserInfo()) { + if (!getUserInfo().isInitialized()) { + 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.writeMessage(1, userInfo_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getProtocolBytes()); + } + 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 + .computeMessageSize(1, userInfo_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(2, getProtocolBytes()); + } + 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.ConnectionHeader)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader other = (org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader) obj; + + boolean result = true; + result = result && (hasUserInfo() == other.hasUserInfo()); + if (hasUserInfo()) { + result = result && getUserInfo() + .equals(other.getUserInfo()); + } + result = result && (hasProtocol() == other.hasProtocol()); + if (hasProtocol()) { + result = result && getProtocol() + .equals(other.getProtocol()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasUserInfo()) { + hash = (37 * hash) + USERINFO_FIELD_NUMBER; + hash = (53 * hash) + getUserInfo().hashCode(); + } + if (hasProtocol()) { + hash = (37 * hash) + PROTOCOL_FIELD_NUMBER; + hash = (53 * hash) + getProtocol().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader 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.ConnectionHeader 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.ConnectionHeader parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader 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.ConnectionHeader parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader 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.ConnectionHeader 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.ConnectionHeader 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.ConnectionHeader parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader 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.ConnectionHeader 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.ConnectionHeaderOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_ConnectionHeader_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_ConnectionHeader_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getUserInfoFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (userInfoBuilder_ == null) { + userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance(); + } else { + userInfoBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + protocol_ = "org.apache.hadoop.hbase.ipc.HRegionInterface"; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader build() { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader result = new org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (userInfoBuilder_ == null) { + result.userInfo_ = userInfo_; + } else { + result.userInfo_ = userInfoBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.protocol_ = protocol_; + 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.ConnectionHeader) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader.getDefaultInstance()) return this; + if (other.hasUserInfo()) { + mergeUserInfo(other.getUserInfo()); + } + if (other.hasProtocol()) { + setProtocol(other.getProtocol()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (hasUserInfo()) { + if (!getUserInfo().isInitialized()) { + + 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: { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.newBuilder(); + if (hasUserInfo()) { + subBuilder.mergeFrom(getUserInfo()); + } + input.readMessage(subBuilder, extensionRegistry); + setUserInfo(subBuilder.buildPartial()); + break; + } + case 18: { + bitField0_ |= 0x00000002; + protocol_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // optional .UserInformation userInfo = 1; + private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_; + public boolean hasUserInfo() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo() { + if (userInfoBuilder_ == null) { + return userInfo_; + } else { + return userInfoBuilder_.getMessage(); + } + } + public Builder setUserInfo(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation value) { + if (userInfoBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + userInfo_ = value; + onChanged(); + } else { + userInfoBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder setUserInfo( + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder builderForValue) { + if (userInfoBuilder_ == null) { + userInfo_ = builderForValue.build(); + onChanged(); + } else { + userInfoBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder mergeUserInfo(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation value) { + if (userInfoBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + userInfo_ != org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance()) { + userInfo_ = + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.newBuilder(userInfo_).mergeFrom(value).buildPartial(); + } else { + userInfo_ = value; + } + onChanged(); + } else { + userInfoBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder clearUserInfo() { + if (userInfoBuilder_ == null) { + userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance(); + onChanged(); + } else { + userInfoBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder getUserInfoBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getUserInfoFieldBuilder().getBuilder(); + } + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder() { + if (userInfoBuilder_ != null) { + return userInfoBuilder_.getMessageOrBuilder(); + } else { + return userInfo_; + } + } + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder> + getUserInfoFieldBuilder() { + if (userInfoBuilder_ == null) { + userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder>( + userInfo_, + getParentForChildren(), + isClean()); + userInfo_ = null; + } + return userInfoBuilder_; + } + + // optional string protocol = 2 [default = "org.apache.hadoop.hbase.ipc.HRegionInterface"]; + private java.lang.Object protocol_ = "org.apache.hadoop.hbase.ipc.HRegionInterface"; + public boolean hasProtocol() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getProtocol() { + java.lang.Object ref = protocol_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + protocol_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setProtocol(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + protocol_ = value; + onChanged(); + return this; + } + public Builder clearProtocol() { + bitField0_ = (bitField0_ & ~0x00000002); + protocol_ = getDefaultInstance().getProtocol(); + onChanged(); + return this; + } + void setProtocol(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000002; + protocol_ = value; + onChanged(); + } + + // @@protoc_insertion_point(builder_scope:ConnectionHeader) + } + + static { + defaultInstance = new ConnectionHeader(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:ConnectionHeader) + } + + public interface RpcRequestOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required int32 callId = 1; + boolean hasCallId(); + int getCallId(); + + // optional bytes request = 2; + boolean hasRequest(); + com.google.protobuf.ByteString getRequest(); + } + public static final class RpcRequest extends + com.google.protobuf.GeneratedMessage + implements RpcRequestOrBuilder { + // Use RpcRequest.newBuilder() to construct. + private RpcRequest(Builder builder) { + super(builder); + } + private RpcRequest(boolean noInit) {} + + private static final RpcRequest defaultInstance; + public static RpcRequest getDefaultInstance() { + return defaultInstance; + } + + public RpcRequest getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcRequest_fieldAccessorTable; + } + + private int bitField0_; + // required int32 callId = 1; + public static final int CALLID_FIELD_NUMBER = 1; + private int callId_; + public boolean hasCallId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public int getCallId() { + 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() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasCallId()) { + 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.writeInt32(1, callId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, request_); + } + 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 + .computeInt32Size(1, callId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(2, request_); + } + 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.RpcRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest other = (org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest) obj; + + boolean result = true; + result = result && (hasCallId() == other.hasCallId()); + if (hasCallId()) { + 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; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasCallId()) { + 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( + 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( + 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) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest 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) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest 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) + 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.RpcRequest 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.RpcRequest 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( + 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.RpcRequest 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.RpcRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcRequest_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest.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(); + callId_ = 0; + bitField0_ = (bitField0_ & ~0x00000001); + request_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest.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.RpcRequest build() { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + 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); + 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); + } 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; + if (other.hasCallId()) { + setCallId(other.getCallId()); + } + if (other.hasRequest()) { + setRequest(other.getRequest()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasCallId()) { + + 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 8: { + bitField0_ |= 0x00000001; + callId_ = input.readInt32(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + request_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required int32 callId = 1; + private int callId_ ; + public boolean hasCallId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public int getCallId() { + return callId_; + } + public Builder setCallId(int value) { + bitField0_ |= 0x00000001; + callId_ = value; + onChanged(); + return this; + } + public Builder clearCallId() { + bitField0_ = (bitField0_ & ~0x00000001); + callId_ = 0; + onChanged(); + 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) + } + + static { + defaultInstance = new RpcRequest(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:RpcRequest) + } + + 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; + } + } + 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; + } + } + 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) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + exceptionName_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + stackTrace_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + 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 { + 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()); + } + if (other.hasStackTrace()) { + setStackTrace(other.getStackTrace()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + 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; + } + } + } + } + + 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 { + return (String) ref; + } + } + public Builder setExceptionName(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + exceptionName_ = value; + onChanged(); + return this; + } + public Builder clearExceptionName() { + bitField0_ = (bitField0_ & ~0x00000001); + exceptionName_ = getDefaultInstance().getExceptionName(); + onChanged(); + return this; + } + void setExceptionName(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + exceptionName_ = value; + onChanged(); + } + + // optional string stackTrace = 2; + private java.lang.Object stackTrace_ = ""; + public boolean hasStackTrace() { + 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 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:RpcException) + } + + static { + defaultInstance = new RpcException(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:RpcException) + } + + public interface RpcResponseOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required int32 callId = 1; + boolean hasCallId(); + int getCallId(); + + // required bool error = 2; + boolean hasError(); + boolean getError(); + + // 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(); + } + public static final class RpcResponse extends + com.google.protobuf.GeneratedMessage + implements RpcResponseOrBuilder { + // Use RpcResponse.newBuilder() to construct. + private RpcResponse(Builder builder) { + super(builder); + } + private RpcResponse(boolean noInit) {} + + private static final RpcResponse defaultInstance; + public static RpcResponse getDefaultInstance() { + return defaultInstance; + } + + public RpcResponse getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcResponse_fieldAccessorTable; + } + + private int bitField0_; + // required int32 callId = 1; + public static final int CALLID_FIELD_NUMBER = 1; + private int callId_; + public boolean hasCallId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public int getCallId() { + return callId_; + } + + // required bool error = 2; + public static final int ERROR_FIELD_NUMBER = 2; + private boolean error_; + public boolean hasError() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public boolean getError() { + return error_; + } + + // 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; + error_ = false; + response_ = com.google.protobuf.ByteString.EMPTY; + exception_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasCallId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasError()) { + memoizedIsInitialized = 0; + return false; + } + if (hasException()) { + if (!getException().isInitialized()) { + 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.writeInt32(1, callId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBool(2, error_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, response_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeMessage(4, exception_); + } + 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 + .computeInt32Size(1, callId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeBoolSize(2, error_); + } + 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; + } + + 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.RpcResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse other = (org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse) obj; + + boolean result = true; + result = result && (hasCallId() == other.hasCallId()); + if (hasCallId()) { + result = result && (getCallId() + == other.getCallId()); + } + result = result && (hasError() == other.hasError()); + if (hasError()) { + result = result && (getError() + == other.getError()); + } + 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; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasCallId()) { + hash = (37 * hash) + CALLID_FIELD_NUMBER; + hash = (53 * hash) + getCallId(); + } + if (hasError()) { + hash = (37 * hash) + ERROR_FIELD_NUMBER; + hash = (53 * hash) + hashBoolean(getError()); + } + 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( + 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( + 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) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse 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) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse 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) + 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.RpcResponse 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.RpcResponse 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( + 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.RpcResponse 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.RpcResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.internal_static_RpcResponse_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getExceptionFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + callId_ = 0; + bitField0_ = (bitField0_ & ~0x00000001); + error_ = false; + 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; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.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.RpcResponse build() { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + 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); + 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.error_ = error_; + 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); + } 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; + if (other.hasCallId()) { + setCallId(other.getCallId()); + } + if (other.hasError()) { + setError(other.getError()); + } + if (other.hasResponse()) { + setResponse(other.getResponse()); + } + if (other.hasException()) { + mergeException(other.getException()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasCallId()) { + + return false; + } + if (!hasError()) { + + return false; + } + if (hasException()) { + if (!getException().isInitialized()) { + + 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 8: { + bitField0_ |= 0x00000001; + callId_ = input.readInt32(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + error_ = input.readBool(); + 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; + } + } + } + } + + private int bitField0_; + + // required int32 callId = 1; + private int callId_ ; + public boolean hasCallId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public int getCallId() { + return callId_; + } + public Builder setCallId(int value) { + bitField0_ |= 0x00000001; + callId_ = value; + onChanged(); + return this; + } + public Builder clearCallId() { + bitField0_ = (bitField0_ & ~0x00000001); + callId_ = 0; + onChanged(); + return this; + } + + // required bool error = 2; + private boolean error_ ; + public boolean hasError() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public boolean getError() { + return error_; + } + public Builder setError(boolean value) { + bitField0_ |= 0x00000002; + error_ = value; + onChanged(); + return this; + } + public Builder clearError() { + bitField0_ = (bitField0_ & ~0x00000002); + error_ = false; + onChanged(); + return this; + } + + // optional bytes response = 3; + private com.google.protobuf.ByteString response_ = com.google.protobuf.ByteString.EMPTY; + public boolean hasResponse() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public com.google.protobuf.ByteString getResponse() { + return response_; + } + public Builder setResponse(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + response_ = value; + onChanged(); + return this; + } + public Builder clearResponse() { + bitField0_ = (bitField0_ & ~0x00000004); + 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); + } + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException getException() { + if (exceptionBuilder_ == null) { + return exception_; + } else { + return exceptionBuilder_.getMessage(); + } + } + 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); + } + bitField0_ |= 0x00000008; + return this; + } + public Builder setException( + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.Builder builderForValue) { + if (exceptionBuilder_ == null) { + exception_ = builderForValue.build(); + onChanged(); + } else { + exceptionBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000008; + 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; + } + 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(); + } else { + exceptionBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException.Builder getExceptionBuilder() { + bitField0_ |= 0x00000008; + onChanged(); + return getExceptionFieldBuilder().getBuilder(); + } + public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcExceptionOrBuilder getExceptionOrBuilder() { + if (exceptionBuilder_ != null) { + return exceptionBuilder_.getMessageOrBuilder(); + } else { + return exception_; + } + } + 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_; + } + + // @@protoc_insertion_point(builder_scope:RpcResponse) + } + + static { + defaultInstance = new RpcResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:RpcResponse) + } + + private static com.google.protobuf.Descriptors.Descriptor + internal_static_UserInformation_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_UserInformation_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_ConnectionHeader_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_ConnectionHeader_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_RpcRequest_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_RpcRequest_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_RpcException_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_RpcException_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_RpcResponse_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_RpcResponse_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\tRPC.proto\":\n\017UserInformation\022\025\n\reffect" + + "iveUser\030\001 \002(\t\022\020\n\010realUser\030\002 \002(\t\"v\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.ipc.HRegionInterface\"-\n\nRpcRe" + + "quest\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\"`\n\013RpcResponse\022\016\n\006call" + + "Id\030\001 \002(\005\022\r\n\005error\030\002 \002(\010\022\020\n\010response\030\003 \001(" + + "\014\022 \n\texception\030\004 \001(\0132\r.RpcExceptionB<\n*o", + "rg.apache.hadoop.hbase.protobuf.generate" + + "dB\tRPCProtosH\001\240\001\001" + }; + com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = + new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { + public com.google.protobuf.ExtensionRegistry assignDescriptors( + com.google.protobuf.Descriptors.FileDescriptor root) { + descriptor = root; + internal_static_UserInformation_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_UserInformation_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_UserInformation_descriptor, + new java.lang.String[] { "EffectiveUser", "RealUser", }, + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.class, + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder.class); + internal_static_ConnectionHeader_descriptor = + getDescriptor().getMessageTypes().get(1); + internal_static_ConnectionHeader_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_ConnectionHeader_descriptor, + 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 = + getDescriptor().getMessageTypes().get(2); + internal_static_RpcRequest_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_RpcException_descriptor = + getDescriptor().getMessageTypes().get(3); + 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", "Error", "Response", "Exception", }, + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.class, + org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponse.Builder.class); + return null; + } + }; + com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }, assigner); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/src/main/java/org/apache/hadoop/hbase/security/User.java b/src/main/java/org/apache/hadoop/hbase/security/User.java index 2e2a61e0052..f9037bab526 100644 --- a/src/main/java/org/apache/hadoop/hbase/security/User.java +++ b/src/main/java/org/apache/hadoop/hbase/security/User.java @@ -24,6 +24,8 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader; +import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation; import org.apache.hadoop.hbase.util.Methods; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapreduce.Job; @@ -164,6 +166,33 @@ public abstract class User { return new HadoopUser(ugi); } + public static User createUser(ConnectionHeader head) { + UserGroupInformation ugi = null; + + if (!head.hasUserInfo()) { + return create(null); + } + UserInformation userInfoProto = head.getUserInfo(); + String effectiveUser = null; + if (userInfoProto.hasEffectiveUser()) { + effectiveUser = userInfoProto.getEffectiveUser(); + } + String realUser = null; + if (userInfoProto.hasRealUser()) { + realUser = userInfoProto.getRealUser(); + } + if (effectiveUser != null) { + if (realUser != null) { + UserGroupInformation realUserUgi = + UserGroupInformation.createRemoteUser(realUser); + ugi = UserGroupInformation.createProxyUser(effectiveUser, realUserUgi); + } else { + ugi = UserGroupInformation.createRemoteUser(effectiveUser); + } + } + return create(ugi); + } + /** * Generates a new {@code User} instance specifically for use in test code. * @param name the full username diff --git a/src/main/protobuf/RPC.proto b/src/main/protobuf/RPC.proto new file mode 100644 index 00000000000..105fb3fc932 --- /dev/null +++ b/src/main/protobuf/RPC.proto @@ -0,0 +1,102 @@ +/** + * 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. + */ + +/** + * Specification of (unsecure) HBase RPC: + * + * Client needs to set up a connection first to a server serving a certain + * HBase protocol (like HRegionInterface). Once the connection is set up, the + * client and server communicates on that channel for RPC requests/responses. + * The sections below describe the flow. + * + * As part of setting up a connection to a server, the client needs to send + * the ConnectionHeader header. At the data level, this looks like + * <"hrpc"-bytearray><5-byte> + * + * For every RPC that the client makes it needs to send the + * RpcRequest. At the data level this looks like + * + * + * The server sends back a RpcResponse object as response. + * At the data level this looks like + * + * + * There is one special message that's sent from client to server - + * the Ping message. At the data level, this is just the bytes corresponding + * to integer -1. + */ +option java_package = "org.apache.hadoop.hbase.protobuf.generated"; +option java_outer_classname = "RPCProtos"; +option java_generate_equals_and_hash = true; +option optimize_for = SPEED; + +message UserInformation { + required string effectiveUser = 1; + required string realUser = 2; +} + +message ConnectionHeader { + /** User Info beyond what is established at connection establishment + * (applies to secure HBase setup) + */ + optional UserInformation userInfo = 1; + /** Protocol name for next rpc layer + * the client created a proxy with this protocol name + */ + optional string protocol = 2 [default = "org.apache.hadoop.hbase.ipc.HRegionInterface"]; +} + + +/** + * The complete RPC request message + */ +message RpcRequest { + /** Monotonically increasing callId, mostly to keep track of RPCs */ + required int32 callId = 1; + /** The request bytes */ + optional bytes request = 2; +} + +/** + * 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; +} + +/** + * The complete RPC response message + */ +message RpcResponse { + /** Echo back the callId the client sent */ + required int32 callId = 1; + /** Did the RPC execution encounter an error at the server */ + required bool error = 2; + /** Optional response bytes */ + optional bytes response = 3; + /** Optional exception when error is true*/ + optional RpcException exception = 4; +}