You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ec...@apache.org on 2012/07/05 21:54:59 UTC

svn commit: r1357842 [1/2] - in /accumulo/branches/ACCUMULO-672: ./ core/src/main/java/org/apache/accumulo/core/client/admin/ core/src/main/java/org/apache/accumulo/core/client/impl/ core/src/main/java/org/apache/accumulo/core/util/ server/src/main/jav...

Author: ecn
Date: Thu Jul  5 19:54:58 2012
New Revision: 1357842

URL: http://svn.apache.org/viewvc?rev=1357842&view=rev
Log:
ACCUMULO-672 TServiceClient is now an abstract class, which prevents us from using auto-created proxies to fake being one

Modified:
    accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
    accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
    accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
    accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
    accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java
    accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
    accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
    accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
    accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
    accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/util/ThriftUtil.java
    accumulo/branches/ACCUMULO-672/pom.xml
    accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
    accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/gc/GarbageCollectWriteAheadLogs.java
    accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java
    accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java
    accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/master/Master.java
    accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
    accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/master/tableOps/BulkImport.java
    accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/monitor/Monitor.java
    accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/monitor/servlets/TServersServlet.java
    accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
    accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/SplitReportMessage.java
    accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/TabletStatusMessage.java
    accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/test/GetMasterStats.java
    accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/test/WrongTabletTest.java
    accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousStatsCollector.java
    accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/test/functional/ZombieTServer.java
    accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/trace/TraceServer.java
    accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/util/Admin.java
    accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/util/TServerUtils.java
    accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
    accumulo/branches/ACCUMULO-672/trace/pom.xml
    accumulo/branches/ACCUMULO-672/trace/src/test/java/org/apache/accumulo/cloudtrace/instrument/TracerTest.java
    accumulo/branches/ACCUMULO-672/trace/thrift.sh

Modified: accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java (original)
+++ accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java Thu Jul  5 19:54:58 2012
@@ -21,6 +21,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.accumulo.cloudtrace.instrument.Tracer;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -35,7 +36,7 @@ import org.apache.accumulo.core.client.i
 import org.apache.accumulo.core.master.thrift.MasterClientService;
 import org.apache.accumulo.core.security.thrift.AuthInfo;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Iface;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
 import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
@@ -66,10 +67,10 @@ public class InstanceOperationsImpl impl
   @Override
   public void setProperty(final String property, final String value) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(property, value);
-    MasterClient.execute(instance, new ClientExec<MasterClientService.Iface>() {
+    MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
       @Override
-      public void execute(MasterClientService.Iface client) throws Exception {
-        client.setSystemProperty(null, credentials, property, value);
+      public void execute(MasterClientService.Client client) throws Exception {
+        client.setSystemProperty(Tracer.traceInfo(), credentials, property, value);
       }
     });
   }
@@ -80,10 +81,10 @@ public class InstanceOperationsImpl impl
   @Override
   public void removeProperty(final String property) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(property);
-    MasterClient.execute(instance, new ClientExec<MasterClientService.Iface>() {
+    MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
       @Override
-      public void execute(MasterClientService.Iface client) throws Exception {
-        client.removeSystemProperty(null, credentials, property);
+      public void execute(MasterClientService.Client client) throws Exception {
+        client.removeSystemProperty(Tracer.traceInfo(), credentials, property);
       }
     });
   }
@@ -93,9 +94,9 @@ public class InstanceOperationsImpl impl
    */
   @Override
   public Map<String,String> getSystemConfiguration() throws AccumuloException, AccumuloSecurityException {
-    return ServerClient.execute(instance, new ClientExecReturn<Map<String,String>,ClientService.Iface>() {
+    return ServerClient.execute(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
       @Override
-      public Map<String,String> execute(ClientService.Iface client) throws Exception {
+      public Map<String,String> execute(ClientService.Client client) throws Exception {
         return client.getConfiguration(ConfigurationType.CURRENT);
       }
     });
@@ -106,9 +107,9 @@ public class InstanceOperationsImpl impl
    */
   @Override
   public Map<String,String> getSiteConfiguration() throws AccumuloException, AccumuloSecurityException {
-    return ServerClient.execute(instance, new ClientExecReturn<Map<String,String>,ClientService.Iface>() {
+    return ServerClient.execute(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
       @Override
-      public Map<String,String> execute(ClientService.Iface client) throws Exception {
+      public Map<String,String> execute(ClientService.Client client) throws Exception {
         return client.getConfiguration(ConfigurationType.SITE);
       }
     });
@@ -144,10 +145,10 @@ public class InstanceOperationsImpl impl
   @Override
   public List<ActiveScan> getActiveScans(String tserver) throws AccumuloException, AccumuloSecurityException {
     List<org.apache.accumulo.core.tabletserver.thrift.ActiveScan> tas = ThriftUtil.execute(tserver, instance.getConfiguration(),
-        new ClientExecReturn<List<org.apache.accumulo.core.tabletserver.thrift.ActiveScan>,TabletClientService.Iface>() {
+        new ClientExecReturn<List<org.apache.accumulo.core.tabletserver.thrift.ActiveScan>,TabletClientService.Client>() {
           @Override
-          public List<org.apache.accumulo.core.tabletserver.thrift.ActiveScan> execute(Iface client) throws Exception {
-            return client.getActiveScans(null, credentials);
+          public List<org.apache.accumulo.core.tabletserver.thrift.ActiveScan> execute(TabletClientService.Client client) throws Exception {
+            return client.getActiveScans(Tracer.traceInfo(), credentials);
           }
         });
     List<ActiveScan> as = new ArrayList<ActiveScan>();
@@ -166,10 +167,10 @@ public class InstanceOperationsImpl impl
    */
   @Override
   public boolean testClassLoad(final String className, final String asTypeName) throws AccumuloException, AccumuloSecurityException {
-    return ServerClient.execute(instance, new ClientExecReturn<Boolean,ClientService.Iface>() {
+    return ServerClient.execute(instance, new ClientExecReturn<Boolean,ClientService.Client>() {
       @Override
-      public Boolean execute(ClientService.Iface client) throws Exception {
-        return client.checkClass(null, className, asTypeName);
+      public Boolean execute(ClientService.Client client) throws Exception {
+        return client.checkClass(Tracer.traceInfo(), className, asTypeName);
       }
     });
   }

Modified: accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java (original)
+++ accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java Thu Jul  5 19:54:58 2012
@@ -20,6 +20,7 @@ import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Set;
 
+import org.apache.accumulo.cloudtrace.instrument.Tracer;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
@@ -43,7 +44,7 @@ public class SecurityOperationsImpl impl
   private Instance instance;
   private AuthInfo credentials;
   
-  private void execute(ClientExec<ClientService.Iface> exec) throws AccumuloException, AccumuloSecurityException {
+  private void execute(ClientExec<ClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
     try {
       ServerClient.executeRaw(instance, exec);
     } catch (ThriftTableOperationException ttoe) {
@@ -61,7 +62,7 @@ public class SecurityOperationsImpl impl
     }
   }
   
-  private <T> T execute(ClientExecReturn<T,ClientService.Iface> exec) throws AccumuloException, AccumuloSecurityException {
+  private <T> T execute(ClientExecReturn<T,ClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
     try {
       return ServerClient.executeRaw(instance, exec);
     } catch (ThriftTableOperationException ttoe) {
@@ -107,10 +108,10 @@ public class SecurityOperationsImpl impl
    */
   public void createUser(final String user, final byte[] password, final Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(user, password, authorizations);
-    execute(new ClientExec<ClientService.Iface>() {
+    execute(new ClientExec<ClientService.Client>() {
       @Override
-      public void execute(ClientService.Iface client) throws Exception {
-        client.createUser(null, credentials, user, ByteBuffer.wrap(password), ByteBufferUtil.toByteBuffers(authorizations.getAuthorizations()));
+      public void execute(ClientService.Client client) throws Exception {
+        client.createUser(Tracer.traceInfo(), credentials, user, ByteBuffer.wrap(password), ByteBufferUtil.toByteBuffers(authorizations.getAuthorizations()));
       }
     });
   }
@@ -127,10 +128,10 @@ public class SecurityOperationsImpl impl
    */
   public void dropUser(final String user) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(user);
-    execute(new ClientExec<ClientService.Iface>() {
+    execute(new ClientExec<ClientService.Client>() {
       @Override
-      public void execute(ClientService.Iface client) throws Exception {
-        client.dropUser(null, credentials, user);
+      public void execute(ClientService.Client client) throws Exception {
+        client.dropUser(Tracer.traceInfo(), credentials, user);
       }
     });
   }
@@ -150,10 +151,10 @@ public class SecurityOperationsImpl impl
    */
   public boolean authenticateUser(final String user, final byte[] password) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(user, password);
-    return execute(new ClientExecReturn<Boolean,ClientService.Iface>() {
+    return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
       @Override
-      public Boolean execute(ClientService.Iface client) throws Exception {
-        return client.authenticateUser(null, credentials, user, ByteBuffer.wrap(password));
+      public Boolean execute(ClientService.Client client) throws Exception {
+        return client.authenticateUser(Tracer.traceInfo(), credentials, user, ByteBuffer.wrap(password));
       }
     });
   }
@@ -172,10 +173,10 @@ public class SecurityOperationsImpl impl
    */
   public void changeUserPassword(final String user, final byte[] password) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(user, password);
-    execute(new ClientExec<ClientService.Iface>() {
+    execute(new ClientExec<ClientService.Client>() {
       @Override
-      public void execute(ClientService.Iface client) throws Exception {
-        client.changePassword(null, credentials, user, ByteBuffer.wrap(password));
+      public void execute(ClientService.Client client) throws Exception {
+        client.changePassword(Tracer.traceInfo(), credentials, user, ByteBuffer.wrap(password));
       }
     });
     if (this.credentials.user.equals(user)) {
@@ -197,10 +198,10 @@ public class SecurityOperationsImpl impl
    */
   public void changeUserAuthorizations(final String user, final Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(user, authorizations);
-    execute(new ClientExec<ClientService.Iface>() {
+    execute(new ClientExec<ClientService.Client>() {
       @Override
-      public void execute(ClientService.Iface client) throws Exception {
-        client.changeAuthorizations(null, credentials, user, ByteBufferUtil.toByteBuffers(authorizations.getAuthorizations()));
+      public void execute(ClientService.Client client) throws Exception {
+        client.changeAuthorizations(Tracer.traceInfo(), credentials, user, ByteBufferUtil.toByteBuffers(authorizations.getAuthorizations()));
       }
     });
   }
@@ -218,10 +219,10 @@ public class SecurityOperationsImpl impl
    */
   public Authorizations getUserAuthorizations(final String user) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(user);
-    return execute(new ClientExecReturn<Authorizations,ClientService.Iface>() {
+    return execute(new ClientExecReturn<Authorizations,ClientService.Client>() {
       @Override
-      public Authorizations execute(ClientService.Iface client) throws Exception {
-        return new Authorizations(client.getUserAuthorizations(null, credentials, user));
+      public Authorizations execute(ClientService.Client client) throws Exception {
+        return new Authorizations(client.getUserAuthorizations(Tracer.traceInfo(), credentials, user));
       }
     });
   }
@@ -241,10 +242,10 @@ public class SecurityOperationsImpl impl
    */
   public boolean hasSystemPermission(final String user, final SystemPermission perm) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(user, perm);
-    return execute(new ClientExecReturn<Boolean,ClientService.Iface>() {
+    return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
       @Override
-      public Boolean execute(ClientService.Iface client) throws Exception {
-        return client.hasSystemPermission(null, credentials, user, perm.getId());
+      public Boolean execute(ClientService.Client client) throws Exception {
+        return client.hasSystemPermission(Tracer.traceInfo(), credentials, user, perm.getId());
       }
     });
   }
@@ -266,10 +267,10 @@ public class SecurityOperationsImpl impl
    */
   public boolean hasTablePermission(final String user, final String table, final TablePermission perm) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(user, table, perm);
-    return execute(new ClientExecReturn<Boolean,ClientService.Iface>() {
+    return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
       @Override
-      public Boolean execute(ClientService.Iface client) throws Exception {
-        return client.hasTablePermission(null, credentials, user, table, perm.getId());
+      public Boolean execute(ClientService.Client client) throws Exception {
+        return client.hasTablePermission(Tracer.traceInfo(), credentials, user, table, perm.getId());
       }
     });
   }
@@ -288,10 +289,10 @@ public class SecurityOperationsImpl impl
    */
   public void grantSystemPermission(final String user, final SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(user, permission);
-    execute(new ClientExec<ClientService.Iface>() {
+    execute(new ClientExec<ClientService.Client>() {
       @Override
-      public void execute(ClientService.Iface client) throws Exception {
-        client.grantSystemPermission(null, credentials, user, permission.getId());
+      public void execute(ClientService.Client client) throws Exception {
+        client.grantSystemPermission(Tracer.traceInfo(), credentials, user, permission.getId());
       }
     });
   }
@@ -312,10 +313,10 @@ public class SecurityOperationsImpl impl
    */
   public void grantTablePermission(final String user, final String table, final TablePermission permission) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(user, table, permission);
-    execute(new ClientExec<ClientService.Iface>() {
+    execute(new ClientExec<ClientService.Client>() {
       @Override
-      public void execute(ClientService.Iface client) throws Exception {
-        client.grantTablePermission(null, credentials, user, table, permission.getId());
+      public void execute(ClientService.Client client) throws Exception {
+        client.grantTablePermission(Tracer.traceInfo(), credentials, user, table, permission.getId());
       }
     });
   }
@@ -334,10 +335,10 @@ public class SecurityOperationsImpl impl
    */
   public void revokeSystemPermission(final String user, final SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(user, permission);
-    execute(new ClientExec<ClientService.Iface>() {
+    execute(new ClientExec<ClientService.Client>() {
       @Override
-      public void execute(ClientService.Iface client) throws Exception {
-        client.revokeSystemPermission(null, credentials, user, permission.getId());
+      public void execute(ClientService.Client client) throws Exception {
+        client.revokeSystemPermission(Tracer.traceInfo(), credentials, user, permission.getId());
       }
     });
   }
@@ -359,10 +360,10 @@ public class SecurityOperationsImpl impl
   public void revokeTablePermission(final String user, final String table, final TablePermission permission) throws AccumuloException,
       AccumuloSecurityException {
     ArgumentChecker.notNull(user, table, permission);
-    execute(new ClientExec<ClientService.Iface>() {
+    execute(new ClientExec<ClientService.Client>() {
       @Override
-      public void execute(ClientService.Iface client) throws Exception {
-        client.revokeTablePermission(null, credentials, user, table, permission.getId());
+      public void execute(ClientService.Client client) throws Exception {
+        client.revokeTablePermission(Tracer.traceInfo(), credentials, user, table, permission.getId());
       }
     });
   }
@@ -377,10 +378,10 @@ public class SecurityOperationsImpl impl
    *           if the user does not have permission to query users
    */
   public Set<String> listUsers() throws AccumuloException, AccumuloSecurityException {
-    return execute(new ClientExecReturn<Set<String>,ClientService.Iface>() {
+    return execute(new ClientExecReturn<Set<String>,ClientService.Client>() {
       @Override
-      public Set<String> execute(ClientService.Iface client) throws Exception {
-        return client.listUsers(null, credentials);
+      public Set<String> execute(ClientService.Client client) throws Exception {
+        return client.listUsers(Tracer.traceInfo(), credentials);
       }
     });
   }

Modified: accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java (original)
+++ accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java Thu Jul  5 19:54:58 2012
@@ -38,6 +38,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
+import org.apache.accumulo.cloudtrace.instrument.Tracer;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -91,7 +92,6 @@ import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.apache.thrift.TApplicationException;
 import org.apache.thrift.TException;
-import org.apache.thrift.TServiceClient;
 import org.apache.thrift.transport.TTransportException;
 
 /**
@@ -204,7 +204,7 @@ public class TableOperationsImpl extends
       MasterClientService.Iface client = null;
       try {
         client = MasterClient.getConnectionWithRetry(instance);
-        return client.beginTableOperation(null, credentials);
+        return client.beginTableOperation(Tracer.traceInfo(), credentials);
       } catch (TTransportException tte) {
         log.debug("Failed to call beginTableOperation(), retrying ... ", tte);
         UtilWaitThread.sleep(100);
@@ -220,7 +220,7 @@ public class TableOperationsImpl extends
       MasterClientService.Iface client = null;
       try {
         client = MasterClient.getConnectionWithRetry(instance);
-        client.executeTableOperation(null, credentials, opid, op, args, opts, autoCleanUp);
+        client.executeTableOperation(Tracer.traceInfo(), credentials, opid, op, args, opts, autoCleanUp);
         break;
       } catch (TTransportException tte) {
         log.debug("Failed to call executeTableOperation(), retrying ... ", tte);
@@ -236,7 +236,7 @@ public class TableOperationsImpl extends
       MasterClientService.Iface client = null;
       try {
         client = MasterClient.getConnectionWithRetry(instance);
-        return client.waitForTableOperation(null, credentials, opid);
+        return client.waitForTableOperation(Tracer.traceInfo(), credentials, opid);
       } catch (TTransportException tte) {
         log.debug("Failed to call waitForTableOperation(), retrying ... ", tte);
         UtilWaitThread.sleep(100);
@@ -251,7 +251,7 @@ public class TableOperationsImpl extends
       MasterClientService.Iface client = null;
       try {
         client = MasterClient.getConnectionWithRetry(instance);
-        client.finishTableOperation(null, credentials, opid);
+        client.finishTableOperation(Tracer.traceInfo(), credentials, opid);
         break;
       } catch (TTransportException tte) {
         log.debug("Failed to call finishTableOperation(), retrying ... ", tte);
@@ -440,13 +440,13 @@ public class TableOperationsImpl extends
         }
         
         try {
-          TabletClientService.Iface client = ThriftUtil.getTServerClient(tl.tablet_location, instance.getConfiguration());
+          TabletClientService.Client client = ThriftUtil.getTServerClient(tl.tablet_location, instance.getConfiguration());
           try {
             OpTimer opTimer = null;
             if (log.isTraceEnabled())
               opTimer = new OpTimer(log, Level.TRACE).start("Splitting tablet " + tl.tablet_extent + " on " + tl.tablet_location + " at " + split);
             
-            client.splitTablet(null, credentials, tl.tablet_extent.toThrift(), TextUtil.getByteBuffer(split));
+            client.splitTablet(Tracer.traceInfo(), credentials, tl.tablet_extent.toThrift(), TextUtil.getByteBuffer(split));
             
             // just split it, might as well invalidate it in the cache
             tabLocator.invalidateCache(tl.tablet_extent);
@@ -454,7 +454,7 @@ public class TableOperationsImpl extends
             if (opTimer != null)
               opTimer.stop("Split tablet in %DURATION%");
           } finally {
-            ThriftUtil.returnClient((TServiceClient) client);
+            ThriftUtil.returnClient(client);
           }
           
         } catch (TApplicationException tae) {
@@ -723,7 +723,7 @@ public class TableOperationsImpl extends
         MasterClientService.Iface client = null;
         try {
           client = MasterClient.getConnectionWithRetry(instance);
-          flushID = client.initiateFlush(null, credentials, tableId);
+          flushID = client.initiateFlush(Tracer.traceInfo(), credentials, tableId);
           break;
         } catch (TTransportException tte) {
           log.debug("Failed to call initiateFlush, retrying ... ", tte);
@@ -737,7 +737,7 @@ public class TableOperationsImpl extends
         MasterClientService.Iface client = null;
         try {
           client = MasterClient.getConnectionWithRetry(instance);
-          client.waitForFlush(null, credentials, tableId, TextUtil.getByteBuffer(start), TextUtil.getByteBuffer(end), flushID, wait ? Long.MAX_VALUE : 1);
+          client.waitForFlush(Tracer.traceInfo(), credentials, tableId, TextUtil.getByteBuffer(start), TextUtil.getByteBuffer(end), flushID, wait ? Long.MAX_VALUE : 1);
           break;
         } catch (TTransportException tte) {
           log.debug("Failed to call initiateFlush, retrying ... ", tte);
@@ -778,10 +778,10 @@ public class TableOperationsImpl extends
    */
   public void setProperty(final String tableName, final String property, final String value) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(tableName, property, value);
-    MasterClient.execute(instance, new ClientExec<MasterClientService.Iface>() {
+    MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
       @Override
-      public void execute(MasterClientService.Iface client) throws Exception {
-        client.setTableProperty(null, credentials, tableName, property, value);
+      public void execute(MasterClientService.Client client) throws Exception {
+        client.setTableProperty(Tracer.traceInfo(), credentials, tableName, property, value);
       }
     });
   }
@@ -800,10 +800,10 @@ public class TableOperationsImpl extends
    */
   public void removeProperty(final String tableName, final String property) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(tableName, property);
-    MasterClient.execute(instance, new ClientExec<MasterClientService.Iface>() {
+    MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
       @Override
-      public void execute(MasterClientService.Iface client) throws Exception {
-        client.removeTableProperty(null, credentials, tableName, property);
+      public void execute(MasterClientService.Client client) throws Exception {
+        client.removeTableProperty(Tracer.traceInfo(), credentials, tableName, property);
       }
     });
   }
@@ -820,9 +820,9 @@ public class TableOperationsImpl extends
   public Iterable<Entry<String,String>> getProperties(final String tableName) throws AccumuloException, TableNotFoundException {
     ArgumentChecker.notNull(tableName);
     try {
-      return ServerClient.executeRaw(instance, new ClientExecReturn<Map<String,String>,ClientService.Iface>() {
+      return ServerClient.executeRaw(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
         @Override
-        public Map<String,String> execute(ClientService.Iface client) throws Exception {
+        public Map<String,String> execute(ClientService.Client client) throws Exception {
           return client.getTableConfiguration(tableName);
         }
       }).entrySet();

Modified: accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java (original)
+++ accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java Thu Jul  5 19:54:58 2012
@@ -18,6 +18,7 @@ package org.apache.accumulo.core.client.
 
 import java.nio.ByteBuffer;
 
+import org.apache.accumulo.cloudtrace.instrument.Tracer;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchDeleter;
@@ -72,10 +73,10 @@ public class ConnectorImpl extends Conne
     // hardcoded string for SYSTEM user since the definition is
     // in server code
     if (!user.equals("!SYSTEM")) {
-      ServerClient.execute(instance, new ClientExec<ClientService.Iface>() {
+      ServerClient.execute(instance, new ClientExec<ClientService.Client>() {
         @Override
-        public void execute(ClientService.Iface iface) throws Exception {
-          iface.authenticateUser(null, credentials, credentials.user, credentials.password);
+        public void execute(ClientService.Client iface) throws Exception {
+          iface.authenticateUser(Tracer.traceInfo(), credentials, credentials.user, credentials.password);
         }
       });
     }

Modified: accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java (original)
+++ accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java Thu Jul  5 19:54:58 2012
@@ -35,12 +35,12 @@ import org.apache.thrift.transport.TTran
 public class MasterClient {
   private static final Logger log = Logger.getLogger(MasterClient.class);
   
-  public static MasterClientService.Iface getConnectionWithRetry(Instance instance) {
+  public static MasterClientService.Client getConnectionWithRetry(Instance instance) {
     ArgumentChecker.notNull(instance);
     
     while (true) {
       
-      MasterClientService.Iface result = getConnection(instance);
+      MasterClientService.Client result = getConnection(instance);
       if (result != null)
         return result;
       UtilWaitThread.sleep(250);
@@ -48,7 +48,7 @@ public class MasterClient {
     
   }
   
-  public static MasterClientService.Iface getConnection(Instance instance) {
+  public static MasterClientService.Client getConnection(Instance instance) {
     List<String> locations = instance.getMasterLocations();
     
     if (locations.size() == 0) {
@@ -61,7 +61,7 @@ public class MasterClient {
     
     try {
       // Master requests can take a long time: don't ever time out
-      MasterClientService.Iface client = ThriftUtil.getClient(new MasterClientService.Client.Factory(), master, Property.MASTER_CLIENTPORT,
+      MasterClientService.Client client = ThriftUtil.getClient(new MasterClientService.Client.Factory(), master, Property.MASTER_CLIENTPORT,
           instance.getConfiguration());
       return client;
     } catch (TTransportException tte) {
@@ -83,8 +83,8 @@ public class MasterClient {
     }
   }
   
-  public static <T> T execute(Instance instance, ClientExecReturn<T,MasterClientService.Iface> exec) throws AccumuloException, AccumuloSecurityException {
-    MasterClientService.Iface client = null;
+  public static <T> T execute(Instance instance, ClientExecReturn<T,MasterClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
+    MasterClientService.Client client = null;
     while (true) {
       try {
         client = getConnectionWithRetry(instance);
@@ -105,8 +105,8 @@ public class MasterClient {
     }
   }
   
-  public static void execute(Instance instance, ClientExec<MasterClientService.Iface> exec) throws AccumuloException, AccumuloSecurityException {
-    MasterClientService.Iface client = null;
+  public static void execute(Instance instance, ClientExec<MasterClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
+    MasterClientService.Client client = null;
     while (true) {
       try {
         client = getConnectionWithRetry(instance);

Modified: accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java (original)
+++ accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java Thu Jul  5 19:54:58 2012
@@ -25,7 +25,7 @@ import org.apache.accumulo.core.client.A
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.thrift.ClientService;
-import org.apache.accumulo.core.client.impl.thrift.ClientService.Iface;
+import org.apache.accumulo.core.client.impl.thrift.ClientService.Client;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.util.ArgumentChecker;
@@ -37,7 +37,6 @@ import org.apache.accumulo.core.util.Uti
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.log4j.Logger;
-import org.apache.thrift.TServiceClient;
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
 
@@ -54,7 +53,7 @@ public class ServerClient {
     return result;
   }
   
-  public static <T> T execute(Instance instance, ClientExecReturn<T,ClientService.Iface> exec) throws AccumuloException, AccumuloSecurityException {
+  public static <T> T execute(Instance instance, ClientExecReturn<T,ClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
     try {
       return executeRaw(instance, exec);
     } catch (ThriftSecurityException e) {
@@ -66,7 +65,7 @@ public class ServerClient {
     }
   }
   
-  public static void execute(Instance instance, ClientExec<ClientService.Iface> exec) throws AccumuloException, AccumuloSecurityException {
+  public static void execute(Instance instance, ClientExec<ClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
     try {
       executeRaw(instance, exec);
     } catch (ThriftSecurityException e) {
@@ -78,12 +77,12 @@ public class ServerClient {
     }
   }
   
-  public static <T> T executeRaw(Instance instance, ClientExecReturn<T,ClientService.Iface> exec) throws Exception {
+  public static <T> T executeRaw(Instance instance, ClientExecReturn<T,ClientService.Client> exec) throws Exception {
     while (true) {
-      ClientService.Iface client = null;
+      ClientService.Client client = null;
       String server = null;
       try {
-        Pair<String,Iface> pair = ServerClient.getConnection(instance);
+        Pair<String,Client> pair = ServerClient.getConnection(instance);
         server = pair.getFirst();
         client = pair.getSecond();
         return exec.execute(client);
@@ -97,12 +96,12 @@ public class ServerClient {
     }
   }
   
-  public static void executeRaw(Instance instance, ClientExec<ClientService.Iface> exec) throws Exception {
+  public static void executeRaw(Instance instance, ClientExec<ClientService.Client> exec) throws Exception {
     while (true) {
-      ClientService.Iface client = null;
+      ClientService.Client client = null;
       String server = null;
       try {
-        Pair<String,Iface> pair = ServerClient.getConnection(instance);
+        Pair<String,Client> pair = ServerClient.getConnection(instance);
         server = pair.getFirst();
         client = pair.getSecond();
         exec.execute(client);
@@ -119,11 +118,11 @@ public class ServerClient {
   
   static volatile boolean warnedAboutTServersBeingDown = false;
 
-  public static Pair<String,ClientService.Iface> getConnection(Instance instance) throws TTransportException {
+  public static Pair<String,ClientService.Client> getConnection(Instance instance) throws TTransportException {
     return getConnection(instance, true);
   }
   
-  public static Pair<String,ClientService.Iface> getConnection(Instance instance, boolean preferCachedConnections) throws TTransportException {
+  public static Pair<String,ClientService.Client> getConnection(Instance instance, boolean preferCachedConnections) throws TTransportException {
     ArgumentChecker.notNull(instance);
     // create list of servers
     ArrayList<ThriftTransportKey> servers = new ArrayList<ThriftTransportKey>();
@@ -143,10 +142,10 @@ public class ServerClient {
     boolean opened = false;
     try {
       Pair<String,TTransport> pair = ThriftTransportPool.getInstance().getAnyTransport(servers, preferCachedConnections);
-      ClientService.Iface client = ThriftUtil.createClient(new ClientService.Client.Factory(), pair.getSecond());
+      ClientService.Client client = ThriftUtil.createClient(new ClientService.Client.Factory(), pair.getSecond());
       opened = true;
       warnedAboutTServersBeingDown = false;
-      return new Pair<String,ClientService.Iface>(pair.getFirst(), client);
+      return new Pair<String,ClientService.Client>(pair.getFirst(), client);
     } finally {
       if (!opened) {
         if (!warnedAboutTServersBeingDown) {
@@ -161,8 +160,7 @@ public class ServerClient {
     }
   }
   
-  public static void close(ClientService.Iface iface) {
-    TServiceClient client = (TServiceClient) iface;
+  public static void close(ClientService.Client client) {
     if (client != null && client.getInputProtocol() != null && client.getInputProtocol().getTransport() != null) {
       ThriftTransportPool.getInstance().returnTransport(client.getInputProtocol().getTransport());
     } else {

Modified: accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java (original)
+++ accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java Thu Jul  5 19:54:58 2012
@@ -38,6 +38,8 @@ import java.util.concurrent.atomic.Atomi
 
 import org.apache.accumulo.cloudtrace.instrument.Span;
 import org.apache.accumulo.cloudtrace.instrument.Trace;
+import org.apache.accumulo.cloudtrace.instrument.Tracer;
+import org.apache.accumulo.cloudtrace.thrift.TInfo;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -747,7 +749,7 @@ public class TabletServerBatchWriter {
       if (tabMuts.size() == 0) {
         return new MutationSet();
       }
-      
+      TInfo tinfo = Tracer.traceInfo();
       TTransport transport = null;
       
       try {
@@ -759,7 +761,7 @@ public class TabletServerBatchWriter {
             Entry<KeyExtent,List<Mutation>> entry = tabMuts.entrySet().iterator().next();
             
             try {
-              client.update(null, credentials, entry.getKey().toThrift(), entry.getValue().get(0).toThrift());
+              client.update(tinfo, credentials, entry.getKey().toThrift(), entry.getValue().get(0).toThrift());
             } catch (NotServingTabletException e) {
               allFailures.addAll(entry.getKey().getTableId().toString(), entry.getValue());
               TabletLocator.getInstance(instance, credentials, new Text(entry.getKey().getTableId())).invalidateCache(entry.getKey());
@@ -768,7 +770,7 @@ public class TabletServerBatchWriter {
             }
           } else {
             
-            long usid = client.startUpdate(null, credentials);
+            long usid = client.startUpdate(tinfo, credentials);
             
             List<TMutation> updates = new ArrayList<TMutation>();
             for (Entry<KeyExtent,List<Mutation>> entry : tabMuts.entrySet()) {
@@ -781,13 +783,13 @@ public class TabletServerBatchWriter {
                   size += mutation.numBytes();
                 }
                 
-                client.applyUpdates(null, usid, entry.getKey().toThrift(), updates);
+                client.applyUpdates(tinfo, usid, entry.getKey().toThrift(), updates);
                 updates.clear();
                 size = 0;
               }
             }
             
-            UpdateErrors updateErrors = client.closeUpdate(null, usid);
+            UpdateErrors updateErrors = client.closeUpdate(tinfo, usid);
             Map<KeyExtent,Long> failures = Translator.translate(updateErrors.failedExtents, Translator.TKET);
             updatedConstraintViolations(Translator.translate(updateErrors.violationSummaries, Translator.TCVST));
             updateAuthorizationFailures(Translator.translate(updateErrors.authorizationFailures, Translator.TKET));

Modified: accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java (original)
+++ accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java Thu Jul  5 19:54:58 2012
@@ -29,6 +29,8 @@ import java.util.SortedSet;
 
 import org.apache.accumulo.cloudtrace.instrument.Span;
 import org.apache.accumulo.cloudtrace.instrument.Trace;
+import org.apache.accumulo.cloudtrace.instrument.Tracer;
+import org.apache.accumulo.cloudtrace.thrift.TInfo;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
@@ -116,7 +118,8 @@ public class ThriftScanner {
       throw new AccumuloException(new IOException());
     
     try {
-      TabletClientService.Iface client = ThriftUtil.getTServerClient(server, conf);
+      TInfo tinfo = Tracer.traceInfo();
+      TabletClientService.Client client = ThriftUtil.getTServerClient(server, conf);
       try {
         List<IterInfo> emptyList = Collections.emptyList();
         Map<String,Map<String,String>> emptyMap = Collections.emptyMap();
@@ -125,7 +128,7 @@ public class ThriftScanner {
         
         TabletType ttype = TabletType.type(extent);
         boolean waitForWrites = !serversWaitedForWrites.get(ttype).contains(server);
-        InitialScan isr = client.startScan(null, scanState.credentials, extent.toThrift(), scanState.range.toThrift(),
+        InitialScan isr = client.startScan(tinfo, scanState.credentials, extent.toThrift(), scanState.range.toThrift(),
             Translator.translate(scanState.columns, Translator.CT), scanState.size, scanState.serverSideIteratorList, scanState.serverSideIteratorOptions,
             scanState.authorizations.getAuthorizationsBB(), waitForWrites, scanState.isolated);
         if (waitForWrites)
@@ -136,7 +139,7 @@ public class ThriftScanner {
         for (TKeyValue kv : isr.result.results)
           results.put(new Key(kv.key), new Value(kv.value));
         
-        client.closeScan(null, isr.scanID);
+        client.closeScan(tinfo, isr.scanID);
         
         return isr.result.more;
       } finally {
@@ -393,7 +396,8 @@ public class ThriftScanner {
     
     OpTimer opTimer = new OpTimer(log, Level.TRACE);
     
-    TabletClientService.Iface client = ThriftUtil.getTServerClient(loc.tablet_location, conf);
+    TInfo tinfo = Tracer.traceInfo();
+    TabletClientService.Client client = ThriftUtil.getTServerClient(loc.tablet_location, conf);
     
     String old = Thread.currentThread().getName();
     try {
@@ -412,7 +416,7 @@ public class ThriftScanner {
         
         TabletType ttype = TabletType.type(loc.tablet_extent);
         boolean waitForWrites = !serversWaitedForWrites.get(ttype).contains(loc.tablet_location);
-        InitialScan is = client.startScan(null, scanState.credentials, loc.tablet_extent.toThrift(), scanState.range.toThrift(),
+        InitialScan is = client.startScan(tinfo, scanState.credentials, loc.tablet_extent.toThrift(), scanState.range.toThrift(),
             Translator.translate(scanState.columns, Translator.CT), scanState.size, scanState.serverSideIteratorList, scanState.serverSideIteratorOptions,
             scanState.authorizations.getAuthorizationsBB(), waitForWrites, scanState.isolated);
         if (waitForWrites)
@@ -423,7 +427,7 @@ public class ThriftScanner {
         if (sr.more)
           scanState.scanID = is.scanID;
         else
-          client.closeScan(null, is.scanID);
+          client.closeScan(tinfo, is.scanID);
         
       } else {
         // log.debug("Calling continue scan : "+scanState.range+"  loc = "+loc);
@@ -431,9 +435,9 @@ public class ThriftScanner {
         Thread.currentThread().setName(msg);
         opTimer.start(msg);
         
-        sr = client.continueScan(null, scanState.scanID);
+        sr = client.continueScan(tinfo, scanState.scanID);
         if (!sr.more) {
-          client.closeScan(null, scanState.scanID);
+          client.closeScan(tinfo, scanState.scanID);
           scanState.scanID = null;
         }
       }

Modified: accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java (original)
+++ accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java Thu Jul  5 19:54:58 2012
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.core.client.impl;
 
+import org.apache.accumulo.cloudtrace.instrument.Tracer;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
@@ -64,7 +65,7 @@ public class Writer {
     TabletClientService.Iface client = null;
     try {
       client = ThriftUtil.getTServerClient(server, configuration);
-      client.update(null, ai, extent.toThrift(), m.toThrift());
+      client.update(Tracer.traceInfo(), ai, extent.toThrift(), m.toThrift());
       return;
     } catch (ThriftSecurityException e) {
       throw new AccumuloSecurityException(e.user, e.code);

Modified: accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/util/ThriftUtil.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/util/ThriftUtil.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/util/ThriftUtil.java (original)
+++ accumulo/branches/ACCUMULO-672/core/src/main/java/org/apache/accumulo/core/util/ThriftUtil.java Thu Jul  5 19:54:58 2012
@@ -18,7 +18,8 @@ package org.apache.accumulo.core.util;
 
 import java.net.InetSocketAddress;
 
-import org.apache.accumulo.cloudtrace.instrument.thrift.TraceWrap;
+import org.apache.accumulo.cloudtrace.instrument.Span;
+import org.apache.accumulo.cloudtrace.instrument.Trace;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.impl.ClientExec;
@@ -29,9 +30,12 @@ import org.apache.accumulo.core.conf.Pro
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.log4j.Logger;
+import org.apache.thrift.TException;
 import org.apache.thrift.TServiceClient;
 import org.apache.thrift.TServiceClientFactory;
 import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.protocol.TMessage;
+import org.apache.thrift.protocol.TProtocol;
 import org.apache.thrift.protocol.TProtocolFactory;
 import org.apache.thrift.transport.TFramedTransport;
 import org.apache.thrift.transport.TTransport;
@@ -41,12 +45,42 @@ import org.apache.thrift.transport.TTran
 
 public class ThriftUtil {
   private static final Logger log = Logger.getLogger(ThriftUtil.class);
+
+  public static class TraceProtocol extends TCompactProtocol {
+
+    @Override
+    public void writeMessageBegin(TMessage message) throws TException {
+      Trace.start("client:" + message.name);
+      super.writeMessageBegin(message);
+    }
+
+    @Override
+    public void writeMessageEnd() throws TException {
+      super.writeMessageEnd();
+      Span currentTrace = Trace.currentTrace();
+      if (currentTrace != null)
+        currentTrace.stop();
+    }
+
+    public TraceProtocol(TTransport transport) {
+      super(transport);
+    }
+  }
+  
+  public static class TraceProtocolFactory extends TCompactProtocol.Factory {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public TProtocol getProtocol(TTransport trans) {
+      return new TraceProtocol(trans);
+    }
+  }
   
-  static private TProtocolFactory protocolFactory = new TCompactProtocol.Factory();
+  static private TProtocolFactory protocolFactory = new TraceProtocolFactory();
   static private TTransportFactory transportFactory = new TFramedTransport.Factory();
   
   static public <T extends TServiceClient> T createClient(TServiceClientFactory<T> factory, TTransport transport) {
-    return TraceWrap.client(factory.getClient(protocolFactory.getProtocol(transport), protocolFactory.getProtocol(transport)));
+    return factory.getClient(protocolFactory.getProtocol(transport), protocolFactory.getProtocol(transport));
   }
   
   static public <T extends TServiceClient> T getClient(TServiceClientFactory<T> factory, InetSocketAddress address, AccumuloConfiguration conf)
@@ -68,21 +102,20 @@ public class ThriftUtil {
     return createClient(factory, transport);
   }
   
-  static public void returnClient(Object iface) { // Eew... the typing here is horrible
+  static public void returnClient(TServiceClient iface) { // Eew... the typing here is horrible
     if (iface != null) {
-      TServiceClient client = (TServiceClient) iface;
-      ThriftTransportPool.getInstance().returnTransport(client.getInputProtocol().getTransport());
+      ThriftTransportPool.getInstance().returnTransport(iface.getInputProtocol().getTransport());
     }
   }
   
-  static public TabletClientService.Iface getTServerClient(String address, AccumuloConfiguration conf) throws TTransportException {
+  static public TabletClientService.Client getTServerClient(String address, AccumuloConfiguration conf) throws TTransportException {
     return getClient(new TabletClientService.Client.Factory(), address, Property.TSERV_CLIENTPORT, Property.GENERAL_RPC_TIMEOUT, conf);
   }
   
-  public static void execute(String address, AccumuloConfiguration conf, ClientExec<TabletClientService.Iface> exec) throws AccumuloException,
+  public static void execute(String address, AccumuloConfiguration conf, ClientExec<TabletClientService.Client> exec) throws AccumuloException,
       AccumuloSecurityException {
     while (true) {
-      TabletClientService.Iface client = null;
+      TabletClientService.Client client = null;
       try {
         exec.execute(client = getTServerClient(address, conf));
         break;
@@ -100,10 +133,10 @@ public class ThriftUtil {
     }
   }
   
-  public static <T> T execute(String address, AccumuloConfiguration conf, ClientExecReturn<T,TabletClientService.Iface> exec) throws AccumuloException,
+  public static <T> T execute(String address, AccumuloConfiguration conf, ClientExecReturn<T,TabletClientService.Client> exec) throws AccumuloException,
       AccumuloSecurityException {
     while (true) {
-      TabletClientService.Iface client = null;
+      TabletClientService.Client client = null;
       try {
         return exec.execute(client = getTServerClient(address, conf));
       } catch (TTransportException tte) {

Modified: accumulo/branches/ACCUMULO-672/pom.xml
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/pom.xml?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/pom.xml (original)
+++ accumulo/branches/ACCUMULO-672/pom.xml Thu Jul  5 19:54:58 2012
@@ -600,7 +600,7 @@
       <dependency>
         <groupId>org.apache.thrift</groupId>
         <artifactId>libthrift</artifactId>
-        <version>0.6.1</version>
+        <version>0.8.0</version>
       </dependency>
       <dependency>
         <groupId>jline</groupId>

Modified: accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java (original)
+++ accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java Thu Jul  5 19:54:58 2012
@@ -33,6 +33,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.cloudtrace.instrument.TraceRunnable;
+import org.apache.accumulo.cloudtrace.instrument.Tracer;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -130,7 +131,7 @@ public class BulkImporter {
       throw new RuntimeException("Directory does not exist " + failureDir);
     }
     
-    ClientService.Iface client = null;
+    ClientService.Client client = null;
     final TabletLocator locator = TabletLocator.getInstance(instance, credentials, new Text(tableId));
     
     try {
@@ -591,7 +592,7 @@ public class BulkImporter {
         }
         
         log.debug("Asking " + location + " to bulk load " + files);
-        List<TKeyExtent> failures = client.bulkImport(null, credentials, tid, Translator.translate(files, Translator.KET), setTime);
+        List<TKeyExtent> failures = client.bulkImport(Tracer.traceInfo(), credentials, tid, Translator.translate(files, Translator.KET), setTime);
         
         return Translator.translate(failures, Translator.TKET);
       } finally {

Modified: accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/gc/GarbageCollectWriteAheadLogs.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/gc/GarbageCollectWriteAheadLogs.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/gc/GarbageCollectWriteAheadLogs.java (original)
+++ accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/gc/GarbageCollectWriteAheadLogs.java Thu Jul  5 19:54:58 2012
@@ -28,6 +28,7 @@ import java.util.UUID;
 
 import org.apache.accumulo.cloudtrace.instrument.Span;
 import org.apache.accumulo.cloudtrace.instrument.Trace;
+import org.apache.accumulo.cloudtrace.instrument.Tracer;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
@@ -35,6 +36,7 @@ import org.apache.accumulo.core.conf.Pro
 import org.apache.accumulo.core.gc.thrift.GCStatus;
 import org.apache.accumulo.core.gc.thrift.GcCycleStats;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Iface;
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
@@ -137,10 +139,10 @@ public class GarbageCollectWriteAheadLog
         InetSocketAddress address = AddressUtil.parseAddress(entry.getKey(), Property.TSERV_CLIENTPORT);
         if (!holdsLock(address))
           continue;
-        Iface tserver = null;
+        Client tserver = null;
         try {
           tserver = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
-          tserver.removeLogs(null, SecurityConstants.getSystemCredentials(), entry.getValue());
+          tserver.removeLogs(Tracer.traceInfo(), SecurityConstants.getSystemCredentials(), entry.getValue());
           log.debug("deleted " + entry.getValue() + " from " + entry.getKey());
           status.currentLog.deleted += entry.getValue().size();
         } catch (TException e) {

Modified: accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java (original)
+++ accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java Thu Jul  5 19:54:58 2012
@@ -58,6 +58,7 @@ import org.apache.accumulo.core.file.Fil
 import org.apache.accumulo.core.file.FileUtil;
 import org.apache.accumulo.core.gc.thrift.GCMonitorService;
 import org.apache.accumulo.core.gc.thrift.GCMonitorService.Iface;
+import org.apache.accumulo.core.gc.thrift.GCMonitorService.Processor;
 import org.apache.accumulo.core.gc.thrift.GCStatus;
 import org.apache.accumulo.core.gc.thrift.GcCycleStats;
 import org.apache.accumulo.core.master.state.tables.TableState;
@@ -375,7 +376,7 @@ public class SimpleGarbageCollector impl
   }
   
   private InetSocketAddress startStatsService() throws UnknownHostException {
-    GCMonitorService.Processor processor = new GCMonitorService.Processor(TraceWrap.service(this));
+    Processor<Iface> processor = new Processor<Iface>(TraceWrap.service(this));
     int port = instance.getConfiguration().getPort(Property.GC_PORT);
     try {
       TServerUtils.startTServer(port, processor, this.getClass().getSimpleName(), "GC Monitor Service", 2, 1000);

Modified: accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java (original)
+++ accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java Thu Jul  5 19:54:58 2012
@@ -26,6 +26,8 @@ import java.util.Map;
 import java.util.Set;
 import java.util.TimerTask;
 
+import org.apache.accumulo.cloudtrace.instrument.Tracer;
+import org.apache.accumulo.cloudtrace.instrument.thrift.TraceWrap;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
@@ -79,54 +81,54 @@ public class LiveTServerSet implements W
     }
     
     public void assignTablet(ZooLock lock, KeyExtent extent) throws TException {
-      TabletClientService.Iface client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
+      TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
       try {
-        client.loadTablet(null, SecurityConstants.getSystemCredentials(), lockString(lock), extent.toThrift());
+        client.loadTablet(Tracer.traceInfo(), SecurityConstants.getSystemCredentials(), lockString(lock), extent.toThrift());
       } finally {
         ThriftUtil.returnClient(client);
       }
     }
     
     public void unloadTablet(ZooLock lock, KeyExtent extent, boolean save) throws TException {
-      TabletClientService.Iface client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
+      TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
       try {
-        client.unloadTablet(null, SecurityConstants.getSystemCredentials(), lockString(lock), extent.toThrift(), save);
+        client.unloadTablet(Tracer.traceInfo(), SecurityConstants.getSystemCredentials(), lockString(lock), extent.toThrift(), save);
       } finally {
         ThriftUtil.returnClient(client);
       }
     }
     
     public TabletServerStatus getTableMap() throws TException, ThriftSecurityException {
-      TabletClientService.Iface client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
+      TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
       try {
-        return client.getTabletServerStatus(null, SecurityConstants.getSystemCredentials());
+        return client.getTabletServerStatus(Tracer.traceInfo(), SecurityConstants.getSystemCredentials());
       } finally {
         ThriftUtil.returnClient(client);
       }
     }
     
     public void halt(ZooLock lock) throws TException, ThriftSecurityException {
-      TabletClientService.Iface client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
+      TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
       try {
-        client.halt(null, SecurityConstants.getSystemCredentials(), lockString(lock));
+        client.halt(Tracer.traceInfo(), SecurityConstants.getSystemCredentials(), lockString(lock));
       } finally {
         ThriftUtil.returnClient(client);
       }
     }
     
     public void fastHalt(ZooLock lock) throws TException {
-      TabletClientService.Iface client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
+      TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
       try {
-        client.fastHalt(null, SecurityConstants.getSystemCredentials(), lockString(lock));
+        client.fastHalt(Tracer.traceInfo(), SecurityConstants.getSystemCredentials(), lockString(lock));
       } finally {
         ThriftUtil.returnClient(client);
       }
     }
     
     public void flush(ZooLock lock, String tableId, byte[] startRow, byte[] endRow) throws TException {
-      TabletClientService.Iface client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
+      TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
       try {
-        client.flush(null, SecurityConstants.getSystemCredentials(), lockString(lock), tableId, startRow == null ? null : ByteBuffer.wrap(startRow),
+        client.flush(Tracer.traceInfo(), SecurityConstants.getSystemCredentials(), lockString(lock), tableId, startRow == null ? null : ByteBuffer.wrap(startRow),
             endRow == null ? null : ByteBuffer.wrap(endRow));
       } finally {
         ThriftUtil.returnClient(client);
@@ -134,37 +136,37 @@ public class LiveTServerSet implements W
     }
     
     public void chop(ZooLock lock, KeyExtent extent) throws TException {
-      TabletClientService.Iface client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
+      TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
       try {
-        client.chop(null, SecurityConstants.getSystemCredentials(), lockString(lock), extent.toThrift());
+        client.chop(Tracer.traceInfo(), SecurityConstants.getSystemCredentials(), lockString(lock), extent.toThrift());
       } finally {
         ThriftUtil.returnClient(client);
       }
     }
     
     public void splitTablet(ZooLock lock, KeyExtent extent, Text splitPoint) throws TException, ThriftSecurityException, NotServingTabletException {
-      TabletClientService.Iface client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
+      TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
       try {
         client
-            .splitTablet(null, SecurityConstants.getSystemCredentials(), extent.toThrift(), ByteBuffer.wrap(splitPoint.getBytes(), 0, splitPoint.getLength()));
+            .splitTablet(Tracer.traceInfo(), SecurityConstants.getSystemCredentials(), extent.toThrift(), ByteBuffer.wrap(splitPoint.getBytes(), 0, splitPoint.getLength()));
       } finally {
         ThriftUtil.returnClient(client);
       }
     }
     
     public void flushTablet(ZooLock lock, KeyExtent extent) throws TException {
-      TabletClientService.Iface client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
+      TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
       try {
-        client.flushTablet(null, SecurityConstants.getSystemCredentials(), lockString(lock), extent.toThrift());
+        client.flushTablet(Tracer.traceInfo(), SecurityConstants.getSystemCredentials(), lockString(lock), extent.toThrift());
       } finally {
         ThriftUtil.returnClient(client);
       }
     }
     
     public void compact(ZooLock lock, String tableId, byte[] startRow, byte[] endRow) throws TException {
-      TabletClientService.Iface client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
+      TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
       try {
-        client.compact(null, SecurityConstants.getSystemCredentials(), lockString(lock), tableId, startRow == null ? null : ByteBuffer.wrap(startRow),
+        client.compact(Tracer.traceInfo(), SecurityConstants.getSystemCredentials(), lockString(lock), tableId, startRow == null ? null : ByteBuffer.wrap(startRow),
             endRow == null ? null : ByteBuffer.wrap(endRow));
       } finally {
         ThriftUtil.returnClient(client);
@@ -172,9 +174,9 @@ public class LiveTServerSet implements W
     }
     
     public boolean isActive(long tid) throws TException {
-      TabletClientService.Iface client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
+      TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
       try {
-        return client.isActive(null, tid);
+        return client.isActive(Tracer.traceInfo(), tid);
       } finally {
         ThriftUtil.returnClient(client);
       }

Modified: accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/master/Master.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/master/Master.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/master/Master.java (original)
+++ accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/master/Master.java Thu Jul  5 19:54:58 2012
@@ -70,6 +70,7 @@ import org.apache.accumulo.core.file.Fil
 import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
+import org.apache.accumulo.core.master.thrift.MasterClientService.Iface;
 import org.apache.accumulo.core.master.thrift.MasterClientService.Processor;
 import org.apache.accumulo.core.master.thrift.MasterGoalState;
 import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
@@ -2103,7 +2104,7 @@ public class Master implements LiveTServ
       throw new IOException(e);
     }
     
-    Processor processor = new MasterClientService.Processor(TraceWrap.service(new MasterClientServiceHandler()));
+    Processor<Iface> processor = new Processor<Iface>(TraceWrap.service(new MasterClientServiceHandler()));
     clientService = TServerUtils.startServer(getSystemConfiguration(), Property.MASTER_CLIENTPORT, processor, "Master", "Master Client Service Handler", null,
         Property.MASTER_MINTHREADS, Property.MASTER_THREADCHECK).server;
     

Modified: accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java (original)
+++ accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java Thu Jul  5 19:54:58 2012
@@ -22,11 +22,12 @@ import java.util.Map;
 import java.util.Set;
 import java.util.SortedMap;
 
+import org.apache.accumulo.cloudtrace.instrument.Tracer;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Iface;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
 import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.server.conf.ServerConfiguration;
@@ -95,9 +96,9 @@ public abstract class TabletBalancer {
    */
   public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, String tableId) throws ThriftSecurityException, TException {
     log.debug("Scanning tablet server " + tserver + " for table " + tableId);
-    Iface client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), tserver.getLocation(), configuration.getConfiguration());
+    Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), tserver.getLocation(), configuration.getConfiguration());
     try {
-      List<TabletStats> onlineTabletsForTable = client.getTabletStats(null, SecurityConstants.getSystemCredentials(), tableId);
+      List<TabletStats> onlineTabletsForTable = client.getTabletStats(Tracer.traceInfo(), SecurityConstants.getSystemCredentials(), tableId);
       return onlineTabletsForTable;
     } catch (TTransportException e) {
       log.error("Unable to connect to " + tserver + ": " + e);

Modified: accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/master/tableOps/BulkImport.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/master/tableOps/BulkImport.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/master/tableOps/BulkImport.java (original)
+++ accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/master/tableOps/BulkImport.java Thu Jul  5 19:54:58 2012
@@ -36,6 +36,7 @@ import java.util.concurrent.Future;
 import java.util.concurrent.ThreadPoolExecutor;
 
 import org.apache.accumulo.cloudtrace.instrument.TraceExecutorService;
+import org.apache.accumulo.cloudtrace.instrument.Tracer;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
@@ -44,6 +45,7 @@ import org.apache.accumulo.core.client.S
 import org.apache.accumulo.core.client.impl.ServerClient;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.ClientService;
+import org.apache.accumulo.core.client.impl.thrift.ClientService.Client;
 import org.apache.accumulo.core.client.impl.thrift.ClientService.Iface;
 import org.apache.accumulo.core.client.impl.thrift.TableOperation;
 import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
@@ -522,18 +524,18 @@ class LoadFiles extends MasterRepo {
           @Override
           public List<String> call() {
             List<String> failures = new ArrayList<String>();
-            ClientService.Iface client = null;
+            ClientService.Client client = null;
             String server = null;
             try {
               // get a connection to a random tablet server, do not prefer cached connections because
               // this is running on the master and there are lots of connections to tablet servers
               // serving the !METADATA tablets
-              Pair<String,Iface> pair = ServerClient.getConnection(master.getInstance(), false);
+              Pair<String,Client> pair = ServerClient.getConnection(master.getInstance(), false);
               client = pair.getSecond();
               server = pair.getFirst();
               List<String> attempt = Collections.singletonList(file);
               log.debug("Asking " + pair.getFirst() + " to bulk import " + file);
-              List<String> fail = client.bulkImportFiles(null, SecurityConstants.getSystemCredentials(), tid, tableId, attempt, errorDir, setTime);
+              List<String> fail = client.bulkImportFiles(Tracer.traceInfo(), SecurityConstants.getSystemCredentials(), tid, tableId, attempt, errorDir, setTime);
               if (fail.isEmpty()) {
                 filesToLoad.remove(file);
               } else {

Modified: accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/monitor/Monitor.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/monitor/Monitor.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/monitor/Monitor.java (original)
+++ accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/monitor/Monitor.java Thu Jul  5 19:54:58 2012
@@ -27,6 +27,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 
+import org.apache.accumulo.cloudtrace.instrument.Tracer;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.MasterClient;
@@ -292,7 +293,7 @@ public class Monitor {
         try {
           client = MasterClient.getConnection(HdfsZooInstance.getInstance());
           if (client != null) {
-            mmi = client.getMasterStats(null, SecurityConstants.getSystemCredentials());
+            mmi = client.getMasterStats(Tracer.traceInfo(), SecurityConstants.getSystemCredentials());
           } else {
             mmi = null;
           }
@@ -429,9 +430,9 @@ public class Monitor {
         if (locks != null && locks.size() > 0) {
           Collections.sort(locks);
           InetSocketAddress address = new ServerServices(new String(zk.getData(path + "/" + locks.get(0), null, null))).getAddress(Service.GC_CLIENT);
-          GCMonitorService.Iface client = ThriftUtil.getClient(new GCMonitorService.Client.Factory(), address, config.getConfiguration());
+          GCMonitorService.Client client = ThriftUtil.getClient(new GCMonitorService.Client.Factory(), address, config.getConfiguration());
           try {
-            result = client.getStatus(null, SecurityConstants.getSystemCredentials());
+            result = client.getStatus(Tracer.traceInfo(), SecurityConstants.getSystemCredentials());
           } finally {
             ThriftUtil.returnClient(client);
           }

Modified: accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/monitor/servlets/TServersServlet.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/monitor/servlets/TServersServlet.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/monitor/servlets/TServersServlet.java (original)
+++ accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/monitor/servlets/TServersServlet.java Thu Jul  5 19:54:58 2012
@@ -27,6 +27,7 @@ import java.util.Map.Entry;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import org.apache.accumulo.cloudtrace.instrument.Tracer;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.master.thrift.DeadServer;
 import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
@@ -122,12 +123,12 @@ public class TServersServlet extends Bas
     TabletStats historical = new TabletStats(null, new ActionStats(), new ActionStats(), new ActionStats(), 0, 0, 0, 0);
     List<TabletStats> tsStats = new ArrayList<TabletStats>();
     try {
-      TabletClientService.Iface client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, Monitor.getSystemConfiguration());
+      TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, Monitor.getSystemConfiguration());
       try {
         for (String tableId : Monitor.getMmi().tableMap.keySet()) {
-          tsStats.addAll(client.getTabletStats(null, SecurityConstants.getSystemCredentials(), tableId));
+          tsStats.addAll(client.getTabletStats(Tracer.traceInfo(), SecurityConstants.getSystemCredentials(), tableId));
         }
-        historical = client.getHistoricalStats(null, SecurityConstants.getSystemCredentials());
+        historical = client.getHistoricalStats(Tracer.traceInfo(), SecurityConstants.getSystemCredentials());
       } finally {
         ThriftUtil.returnClient(client);
       }

Modified: accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java (original)
+++ accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java Thu Jul  5 19:54:58 2012
@@ -118,6 +118,8 @@ import org.apache.accumulo.core.tabletse
 import org.apache.accumulo.core.tabletserver.thrift.ScanState;
 import org.apache.accumulo.core.tabletserver.thrift.ScanType;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Iface;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Processor;
 import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
 import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.core.util.ByteBufferUtil;
@@ -2622,12 +2624,12 @@ public class TabletServer extends Abstra
   }
   
   // Connect to the master for posting asynchronous results
-  private MasterClientService.Iface masterConnection(String address) {
+  private MasterClientService.Client masterConnection(String address) {
     try {
       if (address == null) {
         return null;
       }
-      MasterClientService.Iface client = ThriftUtil.getClient(new MasterClientService.Client.Factory(), address, Property.MASTER_CLIENTPORT,
+      MasterClientService.Client client = ThriftUtil.getClient(new MasterClientService.Client.Factory(), address, Property.MASTER_CLIENTPORT,
           Property.GENERAL_RPC_TIMEOUT, getSystemConfiguration());
       // log.info("Listener API to master has been opened");
       return client;
@@ -2637,14 +2639,14 @@ public class TabletServer extends Abstra
     return null;
   }
   
-  private void returnMasterConnection(MasterClientService.Iface client) {
+  private void returnMasterConnection(MasterClientService.Client client) {
     ThriftUtil.returnClient(client);
   }
   
   private int startTabletClientService() throws UnknownHostException {
     // start listening for client connection last
-    TabletClientService.Iface tch = TraceWrap.service(new ThriftClientHandler());
-    TabletClientService.Processor processor = new TabletClientService.Processor(tch);
+    Iface tch = TraceWrap.service(new ThriftClientHandler());
+    Processor<Iface> processor = new Processor<Iface>(tch);
     int port = startServer(getSystemConfiguration(), Property.TSERV_CLIENTPORT, processor, "Thrift Client Server");
     log.info("port = " + port);
     return port;
@@ -2745,7 +2747,7 @@ public class TabletServer extends Abstra
       // send all of the pending messages
       try {
         MasterMessage mm = null;
-        MasterClientService.Iface iface = null;
+        MasterClientService.Client iface = null;
         
         try {
           // wait until a message is ready to send, or a sever stop

Modified: accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/SplitReportMessage.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/SplitReportMessage.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/SplitReportMessage.java (original)
+++ accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/SplitReportMessage.java Thu Jul  5 19:54:58 2012
@@ -19,6 +19,7 @@ package org.apache.accumulo.server.table
 import java.util.Map;
 import java.util.TreeMap;
 
+import org.apache.accumulo.cloudtrace.instrument.Tracer;
 import org.apache.accumulo.core.client.impl.Translator;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
@@ -48,7 +49,7 @@ public class SplitReportMessage implemen
     TabletSplit split = new TabletSplit();
     split.oldTablet = old_extent.toThrift();
     split.newTablets = Translator.translate(extents.keySet(), Translator.KET);
-    client.reportSplitExtent(null, credentials, serverName, split);
+    client.reportSplitExtent(Tracer.traceInfo(), credentials, serverName, split);
   }
   
 }

Modified: accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/TabletStatusMessage.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/TabletStatusMessage.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/TabletStatusMessage.java (original)
+++ accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/TabletStatusMessage.java Thu Jul  5 19:54:58 2012
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.server.tabletserver.mastermessage;
 
+import org.apache.accumulo.cloudtrace.instrument.Tracer;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.master.thrift.TabletLoadState;
 import org.apache.accumulo.core.master.thrift.MasterClientService.Iface;
@@ -34,6 +35,6 @@ public class TabletStatusMessage impleme
   }
   
   public void send(AuthInfo auth, String serverName, Iface client) throws TException, ThriftSecurityException {
-    client.reportTabletStatus(null, auth, serverName, status, extent.toThrift());
+    client.reportTabletStatus(Tracer.traceInfo(), auth, serverName, status, extent.toThrift());
   }
 }