HBASE-6524 Hooks for hbase tracing; REAPPLICATION
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1378807 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
e88e87d7fe
commit
a2788de625
|
@ -451,6 +451,11 @@
|
||||||
<artifactId>jettison</artifactId>
|
<artifactId>jettison</artifactId>
|
||||||
<scope>test</scope>
|
<scope>test</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.cloudera.htrace</groupId>
|
||||||
|
<artifactId>htrace</artifactId>
|
||||||
|
<version>1.49</version>
|
||||||
|
</dependency>
|
||||||
</dependencies>
|
</dependencies>
|
||||||
<profiles>
|
<profiles>
|
||||||
<!-- Skip the tests in this module -->
|
<!-- Skip the tests in this module -->
|
||||||
|
|
|
@ -27,6 +27,9 @@ import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.Server;
|
import org.apache.hadoop.hbase.Server;
|
||||||
import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
|
import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
|
||||||
|
import org.cloudera.htrace.Sampler;
|
||||||
|
import org.cloudera.htrace.Span;
|
||||||
|
import org.cloudera.htrace.Trace;
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -76,6 +79,8 @@ public abstract class EventHandler implements Runnable, Comparable<Runnable> {
|
||||||
// Time to wait for events to happen, should be kept short
|
// Time to wait for events to happen, should be kept short
|
||||||
protected final int waitingTimeForEvents;
|
protected final int waitingTimeForEvents;
|
||||||
|
|
||||||
|
private final Span parent;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This interface provides pre- and post-process hooks for events.
|
* This interface provides pre- and post-process hooks for events.
|
||||||
*/
|
*/
|
||||||
|
@ -182,6 +187,7 @@ public abstract class EventHandler implements Runnable, Comparable<Runnable> {
|
||||||
* Default base class constructor.
|
* Default base class constructor.
|
||||||
*/
|
*/
|
||||||
public EventHandler(Server server, EventType eventType) {
|
public EventHandler(Server server, EventType eventType) {
|
||||||
|
this.parent = Trace.currentTrace();
|
||||||
this.server = server;
|
this.server = server;
|
||||||
this.eventType = eventType;
|
this.eventType = eventType;
|
||||||
seqid = seqids.incrementAndGet();
|
seqid = seqids.incrementAndGet();
|
||||||
|
@ -190,12 +196,16 @@ public abstract class EventHandler implements Runnable, Comparable<Runnable> {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void run() {
|
public void run() {
|
||||||
|
Span chunk = Trace.startSpan(Thread.currentThread().getName(), parent,
|
||||||
|
Sampler.ALWAYS);
|
||||||
try {
|
try {
|
||||||
if (getListener() != null) getListener().beforeProcess(this);
|
if (getListener() != null) getListener().beforeProcess(this);
|
||||||
process();
|
process();
|
||||||
if (getListener() != null) getListener().afterProcess(this);
|
if (getListener() != null) getListener().afterProcess(this);
|
||||||
} catch(Throwable t) {
|
} catch(Throwable t) {
|
||||||
LOG.error("Caught throwable while processing event " + eventType, t);
|
LOG.error("Caught throwable while processing event " + eventType, t);
|
||||||
|
} finally {
|
||||||
|
chunk.stop();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -63,6 +63,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.St
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader;
|
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader;
|
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
|
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo;
|
||||||
import org.apache.hadoop.hbase.security.HBaseSaslRpcClient;
|
import org.apache.hadoop.hbase.security.HBaseSaslRpcClient;
|
||||||
import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.AuthMethod;
|
import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.AuthMethod;
|
||||||
import org.apache.hadoop.hbase.security.KerberosInfo;
|
import org.apache.hadoop.hbase.security.KerberosInfo;
|
||||||
|
@ -83,6 +84,9 @@ import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.apache.hadoop.security.token.Token;
|
import org.apache.hadoop.security.token.Token;
|
||||||
import org.apache.hadoop.security.token.TokenIdentifier;
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||||
import org.apache.hadoop.security.token.TokenSelector;
|
import org.apache.hadoop.security.token.TokenSelector;
|
||||||
|
import org.apache.hadoop.util.ReflectionUtils;
|
||||||
|
import org.cloudera.htrace.Span;
|
||||||
|
import org.cloudera.htrace.Trace;
|
||||||
|
|
||||||
import com.google.protobuf.CodedOutputStream;
|
import com.google.protobuf.CodedOutputStream;
|
||||||
import com.google.protobuf.Message;
|
import com.google.protobuf.Message;
|
||||||
|
@ -928,8 +932,17 @@ public class HBaseClient {
|
||||||
try {
|
try {
|
||||||
if (LOG.isDebugEnabled())
|
if (LOG.isDebugEnabled())
|
||||||
LOG.debug(getName() + " sending #" + call.id);
|
LOG.debug(getName() + " sending #" + call.id);
|
||||||
|
|
||||||
RpcRequestHeader.Builder headerBuilder = RPCProtos.RpcRequestHeader.newBuilder();
|
RpcRequestHeader.Builder headerBuilder = RPCProtos.RpcRequestHeader.newBuilder();
|
||||||
headerBuilder.setCallId(call.id);
|
headerBuilder.setCallId(call.id);
|
||||||
|
|
||||||
|
if (Trace.isTracing()) {
|
||||||
|
Span s = Trace.currentTrace();
|
||||||
|
headerBuilder.setTinfo(RPCTInfo.newBuilder()
|
||||||
|
.setParentId(s.getSpanId())
|
||||||
|
.setTraceId(s.getTraceId()));
|
||||||
|
}
|
||||||
|
|
||||||
//noinspection SynchronizeOnNonFinalField
|
//noinspection SynchronizeOnNonFinalField
|
||||||
synchronized (this.out) { // FindBugs IS2_INCONSISTENT_SYNC
|
synchronized (this.out) { // FindBugs IS2_INCONSISTENT_SYNC
|
||||||
RpcRequestHeader header = headerBuilder.build();
|
RpcRequestHeader header = headerBuilder.build();
|
||||||
|
|
|
@ -107,6 +107,11 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||||
import com.google.protobuf.Message;
|
import com.google.protobuf.Message;
|
||||||
|
|
||||||
import org.cliffc.high_scale_lib.Counter;
|
import org.cliffc.high_scale_lib.Counter;
|
||||||
|
import org.cloudera.htrace.Sampler;
|
||||||
|
import org.cloudera.htrace.Span;
|
||||||
|
import org.cloudera.htrace.TraceInfo;
|
||||||
|
import org.cloudera.htrace.impl.NullSpan;
|
||||||
|
import org.cloudera.htrace.Trace;
|
||||||
|
|
||||||
/** A client for an IPC service. IPC calls take a single Protobuf message as a
|
/** A client for an IPC service. IPC calls take a single Protobuf message as a
|
||||||
* parameter, and return a single Protobuf message as their value. A service runs on
|
* parameter, and return a single Protobuf message as their value. A service runs on
|
||||||
|
@ -317,9 +322,10 @@ public abstract class HBaseServer implements RpcServer {
|
||||||
// set at call completion
|
// set at call completion
|
||||||
protected long size; // size of current call
|
protected long size; // size of current call
|
||||||
protected boolean isError;
|
protected boolean isError;
|
||||||
|
protected TraceInfo tinfo;
|
||||||
|
|
||||||
public Call(int id, RpcRequestBody rpcRequestBody, Connection connection,
|
public Call(int id, RpcRequestBody rpcRequestBody, Connection connection,
|
||||||
Responder responder, long size) {
|
Responder responder, long size, TraceInfo tinfo) {
|
||||||
this.id = id;
|
this.id = id;
|
||||||
this.rpcRequestBody = rpcRequestBody;
|
this.rpcRequestBody = rpcRequestBody;
|
||||||
this.connection = connection;
|
this.connection = connection;
|
||||||
|
@ -329,6 +335,7 @@ public abstract class HBaseServer implements RpcServer {
|
||||||
this.responder = responder;
|
this.responder = responder;
|
||||||
this.isError = false;
|
this.isError = false;
|
||||||
this.size = size;
|
this.size = size;
|
||||||
|
this.tinfo = tinfo;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -1123,13 +1130,14 @@ public abstract class HBaseServer implements RpcServer {
|
||||||
private boolean useWrap = false;
|
private boolean useWrap = false;
|
||||||
// Fake 'call' for failed authorization response
|
// Fake 'call' for failed authorization response
|
||||||
private final int AUTHROIZATION_FAILED_CALLID = -1;
|
private final int AUTHROIZATION_FAILED_CALLID = -1;
|
||||||
private final Call authFailedCall =
|
private final Call authFailedCall = new Call(AUTHROIZATION_FAILED_CALLID,
|
||||||
new Call(AUTHROIZATION_FAILED_CALLID, null, this, null, 0);
|
null, this, null, 0, null);
|
||||||
private ByteArrayOutputStream authFailedResponse =
|
private ByteArrayOutputStream authFailedResponse =
|
||||||
new ByteArrayOutputStream();
|
new ByteArrayOutputStream();
|
||||||
// Fake 'call' for SASL context setup
|
// Fake 'call' for SASL context setup
|
||||||
private static final int SASL_CALLID = -33;
|
private static final int SASL_CALLID = -33;
|
||||||
private final Call saslCall = new Call(SASL_CALLID, null, this, null, 0);
|
private final Call saslCall = new Call(SASL_CALLID, null, this, null, 0,
|
||||||
|
null);
|
||||||
|
|
||||||
public UserGroupInformation attemptingUser = null; // user name before auth
|
public UserGroupInformation attemptingUser = null; // user name before auth
|
||||||
public Connection(SocketChannel channel, long lastContact) {
|
public Connection(SocketChannel channel, long lastContact) {
|
||||||
|
@ -1477,7 +1485,7 @@ public abstract class HBaseServer implements RpcServer {
|
||||||
// we return 0 which will keep the socket up -- bad clients, unless
|
// we return 0 which will keep the socket up -- bad clients, unless
|
||||||
// they switch to suit the running server -- will fail later doing
|
// they switch to suit the running server -- will fail later doing
|
||||||
// getProtocolVersion.
|
// getProtocolVersion.
|
||||||
Call fakeCall = new Call(0, null, this, responder, 0);
|
Call fakeCall = new Call(0, null, this, responder, 0, null);
|
||||||
// Versions 3 and greater can interpret this exception
|
// Versions 3 and greater can interpret this exception
|
||||||
// response in the same manner
|
// response in the same manner
|
||||||
setupResponse(buffer, fakeCall, Status.FATAL,
|
setupResponse(buffer, fakeCall, Status.FATAL,
|
||||||
|
@ -1592,6 +1600,7 @@ public abstract class HBaseServer implements RpcServer {
|
||||||
DataInputStream dis =
|
DataInputStream dis =
|
||||||
new DataInputStream(new ByteArrayInputStream(buf));
|
new DataInputStream(new ByteArrayInputStream(buf));
|
||||||
RpcRequestHeader request = RpcRequestHeader.parseDelimitedFrom(dis);
|
RpcRequestHeader request = RpcRequestHeader.parseDelimitedFrom(dis);
|
||||||
|
|
||||||
int id = request.getCallId();
|
int id = request.getCallId();
|
||||||
long callSize = buf.length;
|
long callSize = buf.length;
|
||||||
|
|
||||||
|
@ -1600,8 +1609,8 @@ public abstract class HBaseServer implements RpcServer {
|
||||||
}
|
}
|
||||||
// Enforcing the call queue size, this triggers a retry in the client
|
// Enforcing the call queue size, this triggers a retry in the client
|
||||||
if ((callSize + callQueueSize.get()) > maxQueueSize) {
|
if ((callSize + callQueueSize.get()) > maxQueueSize) {
|
||||||
final Call callTooBig =
|
final Call callTooBig = new Call(id, null, this, responder, callSize,
|
||||||
new Call(id, null, this, responder, callSize);
|
null);
|
||||||
ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
|
ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
|
||||||
setupResponse(responseBuffer, callTooBig, Status.FATAL,
|
setupResponse(responseBuffer, callTooBig, Status.FATAL,
|
||||||
IOException.class.getName(),
|
IOException.class.getName(),
|
||||||
|
@ -1616,8 +1625,8 @@ public abstract class HBaseServer implements RpcServer {
|
||||||
} catch (Throwable t) {
|
} catch (Throwable t) {
|
||||||
LOG.warn("Unable to read call parameters for client " +
|
LOG.warn("Unable to read call parameters for client " +
|
||||||
getHostAddress(), t);
|
getHostAddress(), t);
|
||||||
final Call readParamsFailedCall =
|
final Call readParamsFailedCall = new Call(id, null, this, responder,
|
||||||
new Call(id, null, this, responder, callSize);
|
callSize, null);
|
||||||
ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
|
ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
|
||||||
|
|
||||||
setupResponse(responseBuffer, readParamsFailedCall, Status.FATAL,
|
setupResponse(responseBuffer, readParamsFailedCall, Status.FATAL,
|
||||||
|
@ -1626,7 +1635,16 @@ public abstract class HBaseServer implements RpcServer {
|
||||||
responder.doRespond(readParamsFailedCall);
|
responder.doRespond(readParamsFailedCall);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
Call call = new Call(id, rpcRequestBody, this, responder, callSize);
|
|
||||||
|
Call call;
|
||||||
|
if (request.hasTinfo()) {
|
||||||
|
call = new Call(id, rpcRequestBody, this, responder, callSize,
|
||||||
|
new TraceInfo(request.getTinfo().getTraceId(), request.getTinfo()
|
||||||
|
.getParentId()));
|
||||||
|
} else {
|
||||||
|
call = new Call(id, rpcRequestBody, this, responder, callSize, null);
|
||||||
|
}
|
||||||
|
|
||||||
callQueueSize.add(callSize);
|
callQueueSize.add(callSize);
|
||||||
|
|
||||||
if (priorityCallQueue != null && getQosLevel(rpcRequestBody) > highPriorityLevel) {
|
if (priorityCallQueue != null && getQosLevel(rpcRequestBody) > highPriorityLevel) {
|
||||||
|
@ -1744,6 +1762,7 @@ public abstract class HBaseServer implements RpcServer {
|
||||||
status.setStatus("starting");
|
status.setStatus("starting");
|
||||||
SERVER.set(HBaseServer.this);
|
SERVER.set(HBaseServer.this);
|
||||||
while (running) {
|
while (running) {
|
||||||
|
|
||||||
try {
|
try {
|
||||||
status.pause("Waiting for a call");
|
status.pause("Waiting for a call");
|
||||||
Call call = myCallQueue.take(); // pop the queue; maybe blocked here
|
Call call = myCallQueue.take(); // pop the queue; maybe blocked here
|
||||||
|
@ -1761,10 +1780,16 @@ public abstract class HBaseServer implements RpcServer {
|
||||||
Message value = null;
|
Message value = null;
|
||||||
|
|
||||||
CurCall.set(call);
|
CurCall.set(call);
|
||||||
|
Span currentRequestSpan = NullSpan.getInstance();
|
||||||
try {
|
try {
|
||||||
if (!started)
|
if (!started)
|
||||||
throw new ServerNotRunningYetException("Server is not running yet");
|
throw new ServerNotRunningYetException("Server is not running yet");
|
||||||
|
|
||||||
|
if (call.tinfo != null) {
|
||||||
|
currentRequestSpan = Trace.startSpan(
|
||||||
|
"handling " + call.toString(), call.tinfo, Sampler.ALWAYS);
|
||||||
|
}
|
||||||
|
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
UserGroupInformation remoteUser = call.connection.user;
|
UserGroupInformation remoteUser = call.connection.user;
|
||||||
LOG.debug(getName() + ": call #" + call.id + " executing as "
|
LOG.debug(getName() + ": call #" + call.id + " executing as "
|
||||||
|
@ -1774,6 +1799,7 @@ public abstract class HBaseServer implements RpcServer {
|
||||||
|
|
||||||
RequestContext.set(User.create(call.connection.user), getRemoteIp(),
|
RequestContext.set(User.create(call.connection.user), getRemoteIp(),
|
||||||
call.connection.protocol);
|
call.connection.protocol);
|
||||||
|
|
||||||
// make the call
|
// make the call
|
||||||
value = call(call.connection.protocol, call.rpcRequestBody, call.timestamp,
|
value = call(call.connection.protocol, call.rpcRequestBody, call.timestamp,
|
||||||
status);
|
status);
|
||||||
|
@ -1782,6 +1808,7 @@ public abstract class HBaseServer implements RpcServer {
|
||||||
errorClass = e.getClass().getName();
|
errorClass = e.getClass().getName();
|
||||||
error = StringUtils.stringifyException(e);
|
error = StringUtils.stringifyException(e);
|
||||||
} finally {
|
} finally {
|
||||||
|
currentRequestSpan.stop();
|
||||||
// Must always clear the request context to avoid leaking
|
// Must always clear the request context to avoid leaking
|
||||||
// credentials between requests.
|
// credentials between requests.
|
||||||
RequestContext.clear();
|
RequestContext.clear();
|
||||||
|
|
|
@ -180,6 +180,7 @@ import org.apache.hadoop.metrics.util.MBeanUtil;
|
||||||
import org.apache.hadoop.net.DNS;
|
import org.apache.hadoop.net.DNS;
|
||||||
import org.apache.zookeeper.KeeperException;
|
import org.apache.zookeeper.KeeperException;
|
||||||
import org.apache.zookeeper.Watcher;
|
import org.apache.zookeeper.Watcher;
|
||||||
|
import org.apache.hadoop.hbase.trace.SpanReceiverHost;
|
||||||
|
|
||||||
import com.google.protobuf.RpcController;
|
import com.google.protobuf.RpcController;
|
||||||
import com.google.protobuf.ServiceException;
|
import com.google.protobuf.ServiceException;
|
||||||
|
@ -301,6 +302,7 @@ Server {
|
||||||
//should we check the compression codec type at master side, default true, HBASE-6370
|
//should we check the compression codec type at master side, default true, HBASE-6370
|
||||||
private final boolean masterCheckCompression;
|
private final boolean masterCheckCompression;
|
||||||
|
|
||||||
|
private SpanReceiverHost spanReceiverHost;
|
||||||
/**
|
/**
|
||||||
* Initializes the HMaster. The steps are as follows:
|
* Initializes the HMaster. The steps are as follows:
|
||||||
* <p>
|
* <p>
|
||||||
|
@ -637,6 +639,9 @@ Server {
|
||||||
status.setStatus("Initializing master coprocessors");
|
status.setStatus("Initializing master coprocessors");
|
||||||
this.cpHost = new MasterCoprocessorHost(this, this.conf);
|
this.cpHost = new MasterCoprocessorHost(this, this.conf);
|
||||||
|
|
||||||
|
spanReceiverHost = new SpanReceiverHost(getConfiguration());
|
||||||
|
spanReceiverHost.loadSpanReceivers();
|
||||||
|
|
||||||
// start up all service threads.
|
// start up all service threads.
|
||||||
status.setStatus("Initializing master service threads");
|
status.setStatus("Initializing master service threads");
|
||||||
startServiceThreads();
|
startServiceThreads();
|
||||||
|
@ -1964,6 +1969,7 @@ Server {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void shutdown() {
|
public void shutdown() {
|
||||||
|
spanReceiverHost.closeReceivers();
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
try {
|
try {
|
||||||
cpHost.preShutdown();
|
cpHost.preShutdown();
|
||||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||||
import org.apache.hadoop.hbase.master.RegionStates;
|
import org.apache.hadoop.hbase.master.RegionStates;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.zookeeper.KeeperException;
|
import org.apache.zookeeper.KeeperException;
|
||||||
|
import org.cloudera.htrace.Trace;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Handler to run disable of a table.
|
* Handler to run disable of a table.
|
||||||
|
@ -167,11 +168,11 @@ public class DisableTableHandler extends EventHandler {
|
||||||
for (HRegionInfo region: regions) {
|
for (HRegionInfo region: regions) {
|
||||||
if (regionStates.isRegionInTransition(region)) continue;
|
if (regionStates.isRegionInTransition(region)) continue;
|
||||||
final HRegionInfo hri = region;
|
final HRegionInfo hri = region;
|
||||||
pool.execute(new Runnable() {
|
pool.execute(Trace.wrap(new Runnable() {
|
||||||
public void run() {
|
public void run() {
|
||||||
assignmentManager.unassign(hri);
|
assignmentManager.unassign(hri);
|
||||||
}
|
}
|
||||||
});
|
}));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.master.HMaster;
|
||||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.zookeeper.KeeperException;
|
import org.apache.zookeeper.KeeperException;
|
||||||
|
import org.cloudera.htrace.Trace;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Handler to run enable of a table.
|
* Handler to run enable of a table.
|
||||||
|
@ -200,11 +201,11 @@ public class EnableTableHandler extends EventHandler {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
final HRegionInfo hri = region;
|
final HRegionInfo hri = region;
|
||||||
pool.execute(new Runnable() {
|
pool.execute(Trace.wrap(new Runnable() {
|
||||||
public void run() {
|
public void run() {
|
||||||
assignmentManager.assign(hri, true);
|
assignmentManager.assign(hri, true);
|
||||||
}
|
}
|
||||||
});
|
}));
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
try {
|
try {
|
||||||
|
|
|
@ -1105,6 +1105,11 @@ public final class RPCProtos {
|
||||||
// required uint32 callId = 1;
|
// required uint32 callId = 1;
|
||||||
boolean hasCallId();
|
boolean hasCallId();
|
||||||
int getCallId();
|
int getCallId();
|
||||||
|
|
||||||
|
// optional .RPCTInfo tinfo = 2;
|
||||||
|
boolean hasTinfo();
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo getTinfo();
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfoOrBuilder getTinfoOrBuilder();
|
||||||
}
|
}
|
||||||
public static final class RpcRequestHeader extends
|
public static final class RpcRequestHeader extends
|
||||||
com.google.protobuf.GeneratedMessage
|
com.google.protobuf.GeneratedMessage
|
||||||
|
@ -1145,8 +1150,22 @@ public final class RPCProtos {
|
||||||
return callId_;
|
return callId_;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// optional .RPCTInfo tinfo = 2;
|
||||||
|
public static final int TINFO_FIELD_NUMBER = 2;
|
||||||
|
private org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo tinfo_;
|
||||||
|
public boolean hasTinfo() {
|
||||||
|
return ((bitField0_ & 0x00000002) == 0x00000002);
|
||||||
|
}
|
||||||
|
public org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo getTinfo() {
|
||||||
|
return tinfo_;
|
||||||
|
}
|
||||||
|
public org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfoOrBuilder getTinfoOrBuilder() {
|
||||||
|
return tinfo_;
|
||||||
|
}
|
||||||
|
|
||||||
private void initFields() {
|
private void initFields() {
|
||||||
callId_ = 0;
|
callId_ = 0;
|
||||||
|
tinfo_ = org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.getDefaultInstance();
|
||||||
}
|
}
|
||||||
private byte memoizedIsInitialized = -1;
|
private byte memoizedIsInitialized = -1;
|
||||||
public final boolean isInitialized() {
|
public final boolean isInitialized() {
|
||||||
|
@ -1167,6 +1186,9 @@ public final class RPCProtos {
|
||||||
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
||||||
output.writeUInt32(1, callId_);
|
output.writeUInt32(1, callId_);
|
||||||
}
|
}
|
||||||
|
if (((bitField0_ & 0x00000002) == 0x00000002)) {
|
||||||
|
output.writeMessage(2, tinfo_);
|
||||||
|
}
|
||||||
getUnknownFields().writeTo(output);
|
getUnknownFields().writeTo(output);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1180,6 +1202,10 @@ public final class RPCProtos {
|
||||||
size += com.google.protobuf.CodedOutputStream
|
size += com.google.protobuf.CodedOutputStream
|
||||||
.computeUInt32Size(1, callId_);
|
.computeUInt32Size(1, callId_);
|
||||||
}
|
}
|
||||||
|
if (((bitField0_ & 0x00000002) == 0x00000002)) {
|
||||||
|
size += com.google.protobuf.CodedOutputStream
|
||||||
|
.computeMessageSize(2, tinfo_);
|
||||||
|
}
|
||||||
size += getUnknownFields().getSerializedSize();
|
size += getUnknownFields().getSerializedSize();
|
||||||
memoizedSerializedSize = size;
|
memoizedSerializedSize = size;
|
||||||
return size;
|
return size;
|
||||||
|
@ -1208,6 +1234,11 @@ public final class RPCProtos {
|
||||||
result = result && (getCallId()
|
result = result && (getCallId()
|
||||||
== other.getCallId());
|
== other.getCallId());
|
||||||
}
|
}
|
||||||
|
result = result && (hasTinfo() == other.hasTinfo());
|
||||||
|
if (hasTinfo()) {
|
||||||
|
result = result && getTinfo()
|
||||||
|
.equals(other.getTinfo());
|
||||||
|
}
|
||||||
result = result &&
|
result = result &&
|
||||||
getUnknownFields().equals(other.getUnknownFields());
|
getUnknownFields().equals(other.getUnknownFields());
|
||||||
return result;
|
return result;
|
||||||
|
@ -1221,6 +1252,10 @@ public final class RPCProtos {
|
||||||
hash = (37 * hash) + CALLID_FIELD_NUMBER;
|
hash = (37 * hash) + CALLID_FIELD_NUMBER;
|
||||||
hash = (53 * hash) + getCallId();
|
hash = (53 * hash) + getCallId();
|
||||||
}
|
}
|
||||||
|
if (hasTinfo()) {
|
||||||
|
hash = (37 * hash) + TINFO_FIELD_NUMBER;
|
||||||
|
hash = (53 * hash) + getTinfo().hashCode();
|
||||||
|
}
|
||||||
hash = (29 * hash) + getUnknownFields().hashCode();
|
hash = (29 * hash) + getUnknownFields().hashCode();
|
||||||
return hash;
|
return hash;
|
||||||
}
|
}
|
||||||
|
@ -1329,6 +1364,7 @@ public final class RPCProtos {
|
||||||
}
|
}
|
||||||
private void maybeForceBuilderInitialization() {
|
private void maybeForceBuilderInitialization() {
|
||||||
if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
|
if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
|
||||||
|
getTinfoFieldBuilder();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
private static Builder create() {
|
private static Builder create() {
|
||||||
|
@ -1339,6 +1375,12 @@ public final class RPCProtos {
|
||||||
super.clear();
|
super.clear();
|
||||||
callId_ = 0;
|
callId_ = 0;
|
||||||
bitField0_ = (bitField0_ & ~0x00000001);
|
bitField0_ = (bitField0_ & ~0x00000001);
|
||||||
|
if (tinfoBuilder_ == null) {
|
||||||
|
tinfo_ = org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.getDefaultInstance();
|
||||||
|
} else {
|
||||||
|
tinfoBuilder_.clear();
|
||||||
|
}
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000002);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1381,6 +1423,14 @@ public final class RPCProtos {
|
||||||
to_bitField0_ |= 0x00000001;
|
to_bitField0_ |= 0x00000001;
|
||||||
}
|
}
|
||||||
result.callId_ = callId_;
|
result.callId_ = callId_;
|
||||||
|
if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
|
||||||
|
to_bitField0_ |= 0x00000002;
|
||||||
|
}
|
||||||
|
if (tinfoBuilder_ == null) {
|
||||||
|
result.tinfo_ = tinfo_;
|
||||||
|
} else {
|
||||||
|
result.tinfo_ = tinfoBuilder_.build();
|
||||||
|
}
|
||||||
result.bitField0_ = to_bitField0_;
|
result.bitField0_ = to_bitField0_;
|
||||||
onBuilt();
|
onBuilt();
|
||||||
return result;
|
return result;
|
||||||
|
@ -1400,6 +1450,9 @@ public final class RPCProtos {
|
||||||
if (other.hasCallId()) {
|
if (other.hasCallId()) {
|
||||||
setCallId(other.getCallId());
|
setCallId(other.getCallId());
|
||||||
}
|
}
|
||||||
|
if (other.hasTinfo()) {
|
||||||
|
mergeTinfo(other.getTinfo());
|
||||||
|
}
|
||||||
this.mergeUnknownFields(other.getUnknownFields());
|
this.mergeUnknownFields(other.getUnknownFields());
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
@ -1440,6 +1493,15 @@ public final class RPCProtos {
|
||||||
callId_ = input.readUInt32();
|
callId_ = input.readUInt32();
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
case 18: {
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.newBuilder();
|
||||||
|
if (hasTinfo()) {
|
||||||
|
subBuilder.mergeFrom(getTinfo());
|
||||||
|
}
|
||||||
|
input.readMessage(subBuilder, extensionRegistry);
|
||||||
|
setTinfo(subBuilder.buildPartial());
|
||||||
|
break;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1467,6 +1529,96 @@ public final class RPCProtos {
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// optional .RPCTInfo tinfo = 2;
|
||||||
|
private org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo tinfo_ = org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.getDefaultInstance();
|
||||||
|
private com.google.protobuf.SingleFieldBuilder<
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo, org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfoOrBuilder> tinfoBuilder_;
|
||||||
|
public boolean hasTinfo() {
|
||||||
|
return ((bitField0_ & 0x00000002) == 0x00000002);
|
||||||
|
}
|
||||||
|
public org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo getTinfo() {
|
||||||
|
if (tinfoBuilder_ == null) {
|
||||||
|
return tinfo_;
|
||||||
|
} else {
|
||||||
|
return tinfoBuilder_.getMessage();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
public Builder setTinfo(org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo value) {
|
||||||
|
if (tinfoBuilder_ == null) {
|
||||||
|
if (value == null) {
|
||||||
|
throw new NullPointerException();
|
||||||
|
}
|
||||||
|
tinfo_ = value;
|
||||||
|
onChanged();
|
||||||
|
} else {
|
||||||
|
tinfoBuilder_.setMessage(value);
|
||||||
|
}
|
||||||
|
bitField0_ |= 0x00000002;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public Builder setTinfo(
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.Builder builderForValue) {
|
||||||
|
if (tinfoBuilder_ == null) {
|
||||||
|
tinfo_ = builderForValue.build();
|
||||||
|
onChanged();
|
||||||
|
} else {
|
||||||
|
tinfoBuilder_.setMessage(builderForValue.build());
|
||||||
|
}
|
||||||
|
bitField0_ |= 0x00000002;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public Builder mergeTinfo(org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo value) {
|
||||||
|
if (tinfoBuilder_ == null) {
|
||||||
|
if (((bitField0_ & 0x00000002) == 0x00000002) &&
|
||||||
|
tinfo_ != org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.getDefaultInstance()) {
|
||||||
|
tinfo_ =
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.newBuilder(tinfo_).mergeFrom(value).buildPartial();
|
||||||
|
} else {
|
||||||
|
tinfo_ = value;
|
||||||
|
}
|
||||||
|
onChanged();
|
||||||
|
} else {
|
||||||
|
tinfoBuilder_.mergeFrom(value);
|
||||||
|
}
|
||||||
|
bitField0_ |= 0x00000002;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public Builder clearTinfo() {
|
||||||
|
if (tinfoBuilder_ == null) {
|
||||||
|
tinfo_ = org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.getDefaultInstance();
|
||||||
|
onChanged();
|
||||||
|
} else {
|
||||||
|
tinfoBuilder_.clear();
|
||||||
|
}
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000002);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.Builder getTinfoBuilder() {
|
||||||
|
bitField0_ |= 0x00000002;
|
||||||
|
onChanged();
|
||||||
|
return getTinfoFieldBuilder().getBuilder();
|
||||||
|
}
|
||||||
|
public org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfoOrBuilder getTinfoOrBuilder() {
|
||||||
|
if (tinfoBuilder_ != null) {
|
||||||
|
return tinfoBuilder_.getMessageOrBuilder();
|
||||||
|
} else {
|
||||||
|
return tinfo_;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
private com.google.protobuf.SingleFieldBuilder<
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo, org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfoOrBuilder>
|
||||||
|
getTinfoFieldBuilder() {
|
||||||
|
if (tinfoBuilder_ == null) {
|
||||||
|
tinfoBuilder_ = new com.google.protobuf.SingleFieldBuilder<
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo, org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfoOrBuilder>(
|
||||||
|
tinfo_,
|
||||||
|
getParentForChildren(),
|
||||||
|
isClean());
|
||||||
|
tinfo_ = null;
|
||||||
|
}
|
||||||
|
return tinfoBuilder_;
|
||||||
|
}
|
||||||
|
|
||||||
// @@protoc_insertion_point(builder_scope:RpcRequestHeader)
|
// @@protoc_insertion_point(builder_scope:RpcRequestHeader)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -3603,22 +3755,23 @@ public final class RPCProtos {
|
||||||
descriptor;
|
descriptor;
|
||||||
static {
|
static {
|
||||||
java.lang.String[] descriptorData = {
|
java.lang.String[] descriptorData = {
|
||||||
"\n\tRPC.proto\":\n\017UserInformation\022\025\n\reffect" +
|
"\n\tRPC.proto\032\rTracing.proto\":\n\017UserInform" +
|
||||||
"iveUser\030\001 \002(\t\022\020\n\010realUser\030\002 \001(\t\"w\n\020Conne" +
|
"ation\022\025\n\reffectiveUser\030\001 \002(\t\022\020\n\010realUser" +
|
||||||
"ctionHeader\022\"\n\010userInfo\030\001 \001(\0132\020.UserInfo" +
|
"\030\002 \001(\t\"w\n\020ConnectionHeader\022\"\n\010userInfo\030\001" +
|
||||||
"rmation\022?\n\010protocol\030\002 \001(\t:-org.apache.ha" +
|
" \001(\0132\020.UserInformation\022?\n\010protocol\030\002 \001(\t" +
|
||||||
"doop.hbase.client.ClientProtocol\"\"\n\020RpcR" +
|
":-org.apache.hadoop.hbase.client.ClientP" +
|
||||||
"equestHeader\022\016\n\006callId\030\001 \002(\r\"n\n\016RpcReque" +
|
"rotocol\"<\n\020RpcRequestHeader\022\016\n\006callId\030\001 " +
|
||||||
"stBody\022\022\n\nmethodName\030\001 \002(\t\022\035\n\025clientProt" +
|
"\002(\r\022\030\n\005tinfo\030\002 \001(\0132\t.RPCTInfo\"n\n\016RpcRequ" +
|
||||||
"ocolVersion\030\002 \001(\004\022\017\n\007request\030\003 \001(\014\022\030\n\020re" +
|
"estBody\022\022\n\nmethodName\030\001 \002(\t\022\035\n\025clientPro" +
|
||||||
"questClassName\030\004 \001(\t\"{\n\021RpcResponseHeade" +
|
"tocolVersion\030\002 \001(\004\022\017\n\007request\030\003 \001(\014\022\030\n\020r" +
|
||||||
"r\022\016\n\006callId\030\001 \002(\r\022)\n\006status\030\002 \002(\0162\031.RpcR",
|
"equestClassName\030\004 \001(\t\"{\n\021RpcResponseHead",
|
||||||
"esponseHeader.Status\"+\n\006Status\022\013\n\007SUCCES" +
|
"er\022\016\n\006callId\030\001 \002(\r\022)\n\006status\030\002 \002(\0162\031.Rpc" +
|
||||||
"S\020\000\022\t\n\005ERROR\020\001\022\t\n\005FATAL\020\002\"#\n\017RpcResponse" +
|
"ResponseHeader.Status\"+\n\006Status\022\013\n\007SUCCE" +
|
||||||
"Body\022\020\n\010response\030\001 \001(\014\"9\n\014RpcException\022\025" +
|
"SS\020\000\022\t\n\005ERROR\020\001\022\t\n\005FATAL\020\002\"#\n\017RpcRespons" +
|
||||||
"\n\rexceptionName\030\001 \002(\t\022\022\n\nstackTrace\030\002 \001(" +
|
"eBody\022\020\n\010response\030\001 \001(\014\"9\n\014RpcException\022" +
|
||||||
"\tB<\n*org.apache.hadoop.hbase.protobuf.ge" +
|
"\025\n\rexceptionName\030\001 \002(\t\022\022\n\nstackTrace\030\002 \001" +
|
||||||
"neratedB\tRPCProtosH\001\240\001\001"
|
"(\tB<\n*org.apache.hadoop.hbase.protobuf.g" +
|
||||||
|
"eneratedB\tRPCProtosH\001\240\001\001"
|
||||||
};
|
};
|
||||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||||
|
@ -3646,7 +3799,7 @@ public final class RPCProtos {
|
||||||
internal_static_RpcRequestHeader_fieldAccessorTable = new
|
internal_static_RpcRequestHeader_fieldAccessorTable = new
|
||||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||||
internal_static_RpcRequestHeader_descriptor,
|
internal_static_RpcRequestHeader_descriptor,
|
||||||
new java.lang.String[] { "CallId", },
|
new java.lang.String[] { "CallId", "Tinfo", },
|
||||||
org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader.class,
|
org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader.class,
|
||||||
org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader.Builder.class);
|
org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader.Builder.class);
|
||||||
internal_static_RpcRequestBody_descriptor =
|
internal_static_RpcRequestBody_descriptor =
|
||||||
|
@ -3687,6 +3840,7 @@ public final class RPCProtos {
|
||||||
com.google.protobuf.Descriptors.FileDescriptor
|
com.google.protobuf.Descriptors.FileDescriptor
|
||||||
.internalBuildGeneratedFileFrom(descriptorData,
|
.internalBuildGeneratedFileFrom(descriptorData,
|
||||||
new com.google.protobuf.Descriptors.FileDescriptor[] {
|
new com.google.protobuf.Descriptors.FileDescriptor[] {
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.Tracing.getDescriptor(),
|
||||||
}, assigner);
|
}, assigner);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -46,6 +46,9 @@
|
||||||
* the Ping message. At the data level, this is just the bytes corresponding
|
* the Ping message. At the data level, this is just the bytes corresponding
|
||||||
* to integer -1.
|
* to integer -1.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
|
import "Tracing.proto";
|
||||||
|
|
||||||
option java_package = "org.apache.hadoop.hbase.protobuf.generated";
|
option java_package = "org.apache.hadoop.hbase.protobuf.generated";
|
||||||
option java_outer_classname = "RPCProtos";
|
option java_outer_classname = "RPCProtos";
|
||||||
option java_generate_equals_and_hash = true;
|
option java_generate_equals_and_hash = true;
|
||||||
|
@ -74,6 +77,7 @@ message ConnectionHeader {
|
||||||
message RpcRequestHeader {
|
message RpcRequestHeader {
|
||||||
/** Monotonically increasing callId, mostly to keep track of RPCs */
|
/** Monotonically increasing callId, mostly to keep track of RPCs */
|
||||||
required uint32 callId = 1;
|
required uint32 callId = 1;
|
||||||
|
optional RPCTInfo tinfo = 2;
|
||||||
}
|
}
|
||||||
/**
|
/**
|
||||||
* The RPC request body
|
* The RPC request body
|
||||||
|
|
Loading…
Reference in New Issue