You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2012/08/23 07:07:01 UTC

svn commit: r1376365 - in /hbase/trunk/hbase-server: ./ src/main/java/org/apache/hadoop/hbase/executor/ src/main/java/org/apache/hadoop/hbase/ipc/ src/main/java/org/apache/hadoop/hbase/master/ src/main/java/org/apache/hadoop/hbase/master/handler/ src/m...

Author: tedyu
Date: Thu Aug 23 05:07:00 2012
New Revision: 1376365

URL: http://svn.apache.org/viewvc?rev=1376365&view=rev
Log:
HBASE-6524 revert due to new test failures against hadoop 2.0


Removed:
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/Tracing.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/trace/HBaseLocalFileSpanReceiver.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/trace/SpanReceiverHost.java
    hbase/trunk/hbase-server/src/main/protobuf/Tracing.proto
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java
Modified:
    hbase/trunk/hbase-server/pom.xml
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java
    hbase/trunk/hbase-server/src/main/protobuf/RPC.proto

Modified: hbase/trunk/hbase-server/pom.xml
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/pom.xml?rev=1376365&r1=1376364&r2=1376365&view=diff
==============================================================================
--- hbase/trunk/hbase-server/pom.xml (original)
+++ hbase/trunk/hbase-server/pom.xml Thu Aug 23 05:07:00 2012
@@ -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 -->

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java?rev=1376365&r1=1376364&r2=1376365&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java Thu Aug 23 05:07:00 2012
@@ -27,9 +27,6 @@ import org.apache.commons.logging.LogFac
 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 imple
   // 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 imple
    * 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 imple
   }
 
   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();
     }
   }
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java?rev=1376365&r1=1376364&r2=1376365&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java Thu Aug 23 05:07:00 2012
@@ -63,7 +63,6 @@ import org.apache.hadoop.hbase.protobuf.
 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.UserGr
 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();

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java?rev=1376365&r1=1376364&r2=1376365&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java Thu Aug 23 05:07:00 2012
@@ -107,11 +107,6 @@ import com.google.common.util.concurrent
 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 implem
                                                   // 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 implem
       this.responder = responder;
       this.isError = false;
       this.size = size;
-      this.tinfo = tinfo;
     }
 
     @Override
@@ -1130,14 +1123,13 @@ public abstract class HBaseServer implem
     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 implem
         // 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 implem
       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 implem
       }
       // 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 implem
       } 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 implem
         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 implem
       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 implem
           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 implem
 
             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 implem
             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();

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1376365&r1=1376364&r2=1376365&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Thu Aug 23 05:07:00 2012
@@ -182,7 +182,6 @@ import org.apache.hadoop.metrics.util.MB
 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>
@@ -638,10 +636,7 @@ Server {
       // initialize master side coprocessors before we start handling requests
       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();

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java?rev=1376365&r1=1376364&r2=1376365&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java Thu Aug 23 05:07:00 2012
@@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.master.Ma
 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
       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);
           }
-        }));
+        });
       }
     }
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java?rev=1376365&r1=1376364&r2=1376365&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java Thu Aug 23 05:07:00 2012
@@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.master.HM
 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 
             continue;
           }
           final HRegionInfo hri = region;
-          pool.execute(Trace.wrap(new Runnable() {
+          pool.execute(new Runnable() {
             public void run() {
               assignmentManager.assign(hri, true);
             }
-          }));
+          });
         }
       } else {
         try {

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java?rev=1376365&r1=1376364&r2=1376365&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java Thu Aug 23 05:07:00 2012
@@ -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(" +

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java?rev=1376365&r1=1376364&r2=1376365&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java Thu Aug 23 05:07:00 2012
@@ -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);
   }
   

Modified: hbase/trunk/hbase-server/src/main/protobuf/RPC.proto
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/protobuf/RPC.proto?rev=1376365&r1=1376364&r2=1376365&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/protobuf/RPC.proto (original)
+++ hbase/trunk/hbase-server/src/main/protobuf/RPC.proto Thu Aug 23 05:07:00 2012
@@ -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