HBASE-6524 revert due to new test failures against hadoop 2.0
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1376365 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
080b8fce76
commit
82609fa7b1
|
@ -451,11 +451,6 @@
|
|||
<artifactId>jettison</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.cloudera.htrace</groupId>
|
||||
<artifactId>htrace</artifactId>
|
||||
<version>1.48</version>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
<profiles>
|
||||
<!-- Skip the tests in this module -->
|
||||
|
|
|
@ -27,9 +27,6 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
|
||||
import org.cloudera.htrace.Sampler;
|
||||
import org.cloudera.htrace.Span;
|
||||
import org.cloudera.htrace.Trace;
|
||||
|
||||
|
||||
/**
|
||||
|
@ -79,8 +76,6 @@ public abstract class EventHandler implements Runnable, Comparable<Runnable> {
|
|||
// Time to wait for events to happen, should be kept short
|
||||
protected final int waitingTimeForEvents;
|
||||
|
||||
private final Span parent;
|
||||
|
||||
/**
|
||||
* This interface provides pre- and post-process hooks for events.
|
||||
*/
|
||||
|
@ -187,7 +182,6 @@ public abstract class EventHandler implements Runnable, Comparable<Runnable> {
|
|||
* Default base class constructor.
|
||||
*/
|
||||
public EventHandler(Server server, EventType eventType) {
|
||||
this.parent = Trace.currentTrace();
|
||||
this.server = server;
|
||||
this.eventType = eventType;
|
||||
seqid = seqids.incrementAndGet();
|
||||
|
@ -196,16 +190,12 @@ public abstract class EventHandler implements Runnable, Comparable<Runnable> {
|
|||
}
|
||||
|
||||
public void run() {
|
||||
Span chunk = Trace.startSpan(Thread.currentThread().getName(), parent,
|
||||
Sampler.ALWAYS);
|
||||
try {
|
||||
if (getListener() != null) getListener().beforeProcess(this);
|
||||
process();
|
||||
if (getListener() != null) getListener().afterProcess(this);
|
||||
} catch(Throwable t) {
|
||||
LOG.error("Caught throwable while processing event " + eventType, t);
|
||||
} finally {
|
||||
chunk.stop();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -63,7 +63,6 @@ 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.RpcRequestHeader;
|
||||
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.HBaseSaslRpcServer.AuthMethod;
|
||||
import org.apache.hadoop.hbase.security.KerberosInfo;
|
||||
|
@ -84,9 +83,6 @@ import org.apache.hadoop.security.UserGroupInformation;
|
|||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||
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.Message;
|
||||
|
@ -932,17 +928,8 @@ public class HBaseClient {
|
|||
try {
|
||||
if (LOG.isDebugEnabled())
|
||||
LOG.debug(getName() + " sending #" + call.id);
|
||||
|
||||
RpcRequestHeader.Builder headerBuilder = RPCProtos.RpcRequestHeader.newBuilder();
|
||||
headerBuilder.setCallId(call.id);
|
||||
|
||||
if (Trace.isTracing()) {
|
||||
Span s = Trace.currentTrace();
|
||||
headerBuilder.setTinfo(RPCTInfo.newBuilder()
|
||||
.setParentId(s.getSpanId())
|
||||
.setTraceId(s.getTraceId()));
|
||||
}
|
||||
|
||||
//noinspection SynchronizeOnNonFinalField
|
||||
synchronized (this.out) { // FindBugs IS2_INCONSISTENT_SYNC
|
||||
RpcRequestHeader header = headerBuilder.build();
|
||||
|
|
|
@ -107,11 +107,6 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|||
import com.google.protobuf.Message;
|
||||
|
||||
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
|
||||
* parameter, and return a single Protobuf message as their value. A service runs on
|
||||
|
@ -322,10 +317,9 @@ public abstract class HBaseServer implements RpcServer {
|
|||
// set at call completion
|
||||
protected long size; // size of current call
|
||||
protected boolean isError;
|
||||
protected TraceInfo tinfo;
|
||||
|
||||
public Call(int id, RpcRequestBody rpcRequestBody, Connection connection,
|
||||
Responder responder, long size, TraceInfo tinfo) {
|
||||
Responder responder, long size) {
|
||||
this.id = id;
|
||||
this.rpcRequestBody = rpcRequestBody;
|
||||
this.connection = connection;
|
||||
|
@ -335,7 +329,6 @@ public abstract class HBaseServer implements RpcServer {
|
|||
this.responder = responder;
|
||||
this.isError = false;
|
||||
this.size = size;
|
||||
this.tinfo = tinfo;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1130,14 +1123,13 @@ public abstract class HBaseServer implements RpcServer {
|
|||
private boolean useWrap = false;
|
||||
// Fake 'call' for failed authorization response
|
||||
private final int AUTHROIZATION_FAILED_CALLID = -1;
|
||||
private final Call authFailedCall = new Call(AUTHROIZATION_FAILED_CALLID,
|
||||
null, this, null, 0, null);
|
||||
private final Call authFailedCall =
|
||||
new Call(AUTHROIZATION_FAILED_CALLID, null, this, null, 0);
|
||||
private ByteArrayOutputStream authFailedResponse =
|
||||
new ByteArrayOutputStream();
|
||||
// Fake 'call' for SASL context setup
|
||||
private static final int SASL_CALLID = -33;
|
||||
private final Call saslCall = new Call(SASL_CALLID, null, this, null, 0,
|
||||
null);
|
||||
private final Call saslCall = new Call(SASL_CALLID, null, this, null, 0);
|
||||
|
||||
public UserGroupInformation attemptingUser = null; // user name before auth
|
||||
public Connection(SocketChannel channel, long lastContact) {
|
||||
|
@ -1485,7 +1477,7 @@ public abstract class HBaseServer implements RpcServer {
|
|||
// we return 0 which will keep the socket up -- bad clients, unless
|
||||
// they switch to suit the running server -- will fail later doing
|
||||
// getProtocolVersion.
|
||||
Call fakeCall = new Call(0, null, this, responder, 0, null);
|
||||
Call fakeCall = new Call(0, null, this, responder, 0);
|
||||
// Versions 3 and greater can interpret this exception
|
||||
// response in the same manner
|
||||
setupResponse(buffer, fakeCall, Status.FATAL,
|
||||
|
@ -1600,7 +1592,6 @@ public abstract class HBaseServer implements RpcServer {
|
|||
DataInputStream dis =
|
||||
new DataInputStream(new ByteArrayInputStream(buf));
|
||||
RpcRequestHeader request = RpcRequestHeader.parseDelimitedFrom(dis);
|
||||
|
||||
int id = request.getCallId();
|
||||
long callSize = buf.length;
|
||||
|
||||
|
@ -1609,8 +1600,8 @@ public abstract class HBaseServer implements RpcServer {
|
|||
}
|
||||
// Enforcing the call queue size, this triggers a retry in the client
|
||||
if ((callSize + callQueueSize.get()) > maxQueueSize) {
|
||||
final Call callTooBig = new Call(id, null, this, responder, callSize,
|
||||
null);
|
||||
final Call callTooBig =
|
||||
new Call(id, null, this, responder, callSize);
|
||||
ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
|
||||
setupResponse(responseBuffer, callTooBig, Status.FATAL,
|
||||
IOException.class.getName(),
|
||||
|
@ -1625,8 +1616,8 @@ public abstract class HBaseServer implements RpcServer {
|
|||
} catch (Throwable t) {
|
||||
LOG.warn("Unable to read call parameters for client " +
|
||||
getHostAddress(), t);
|
||||
final Call readParamsFailedCall = new Call(id, null, this, responder,
|
||||
callSize, null);
|
||||
final Call readParamsFailedCall =
|
||||
new Call(id, null, this, responder, callSize);
|
||||
ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
|
||||
|
||||
setupResponse(responseBuffer, readParamsFailedCall, Status.FATAL,
|
||||
|
@ -1635,16 +1626,7 @@ public abstract class HBaseServer implements RpcServer {
|
|||
responder.doRespond(readParamsFailedCall);
|
||||
return;
|
||||
}
|
||||
|
||||
Call call;
|
||||
if (request.hasTinfo()) {
|
||||
call = new Call(id, rpcRequestBody, this, responder, callSize,
|
||||
new TraceInfo(request.getTinfo().getTraceId(), request.getTinfo()
|
||||
.getParentId()));
|
||||
} else {
|
||||
call = new Call(id, rpcRequestBody, this, responder, callSize, null);
|
||||
}
|
||||
|
||||
Call call = new Call(id, rpcRequestBody, this, responder, callSize);
|
||||
callQueueSize.add(callSize);
|
||||
|
||||
if (priorityCallQueue != null && getQosLevel(rpcRequestBody) > highPriorityLevel) {
|
||||
|
@ -1762,7 +1744,6 @@ public abstract class HBaseServer implements RpcServer {
|
|||
status.setStatus("starting");
|
||||
SERVER.set(HBaseServer.this);
|
||||
while (running) {
|
||||
|
||||
try {
|
||||
status.pause("Waiting for a call");
|
||||
Call call = myCallQueue.take(); // pop the queue; maybe blocked here
|
||||
|
@ -1780,16 +1761,10 @@ public abstract class HBaseServer implements RpcServer {
|
|||
Message value = null;
|
||||
|
||||
CurCall.set(call);
|
||||
Span currentRequestSpan = NullSpan.getInstance();
|
||||
try {
|
||||
if (!started)
|
||||
throw new ServerNotRunningYetException("Server is not running yet");
|
||||
|
||||
if (call.tinfo != null) {
|
||||
currentRequestSpan = Trace.startSpan(
|
||||
"handling " + call.toString(), call.tinfo, Sampler.ALWAYS);
|
||||
}
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
UserGroupInformation remoteUser = call.connection.user;
|
||||
LOG.debug(getName() + ": call #" + call.id + " executing as "
|
||||
|
@ -1799,7 +1774,6 @@ public abstract class HBaseServer implements RpcServer {
|
|||
|
||||
RequestContext.set(User.create(call.connection.user), getRemoteIp(),
|
||||
call.connection.protocol);
|
||||
|
||||
// make the call
|
||||
value = call(call.connection.protocol, call.rpcRequestBody, call.timestamp,
|
||||
status);
|
||||
|
@ -1808,7 +1782,6 @@ public abstract class HBaseServer implements RpcServer {
|
|||
errorClass = e.getClass().getName();
|
||||
error = StringUtils.stringifyException(e);
|
||||
} finally {
|
||||
currentRequestSpan.stop();
|
||||
// Must always clear the request context to avoid leaking
|
||||
// credentials between requests.
|
||||
RequestContext.clear();
|
||||
|
|
|
@ -182,7 +182,6 @@ import org.apache.hadoop.metrics.util.MBeanUtil;
|
|||
import org.apache.hadoop.net.DNS;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.Watcher;
|
||||
import org.apache.hadoop.hbase.trace.SpanReceiverHost;
|
||||
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
@ -304,7 +303,6 @@ Server {
|
|||
//should we check the compression codec type at master side, default true, HBASE-6370
|
||||
private final boolean masterCheckCompression;
|
||||
|
||||
private SpanReceiverHost spanReceiverHost;
|
||||
/**
|
||||
* Initializes the HMaster. The steps are as follows:
|
||||
* <p>
|
||||
|
@ -639,9 +637,6 @@ Server {
|
|||
status.setStatus("Initializing master coprocessors");
|
||||
this.cpHost = new MasterCoprocessorHost(this, this.conf);
|
||||
|
||||
spanReceiverHost = new SpanReceiverHost(getConfiguration());
|
||||
spanReceiverHost.loadSpanReceivers();
|
||||
|
||||
// start up all service threads.
|
||||
status.setStatus("Initializing master service threads");
|
||||
startServiceThreads();
|
||||
|
@ -1967,7 +1962,6 @@ Server {
|
|||
}
|
||||
|
||||
public void shutdown() {
|
||||
spanReceiverHost.closeReceivers();
|
||||
if (cpHost != null) {
|
||||
try {
|
||||
cpHost.preShutdown();
|
||||
|
|
|
@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
|||
import org.apache.hadoop.hbase.master.RegionStates;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.cloudera.htrace.Trace;
|
||||
|
||||
/**
|
||||
* Handler to run disable of a table.
|
||||
|
@ -168,11 +167,11 @@ public class DisableTableHandler extends EventHandler {
|
|||
for (HRegionInfo region: regions) {
|
||||
if (regionStates.isRegionInTransition(region)) continue;
|
||||
final HRegionInfo hri = region;
|
||||
pool.execute(Trace.wrap(new Runnable() {
|
||||
pool.execute(new Runnable() {
|
||||
public void run() {
|
||||
assignmentManager.unassign(hri);
|
||||
}
|
||||
}));
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.master.HMaster;
|
|||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.cloudera.htrace.Trace;
|
||||
|
||||
/**
|
||||
* Handler to run enable of a table.
|
||||
|
@ -201,11 +200,11 @@ public class EnableTableHandler extends EventHandler {
|
|||
continue;
|
||||
}
|
||||
final HRegionInfo hri = region;
|
||||
pool.execute(Trace.wrap(new Runnable() {
|
||||
pool.execute(new Runnable() {
|
||||
public void run() {
|
||||
assignmentManager.assign(hri, true);
|
||||
}
|
||||
}));
|
||||
});
|
||||
}
|
||||
} else {
|
||||
try {
|
||||
|
|
|
@ -1,5 +1,5 @@
|
|||
// Generated by the protocol buffer compiler. DO NOT EDIT!
|
||||
// source: Filter.proto
|
||||
// source: filter.proto
|
||||
|
||||
package org.apache.hadoop.hbase.protobuf.generated;
|
||||
|
||||
|
@ -11041,7 +11041,7 @@ public final class FilterProtos {
|
|||
descriptor;
|
||||
static {
|
||||
java.lang.String[] descriptorData = {
|
||||
"\n\014Filter.proto\032\013hbase.proto\"%\n\024ColumnCou" +
|
||||
"\n\014filter.proto\032\013hbase.proto\"%\n\024ColumnCou" +
|
||||
"ntGetFilter\022\r\n\005limit\030\001 \002(\005\"7\n\026ColumnPagi" +
|
||||
"nationFilter\022\r\n\005limit\030\001 \002(\005\022\016\n\006offset\030\002 " +
|
||||
"\001(\005\"$\n\022ColumnPrefixFilter\022\016\n\006prefix\030\001 \002(" +
|
||||
|
|
|
@ -1105,11 +1105,6 @@ public final class RPCProtos {
|
|||
// required uint32 callId = 1;
|
||||
boolean hasCallId();
|
||||
int getCallId();
|
||||
|
||||
// optional .RPCTInfo tinfo = 2;
|
||||
boolean hasTinfo();
|
||||
org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo getTinfo();
|
||||
org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfoOrBuilder getTinfoOrBuilder();
|
||||
}
|
||||
public static final class RpcRequestHeader extends
|
||||
com.google.protobuf.GeneratedMessage
|
||||
|
@ -1150,22 +1145,8 @@ public final class RPCProtos {
|
|||
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() {
|
||||
callId_ = 0;
|
||||
tinfo_ = org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.getDefaultInstance();
|
||||
}
|
||||
private byte memoizedIsInitialized = -1;
|
||||
public final boolean isInitialized() {
|
||||
|
@ -1186,9 +1167,6 @@ public final class RPCProtos {
|
|||
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
output.writeUInt32(1, callId_);
|
||||
}
|
||||
if (((bitField0_ & 0x00000002) == 0x00000002)) {
|
||||
output.writeMessage(2, tinfo_);
|
||||
}
|
||||
getUnknownFields().writeTo(output);
|
||||
}
|
||||
|
||||
|
@ -1202,10 +1180,6 @@ public final class RPCProtos {
|
|||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeUInt32Size(1, callId_);
|
||||
}
|
||||
if (((bitField0_ & 0x00000002) == 0x00000002)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeMessageSize(2, tinfo_);
|
||||
}
|
||||
size += getUnknownFields().getSerializedSize();
|
||||
memoizedSerializedSize = size;
|
||||
return size;
|
||||
|
@ -1234,11 +1208,6 @@ public final class RPCProtos {
|
|||
result = result && (getCallId()
|
||||
== other.getCallId());
|
||||
}
|
||||
result = result && (hasTinfo() == other.hasTinfo());
|
||||
if (hasTinfo()) {
|
||||
result = result && getTinfo()
|
||||
.equals(other.getTinfo());
|
||||
}
|
||||
result = result &&
|
||||
getUnknownFields().equals(other.getUnknownFields());
|
||||
return result;
|
||||
|
@ -1252,10 +1221,6 @@ public final class RPCProtos {
|
|||
hash = (37 * hash) + CALLID_FIELD_NUMBER;
|
||||
hash = (53 * hash) + getCallId();
|
||||
}
|
||||
if (hasTinfo()) {
|
||||
hash = (37 * hash) + TINFO_FIELD_NUMBER;
|
||||
hash = (53 * hash) + getTinfo().hashCode();
|
||||
}
|
||||
hash = (29 * hash) + getUnknownFields().hashCode();
|
||||
return hash;
|
||||
}
|
||||
|
@ -1364,7 +1329,6 @@ public final class RPCProtos {
|
|||
}
|
||||
private void maybeForceBuilderInitialization() {
|
||||
if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
|
||||
getTinfoFieldBuilder();
|
||||
}
|
||||
}
|
||||
private static Builder create() {
|
||||
|
@ -1375,12 +1339,6 @@ public final class RPCProtos {
|
|||
super.clear();
|
||||
callId_ = 0;
|
||||
bitField0_ = (bitField0_ & ~0x00000001);
|
||||
if (tinfoBuilder_ == null) {
|
||||
tinfo_ = org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.getDefaultInstance();
|
||||
} else {
|
||||
tinfoBuilder_.clear();
|
||||
}
|
||||
bitField0_ = (bitField0_ & ~0x00000002);
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -1423,14 +1381,6 @@ public final class RPCProtos {
|
|||
to_bitField0_ |= 0x00000001;
|
||||
}
|
||||
result.callId_ = callId_;
|
||||
if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
|
||||
to_bitField0_ |= 0x00000002;
|
||||
}
|
||||
if (tinfoBuilder_ == null) {
|
||||
result.tinfo_ = tinfo_;
|
||||
} else {
|
||||
result.tinfo_ = tinfoBuilder_.build();
|
||||
}
|
||||
result.bitField0_ = to_bitField0_;
|
||||
onBuilt();
|
||||
return result;
|
||||
|
@ -1450,9 +1400,6 @@ public final class RPCProtos {
|
|||
if (other.hasCallId()) {
|
||||
setCallId(other.getCallId());
|
||||
}
|
||||
if (other.hasTinfo()) {
|
||||
mergeTinfo(other.getTinfo());
|
||||
}
|
||||
this.mergeUnknownFields(other.getUnknownFields());
|
||||
return this;
|
||||
}
|
||||
|
@ -1493,15 +1440,6 @@ public final class RPCProtos {
|
|||
callId_ = input.readUInt32();
|
||||
break;
|
||||
}
|
||||
case 18: {
|
||||
org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.newBuilder();
|
||||
if (hasTinfo()) {
|
||||
subBuilder.mergeFrom(getTinfo());
|
||||
}
|
||||
input.readMessage(subBuilder, extensionRegistry);
|
||||
setTinfo(subBuilder.buildPartial());
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -1529,96 +1467,6 @@ public final class RPCProtos {
|
|||
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)
|
||||
}
|
||||
|
||||
|
@ -3755,23 +3603,22 @@ public final class RPCProtos {
|
|||
descriptor;
|
||||
static {
|
||||
java.lang.String[] descriptorData = {
|
||||
"\n\tRPC.proto\032\rTracing.proto\":\n\017UserInform" +
|
||||
"ation\022\025\n\reffectiveUser\030\001 \002(\t\022\020\n\010realUser" +
|
||||
"\030\002 \001(\t\"w\n\020ConnectionHeader\022\"\n\010userInfo\030\001" +
|
||||
" \001(\0132\020.UserInformation\022?\n\010protocol\030\002 \001(\t" +
|
||||
":-org.apache.hadoop.hbase.client.ClientP" +
|
||||
"rotocol\"<\n\020RpcRequestHeader\022\016\n\006callId\030\001 " +
|
||||
"\002(\r\022\030\n\005tinfo\030\002 \001(\0132\t.RPCTInfo\"n\n\016RpcRequ" +
|
||||
"estBody\022\022\n\nmethodName\030\001 \002(\t\022\035\n\025clientPro" +
|
||||
"tocolVersion\030\002 \001(\004\022\017\n\007request\030\003 \001(\014\022\030\n\020r" +
|
||||
"equestClassName\030\004 \001(\t\"{\n\021RpcResponseHead",
|
||||
"er\022\016\n\006callId\030\001 \002(\r\022)\n\006status\030\002 \002(\0162\031.Rpc" +
|
||||
"ResponseHeader.Status\"+\n\006Status\022\013\n\007SUCCE" +
|
||||
"SS\020\000\022\t\n\005ERROR\020\001\022\t\n\005FATAL\020\002\"#\n\017RpcRespons" +
|
||||
"eBody\022\020\n\010response\030\001 \001(\014\"9\n\014RpcException\022" +
|
||||
"\025\n\rexceptionName\030\001 \002(\t\022\022\n\nstackTrace\030\002 \001" +
|
||||
"(\tB<\n*org.apache.hadoop.hbase.protobuf.g" +
|
||||
"eneratedB\tRPCProtosH\001\240\001\001"
|
||||
"\n\tRPC.proto\":\n\017UserInformation\022\025\n\reffect" +
|
||||
"iveUser\030\001 \002(\t\022\020\n\010realUser\030\002 \001(\t\"w\n\020Conne" +
|
||||
"ctionHeader\022\"\n\010userInfo\030\001 \001(\0132\020.UserInfo" +
|
||||
"rmation\022?\n\010protocol\030\002 \001(\t:-org.apache.ha" +
|
||||
"doop.hbase.client.ClientProtocol\"\"\n\020RpcR" +
|
||||
"equestHeader\022\016\n\006callId\030\001 \002(\r\"n\n\016RpcReque" +
|
||||
"stBody\022\022\n\nmethodName\030\001 \002(\t\022\035\n\025clientProt" +
|
||||
"ocolVersion\030\002 \001(\004\022\017\n\007request\030\003 \001(\014\022\030\n\020re" +
|
||||
"questClassName\030\004 \001(\t\"{\n\021RpcResponseHeade" +
|
||||
"r\022\016\n\006callId\030\001 \002(\r\022)\n\006status\030\002 \002(\0162\031.RpcR",
|
||||
"esponseHeader.Status\"+\n\006Status\022\013\n\007SUCCES" +
|
||||
"S\020\000\022\t\n\005ERROR\020\001\022\t\n\005FATAL\020\002\"#\n\017RpcResponse" +
|
||||
"Body\022\020\n\010response\030\001 \001(\014\"9\n\014RpcException\022\025" +
|
||||
"\n\rexceptionName\030\001 \002(\t\022\022\n\nstackTrace\030\002 \001(" +
|
||||
"\tB<\n*org.apache.hadoop.hbase.protobuf.ge" +
|
||||
"neratedB\tRPCProtosH\001\240\001\001"
|
||||
};
|
||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||
|
@ -3799,7 +3646,7 @@ public final class RPCProtos {
|
|||
internal_static_RpcRequestHeader_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_RpcRequestHeader_descriptor,
|
||||
new java.lang.String[] { "CallId", "Tinfo", },
|
||||
new java.lang.String[] { "CallId", },
|
||||
org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader.class,
|
||||
org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader.Builder.class);
|
||||
internal_static_RpcRequestBody_descriptor =
|
||||
|
@ -3840,7 +3687,6 @@ public final class RPCProtos {
|
|||
com.google.protobuf.Descriptors.FileDescriptor
|
||||
.internalBuildGeneratedFileFrom(descriptorData,
|
||||
new com.google.protobuf.Descriptors.FileDescriptor[] {
|
||||
org.apache.hadoop.hbase.protobuf.generated.Tracing.getDescriptor(),
|
||||
}, assigner);
|
||||
}
|
||||
|
||||
|
|
|
@ -1,490 +0,0 @@
|
|||
// Generated by the protocol buffer compiler. DO NOT EDIT!
|
||||
// source: Tracing.proto
|
||||
|
||||
package org.apache.hadoop.hbase.protobuf.generated;
|
||||
|
||||
public final class Tracing {
|
||||
private Tracing() {}
|
||||
public static void registerAllExtensions(
|
||||
com.google.protobuf.ExtensionRegistry registry) {
|
||||
}
|
||||
public interface RPCTInfoOrBuilder
|
||||
extends com.google.protobuf.MessageOrBuilder {
|
||||
|
||||
// optional int64 traceId = 1;
|
||||
boolean hasTraceId();
|
||||
long getTraceId();
|
||||
|
||||
// optional int64 parentId = 2;
|
||||
boolean hasParentId();
|
||||
long getParentId();
|
||||
}
|
||||
public static final class RPCTInfo extends
|
||||
com.google.protobuf.GeneratedMessage
|
||||
implements RPCTInfoOrBuilder {
|
||||
// Use RPCTInfo.newBuilder() to construct.
|
||||
private RPCTInfo(Builder builder) {
|
||||
super(builder);
|
||||
}
|
||||
private RPCTInfo(boolean noInit) {}
|
||||
|
||||
private static final RPCTInfo defaultInstance;
|
||||
public static RPCTInfo getDefaultInstance() {
|
||||
return defaultInstance;
|
||||
}
|
||||
|
||||
public RPCTInfo getDefaultInstanceForType() {
|
||||
return defaultInstance;
|
||||
}
|
||||
|
||||
public static final com.google.protobuf.Descriptors.Descriptor
|
||||
getDescriptor() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.Tracing.internal_static_RPCTInfo_descriptor;
|
||||
}
|
||||
|
||||
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||
internalGetFieldAccessorTable() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.Tracing.internal_static_RPCTInfo_fieldAccessorTable;
|
||||
}
|
||||
|
||||
private int bitField0_;
|
||||
// optional int64 traceId = 1;
|
||||
public static final int TRACEID_FIELD_NUMBER = 1;
|
||||
private long traceId_;
|
||||
public boolean hasTraceId() {
|
||||
return ((bitField0_ & 0x00000001) == 0x00000001);
|
||||
}
|
||||
public long getTraceId() {
|
||||
return traceId_;
|
||||
}
|
||||
|
||||
// optional int64 parentId = 2;
|
||||
public static final int PARENTID_FIELD_NUMBER = 2;
|
||||
private long parentId_;
|
||||
public boolean hasParentId() {
|
||||
return ((bitField0_ & 0x00000002) == 0x00000002);
|
||||
}
|
||||
public long getParentId() {
|
||||
return parentId_;
|
||||
}
|
||||
|
||||
private void initFields() {
|
||||
traceId_ = 0L;
|
||||
parentId_ = 0L;
|
||||
}
|
||||
private byte memoizedIsInitialized = -1;
|
||||
public final boolean isInitialized() {
|
||||
byte isInitialized = memoizedIsInitialized;
|
||||
if (isInitialized != -1) return isInitialized == 1;
|
||||
|
||||
memoizedIsInitialized = 1;
|
||||
return true;
|
||||
}
|
||||
|
||||
public void writeTo(com.google.protobuf.CodedOutputStream output)
|
||||
throws java.io.IOException {
|
||||
getSerializedSize();
|
||||
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
output.writeInt64(1, traceId_);
|
||||
}
|
||||
if (((bitField0_ & 0x00000002) == 0x00000002)) {
|
||||
output.writeInt64(2, parentId_);
|
||||
}
|
||||
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
|
||||
.computeInt64Size(1, traceId_);
|
||||
}
|
||||
if (((bitField0_ & 0x00000002) == 0x00000002)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeInt64Size(2, parentId_);
|
||||
}
|
||||
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.Tracing.RPCTInfo)) {
|
||||
return super.equals(obj);
|
||||
}
|
||||
org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo other = (org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo) obj;
|
||||
|
||||
boolean result = true;
|
||||
result = result && (hasTraceId() == other.hasTraceId());
|
||||
if (hasTraceId()) {
|
||||
result = result && (getTraceId()
|
||||
== other.getTraceId());
|
||||
}
|
||||
result = result && (hasParentId() == other.hasParentId());
|
||||
if (hasParentId()) {
|
||||
result = result && (getParentId()
|
||||
== other.getParentId());
|
||||
}
|
||||
result = result &&
|
||||
getUnknownFields().equals(other.getUnknownFields());
|
||||
return result;
|
||||
}
|
||||
|
||||
@java.lang.Override
|
||||
public int hashCode() {
|
||||
int hash = 41;
|
||||
hash = (19 * hash) + getDescriptorForType().hashCode();
|
||||
if (hasTraceId()) {
|
||||
hash = (37 * hash) + TRACEID_FIELD_NUMBER;
|
||||
hash = (53 * hash) + hashLong(getTraceId());
|
||||
}
|
||||
if (hasParentId()) {
|
||||
hash = (37 * hash) + PARENTID_FIELD_NUMBER;
|
||||
hash = (53 * hash) + hashLong(getParentId());
|
||||
}
|
||||
hash = (29 * hash) + getUnknownFields().hashCode();
|
||||
return hash;
|
||||
}
|
||||
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo parseFrom(
|
||||
com.google.protobuf.ByteString data)
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
return newBuilder().mergeFrom(data).buildParsed();
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo 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.Tracing.RPCTInfo parseFrom(byte[] data)
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
return newBuilder().mergeFrom(data).buildParsed();
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo 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.Tracing.RPCTInfo parseFrom(java.io.InputStream input)
|
||||
throws java.io.IOException {
|
||||
return newBuilder().mergeFrom(input).buildParsed();
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo 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.Tracing.RPCTInfo 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.Tracing.RPCTInfo 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.Tracing.RPCTInfo parseFrom(
|
||||
com.google.protobuf.CodedInputStream input)
|
||||
throws java.io.IOException {
|
||||
return newBuilder().mergeFrom(input).buildParsed();
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo 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.Tracing.RPCTInfo 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<Builder>
|
||||
implements org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfoOrBuilder {
|
||||
public static final com.google.protobuf.Descriptors.Descriptor
|
||||
getDescriptor() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.Tracing.internal_static_RPCTInfo_descriptor;
|
||||
}
|
||||
|
||||
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||
internalGetFieldAccessorTable() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.Tracing.internal_static_RPCTInfo_fieldAccessorTable;
|
||||
}
|
||||
|
||||
// Construct using org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.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();
|
||||
traceId_ = 0L;
|
||||
bitField0_ = (bitField0_ & ~0x00000001);
|
||||
parentId_ = 0L;
|
||||
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.Tracing.RPCTInfo.getDescriptor();
|
||||
}
|
||||
|
||||
public org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo getDefaultInstanceForType() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.getDefaultInstance();
|
||||
}
|
||||
|
||||
public org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo build() {
|
||||
org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo result = buildPartial();
|
||||
if (!result.isInitialized()) {
|
||||
throw newUninitializedMessageException(result);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
private org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo buildParsed()
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo result = buildPartial();
|
||||
if (!result.isInitialized()) {
|
||||
throw newUninitializedMessageException(
|
||||
result).asInvalidProtocolBufferException();
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
public org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo buildPartial() {
|
||||
org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo result = new org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo(this);
|
||||
int from_bitField0_ = bitField0_;
|
||||
int to_bitField0_ = 0;
|
||||
if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
to_bitField0_ |= 0x00000001;
|
||||
}
|
||||
result.traceId_ = traceId_;
|
||||
if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
|
||||
to_bitField0_ |= 0x00000002;
|
||||
}
|
||||
result.parentId_ = parentId_;
|
||||
result.bitField0_ = to_bitField0_;
|
||||
onBuilt();
|
||||
return result;
|
||||
}
|
||||
|
||||
public Builder mergeFrom(com.google.protobuf.Message other) {
|
||||
if (other instanceof org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo) {
|
||||
return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo)other);
|
||||
} else {
|
||||
super.mergeFrom(other);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo other) {
|
||||
if (other == org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.getDefaultInstance()) return this;
|
||||
if (other.hasTraceId()) {
|
||||
setTraceId(other.getTraceId());
|
||||
}
|
||||
if (other.hasParentId()) {
|
||||
setParentId(other.getParentId());
|
||||
}
|
||||
this.mergeUnknownFields(other.getUnknownFields());
|
||||
return this;
|
||||
}
|
||||
|
||||
public final boolean isInitialized() {
|
||||
return true;
|
||||
}
|
||||
|
||||
public Builder mergeFrom(
|
||||
com.google.protobuf.CodedInputStream input,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws java.io.IOException {
|
||||
com.google.protobuf.UnknownFieldSet.Builder unknownFields =
|
||||
com.google.protobuf.UnknownFieldSet.newBuilder(
|
||||
this.getUnknownFields());
|
||||
while (true) {
|
||||
int tag = input.readTag();
|
||||
switch (tag) {
|
||||
case 0:
|
||||
this.setUnknownFields(unknownFields.build());
|
||||
onChanged();
|
||||
return this;
|
||||
default: {
|
||||
if (!parseUnknownField(input, unknownFields,
|
||||
extensionRegistry, tag)) {
|
||||
this.setUnknownFields(unknownFields.build());
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
break;
|
||||
}
|
||||
case 8: {
|
||||
bitField0_ |= 0x00000001;
|
||||
traceId_ = input.readInt64();
|
||||
break;
|
||||
}
|
||||
case 16: {
|
||||
bitField0_ |= 0x00000002;
|
||||
parentId_ = input.readInt64();
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private int bitField0_;
|
||||
|
||||
// optional int64 traceId = 1;
|
||||
private long traceId_ ;
|
||||
public boolean hasTraceId() {
|
||||
return ((bitField0_ & 0x00000001) == 0x00000001);
|
||||
}
|
||||
public long getTraceId() {
|
||||
return traceId_;
|
||||
}
|
||||
public Builder setTraceId(long value) {
|
||||
bitField0_ |= 0x00000001;
|
||||
traceId_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
public Builder clearTraceId() {
|
||||
bitField0_ = (bitField0_ & ~0x00000001);
|
||||
traceId_ = 0L;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
|
||||
// optional int64 parentId = 2;
|
||||
private long parentId_ ;
|
||||
public boolean hasParentId() {
|
||||
return ((bitField0_ & 0x00000002) == 0x00000002);
|
||||
}
|
||||
public long getParentId() {
|
||||
return parentId_;
|
||||
}
|
||||
public Builder setParentId(long value) {
|
||||
bitField0_ |= 0x00000002;
|
||||
parentId_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
public Builder clearParentId() {
|
||||
bitField0_ = (bitField0_ & ~0x00000002);
|
||||
parentId_ = 0L;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(builder_scope:RPCTInfo)
|
||||
}
|
||||
|
||||
static {
|
||||
defaultInstance = new RPCTInfo(true);
|
||||
defaultInstance.initFields();
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(class_scope:RPCTInfo)
|
||||
}
|
||||
|
||||
private static com.google.protobuf.Descriptors.Descriptor
|
||||
internal_static_RPCTInfo_descriptor;
|
||||
private static
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||
internal_static_RPCTInfo_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\rTracing.proto\"-\n\010RPCTInfo\022\017\n\007traceId\030\001" +
|
||||
" \001(\003\022\020\n\010parentId\030\002 \001(\003B:\n*org.apache.had" +
|
||||
"oop.hbase.protobuf.generatedB\007TracingH\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_RPCTInfo_descriptor =
|
||||
getDescriptor().getMessageTypes().get(0);
|
||||
internal_static_RPCTInfo_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_RPCTInfo_descriptor,
|
||||
new java.lang.String[] { "TraceId", "ParentId", },
|
||||
org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.class,
|
||||
org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo.Builder.class);
|
||||
return null;
|
||||
}
|
||||
};
|
||||
com.google.protobuf.Descriptors.FileDescriptor
|
||||
.internalBuildGeneratedFileFrom(descriptorData,
|
||||
new com.google.protobuf.Descriptors.FileDescriptor[] {
|
||||
}, assigner);
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(outer_class_scope)
|
||||
}
|
|
@ -1,91 +0,0 @@
|
|||
/**
|
||||
* Copyright 2010 The Apache Software Foundation
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.trace;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configurable;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.cloudera.htrace.Span;
|
||||
import org.cloudera.htrace.SpanReceiver;
|
||||
import org.cloudera.htrace.Trace;
|
||||
import org.cloudera.htrace.impl.LocalFileSpanReceiver;
|
||||
|
||||
/**
|
||||
* Wraps the LocalFileSpanReceiver provided in
|
||||
* org.cloudera.htrace.impl.LocalFileSpanReceiver to read the file name
|
||||
* destination for spans from hbase-site.xml.
|
||||
*
|
||||
* The file path should be added as a property with name
|
||||
* "hbase.trace.spanreceiver.localfilespanreceiver.filename".
|
||||
*/
|
||||
public class HBaseLocalFileSpanReceiver implements SpanReceiver, Configurable {
|
||||
public static final Log LOG = LogFactory
|
||||
.getLog(HBaseLocalFileSpanReceiver.class);
|
||||
public static final String FILE_NAME_CONF_KEY = "hbase.trace.spanreceiver.localfilespanreceiver.filename";
|
||||
private Configuration conf;
|
||||
private LocalFileSpanReceiver rcvr;
|
||||
|
||||
@Override
|
||||
public Configuration getConf() {
|
||||
return conf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setConf(Configuration arg0) {
|
||||
this.conf = arg0;
|
||||
// replace rcvr if it was already created
|
||||
if (rcvr != null) {
|
||||
try {
|
||||
rcvr.close();
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Error closing LocalFileSpanReceiver.", e);
|
||||
}
|
||||
}
|
||||
try {
|
||||
rcvr = new LocalFileSpanReceiver(conf.get(FILE_NAME_CONF_KEY));
|
||||
} catch (IOException e) {
|
||||
Trace.removeReceiver(this);
|
||||
rcvr = null;
|
||||
LOG.warn(
|
||||
"Unable to initialize LocalFileSpanReceiver, removing owner (HBaseLocalFileSpanReceiver) from receiver list.",
|
||||
e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try{
|
||||
if (rcvr != null) {
|
||||
rcvr.close();
|
||||
}
|
||||
} finally {
|
||||
rcvr = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void receiveSpan(Span span) {
|
||||
if (rcvr != null) {
|
||||
rcvr.receiveSpan(span);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,115 +0,0 @@
|
|||
/**
|
||||
* Copyright 2010 The Apache Software Foundation
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.trace;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.HashSet;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
import org.cloudera.htrace.SpanReceiver;
|
||||
import org.cloudera.htrace.Trace;
|
||||
|
||||
/**
|
||||
* This class provides functions for reading the names of SpanReceivers from
|
||||
* hbase-site.xml, adding those SpanReceivers to the Tracer, and closing those
|
||||
* SpanReceivers when appropriate.
|
||||
*/
|
||||
public class SpanReceiverHost {
|
||||
public static final String SPAN_RECEIVERS_CONF_KEY = "hbase.trace.spanreceiver.classes";
|
||||
private static final Log LOG = LogFactory.getLog(SpanReceiverHost.class);
|
||||
private Collection<SpanReceiver> receivers;
|
||||
private Configuration conf;
|
||||
|
||||
public SpanReceiverHost(Configuration conf) {
|
||||
receivers = new HashSet<SpanReceiver>();
|
||||
this.conf = conf;
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the names of classes specified in the
|
||||
* "hbase.trace.spanreceiver.classes" property and instantiates and registers
|
||||
* them with the Tracer as SpanReceiver's.
|
||||
*
|
||||
* The nullary constructor is called during construction, but if the classes
|
||||
* specified implement the Configurable interface, setConfiguration() will be
|
||||
* called on them. This allows SpanReceivers to use values from
|
||||
* hbase-site.xml. See
|
||||
* {@link org.apache.hadoop.hbase.trace.HBaseLocalFileSpanReceiver} for an
|
||||
* example.
|
||||
*/
|
||||
public void loadSpanReceivers() {
|
||||
Class<?> implClass = null;
|
||||
String[] receiverNames = conf.getStrings(SPAN_RECEIVERS_CONF_KEY);
|
||||
if (receiverNames == null || receiverNames.length == 0) {
|
||||
return;
|
||||
}
|
||||
for (String className : receiverNames) {
|
||||
className = className.trim();
|
||||
|
||||
try {
|
||||
implClass = Class.forName(className);
|
||||
receivers.add(loadInstance(implClass));
|
||||
LOG.info("SpanReceiver " + className + " was loaded successfully.");
|
||||
} catch (ClassNotFoundException e) {
|
||||
LOG.warn("Class " + className + " cannot be found. " + e.getMessage());
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Load SpanReceiver " + className + " failed. "
|
||||
+ e.getMessage());
|
||||
}
|
||||
}
|
||||
for (SpanReceiver rcvr : receivers) {
|
||||
Trace.addReceiver(rcvr);
|
||||
}
|
||||
}
|
||||
|
||||
private SpanReceiver loadInstance(Class<?> implClass)
|
||||
throws IOException {
|
||||
SpanReceiver impl;
|
||||
try {
|
||||
Object o = ReflectionUtils.newInstance(implClass, conf);
|
||||
impl = (SpanReceiver)o;
|
||||
} catch (SecurityException e) {
|
||||
throw new IOException(e);
|
||||
} catch (IllegalArgumentException e) {
|
||||
throw new IOException(e);
|
||||
} catch (RuntimeException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
|
||||
return impl;
|
||||
}
|
||||
|
||||
/**
|
||||
* Calls close() on all SpanReceivers created by this SpanReceiverHost.
|
||||
*/
|
||||
public void closeReceivers() {
|
||||
for (SpanReceiver rcvr : receivers) {
|
||||
try {
|
||||
rcvr.close();
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Unable to close SpanReceiver correctly: " + e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -46,9 +46,6 @@
|
|||
* the Ping message. At the data level, this is just the bytes corresponding
|
||||
* to integer -1.
|
||||
*/
|
||||
|
||||
import "Tracing.proto";
|
||||
|
||||
option java_package = "org.apache.hadoop.hbase.protobuf.generated";
|
||||
option java_outer_classname = "RPCProtos";
|
||||
option java_generate_equals_and_hash = true;
|
||||
|
@ -77,7 +74,6 @@ message ConnectionHeader {
|
|||
message RpcRequestHeader {
|
||||
/** Monotonically increasing callId, mostly to keep track of RPCs */
|
||||
required uint32 callId = 1;
|
||||
optional RPCTInfo tinfo = 2;
|
||||
}
|
||||
/**
|
||||
* The RPC request body
|
||||
|
|
|
@ -1,33 +0,0 @@
|
|||
/**
|
||||
* Copyright 2010 The Apache Software Foundation
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
option java_package = "org.apache.hadoop.hbase.protobuf.generated";
|
||||
option java_outer_classname = "Tracing";
|
||||
option java_generate_equals_and_hash = true;
|
||||
option optimize_for = SPEED;
|
||||
|
||||
//Used to pass through the information necessary to continue
|
||||
//a trace after an RPC is made. All we need is the traceid
|
||||
//(so we know the overarching trace this message is a part of), and
|
||||
//the id of the current span when this message was sent, so we know
|
||||
//what span caused the new span we will create when this message is received.
|
||||
message RPCTInfo {
|
||||
optional int64 traceId = 1;
|
||||
optional int64 parentId = 2;
|
||||
}
|
|
@ -1,102 +0,0 @@
|
|||
package org.apache.hadoop.hbase.trace;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.util.Collection;
|
||||
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.cloudera.htrace.Sampler;
|
||||
import org.cloudera.htrace.Span;
|
||||
import org.cloudera.htrace.Trace;
|
||||
import org.cloudera.htrace.TraceTree;
|
||||
import org.cloudera.htrace.impl.POJOSpanReceiver;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import com.google.common.collect.Multimap;
|
||||
|
||||
@Category(MediumTests.class)
|
||||
public class TestHTraceHooks {
|
||||
|
||||
private static final byte[] FAMILY_BYTES = "family".getBytes();
|
||||
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
private static final POJOSpanReceiver rcvr = new POJOSpanReceiver();
|
||||
|
||||
@BeforeClass
|
||||
public static void before() throws Exception {
|
||||
TEST_UTIL.startMiniCluster(2, 3);
|
||||
Trace.addReceiver(rcvr);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void after() throws Exception {
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
Trace.removeReceiver(rcvr);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTraceCreateTable() throws Exception {
|
||||
Span tableCreationSpan = Trace.startSpan("creating table", Sampler.ALWAYS);
|
||||
HTable table;
|
||||
try {
|
||||
table = TEST_UTIL.createTable("table".getBytes(),
|
||||
FAMILY_BYTES);
|
||||
} finally {
|
||||
tableCreationSpan.stop();
|
||||
}
|
||||
|
||||
Collection<Span> spans = rcvr.getSpans();
|
||||
TraceTree traceTree = new TraceTree(spans);
|
||||
Collection<Span> roots = traceTree.getRoots();
|
||||
|
||||
assertEquals(1, roots.size());
|
||||
Span createTableRoot = roots.iterator().next();
|
||||
|
||||
assertEquals("creating table", createTableRoot.getDescription());
|
||||
Multimap<Long, Span> spansByParentIdMap = traceTree
|
||||
.getSpansByParentIdMap();
|
||||
|
||||
int startsWithHandlingCount = 0;
|
||||
|
||||
for (Span s : spansByParentIdMap.get(createTableRoot.getSpanId())) {
|
||||
if (s.getDescription().startsWith("handling")) {
|
||||
startsWithHandlingCount++;
|
||||
}
|
||||
}
|
||||
|
||||
assertTrue(startsWithHandlingCount > 3);
|
||||
assertTrue(spansByParentIdMap.get(createTableRoot.getSpanId()).size() > 3);
|
||||
assertTrue(spans.size() > 5);
|
||||
|
||||
Put put = new Put("row".getBytes());
|
||||
put.add(FAMILY_BYTES, "col".getBytes(), "value".getBytes());
|
||||
|
||||
Span putSpan = Trace.startSpan("doing put", Sampler.ALWAYS);
|
||||
try {
|
||||
table.put(put);
|
||||
} finally {
|
||||
putSpan.stop();
|
||||
}
|
||||
|
||||
spans = rcvr.getSpans();
|
||||
traceTree = new TraceTree(spans);
|
||||
roots = traceTree.getRoots();
|
||||
|
||||
assertEquals(2, roots.size());
|
||||
Span putRoot = null;
|
||||
for (Span root : roots) {
|
||||
if (root.getDescription().equals("doing put")) {
|
||||
putRoot = root;
|
||||
}
|
||||
}
|
||||
|
||||
assertNotNull(putRoot);
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue