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 [2/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...

Modified: accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/test/GetMasterStats.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/test/GetMasterStats.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/test/GetMasterStats.java (original)
+++ accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/test/GetMasterStats.java Thu Jul  5 19:54:58 2012
@@ -19,6 +19,7 @@ package org.apache.accumulo.server.test;
 import java.io.IOException;
 import java.util.Map.Entry;
 
+import org.apache.accumulo.cloudtrace.instrument.Tracer;
 import org.apache.accumulo.core.client.impl.MasterClient;
 import org.apache.accumulo.core.master.MasterNotRunningException;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
@@ -43,7 +44,7 @@ public class GetMasterStats {
     MasterMonitorInfo stats = null;
     try {
       client = MasterClient.getConnectionWithRetry(HdfsZooInstance.getInstance());
-      stats = client.getMasterStats(null, SecurityConstants.getSystemCredentials());
+      stats = client.getMasterStats(Tracer.traceInfo(), SecurityConstants.getSystemCredentials());
     } finally {
       if (client != null)
         MasterClient.close(client);

Modified: accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/test/WrongTabletTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/test/WrongTabletTest.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/test/WrongTabletTest.java (original)
+++ accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/test/WrongTabletTest.java Thu Jul  5 19:54:58 2012
@@ -18,6 +18,7 @@ package org.apache.accumulo.server.test;
 
 import java.nio.ByteBuffer;
 
+import org.apache.accumulo.cloudtrace.instrument.Tracer;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.security.thrift.AuthInfo;
@@ -39,7 +40,7 @@ public class WrongTabletTest {
       Mutation mutation = new Mutation(new Text("row_0003750001"));
       // mutation.set(new Text("colf:colq"), new Value("val".getBytes()));
       mutation.putDelete(new Text("colf"), new Text("colq"));
-      client.update(null, rootCredentials, new KeyExtent(new Text("test_ingest"), null, new Text("row_0003750000")).toThrift(), mutation.toThrift());
+      client.update(Tracer.traceInfo(), rootCredentials, new KeyExtent(new Text("test_ingest"), null, new Text("row_0003750000")).toThrift(), mutation.toThrift());
     } catch (Exception e) {
       throw new RuntimeException(e);
     }

Modified: accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousStatsCollector.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousStatsCollector.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousStatsCollector.java (original)
+++ accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousStatsCollector.java Thu Jul  5 19:54:58 2012
@@ -22,6 +22,7 @@ import java.util.Map.Entry;
 import java.util.Timer;
 import java.util.TimerTask;
 
+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.IteratorSetting;
@@ -126,7 +127,7 @@ public class ContinuousStatsCollector {
       MasterClientService.Iface client = null;
       try {
         client = MasterClient.getConnectionWithRetry(HdfsZooInstance.getInstance());
-        MasterMonitorInfo stats = client.getMasterStats(null, SecurityConstants.getSystemCredentials());
+        MasterMonitorInfo stats = client.getMasterStats(Tracer.traceInfo(), SecurityConstants.getSystemCredentials());
         
         TableInfo all = new TableInfo();
         Map<String,TableInfo> tableSummaries = new HashMap<String,TableInfo>();

Modified: accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/test/functional/ZombieTServer.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/test/functional/ZombieTServer.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/test/functional/ZombieTServer.java (original)
+++ accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/test/functional/ZombieTServer.java Thu Jul  5 19:54:58 2012
@@ -21,6 +21,7 @@ import java.net.InetSocketAddress;
 import java.util.HashMap;
 import java.util.Random;
 
+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.Instance;
@@ -29,6 +30,8 @@ import org.apache.accumulo.core.master.t
 import org.apache.accumulo.core.security.thrift.AuthInfo;
 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.Processor;
 import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.core.util.ServerServices;
 import org.apache.accumulo.core.util.ServerServices.Service;
@@ -96,7 +99,7 @@ public class ZombieTServer {
     
     TransactionWatcher watcher = new TransactionWatcher();
     final ThriftClientHandler tch = new ThriftClientHandler(instance, watcher);
-    TabletClientService.Processor processor = new TabletClientService.Processor(tch);
+    Processor<Iface> processor = new Processor<Iface>(tch);
     ServerPort serverPort = TServerUtils.startTServer(port, processor, "ZombieTServer", "walking dead", 2, 1000);
     
     InetSocketAddress addr = new InetSocketAddress(InetAddress.getLocalHost(), serverPort.port);
@@ -111,7 +114,7 @@ public class ZombieTServer {
       @Override
       public void lostLock(final LockLossReason reason) {
         try {
-          tch.halt(null, null, null);
+          tch.halt(Tracer.traceInfo(), null, null);
         } catch (Exception ex) {
           log.error(ex, ex);
           System.exit(1);

Modified: accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/trace/TraceServer.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/trace/TraceServer.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/trace/TraceServer.java (original)
+++ accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/trace/TraceServer.java Thu Jul  5 19:54:58 2012
@@ -23,7 +23,8 @@ import java.util.TimerTask;
 
 import org.apache.accumulo.cloudtrace.instrument.Span;
 import org.apache.accumulo.cloudtrace.thrift.RemoteSpan;
-import org.apache.accumulo.cloudtrace.thrift.SpanReceiver;
+import org.apache.accumulo.cloudtrace.thrift.SpanReceiver.Processor;
+import org.apache.accumulo.cloudtrace.thrift.SpanReceiver.Iface;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
@@ -110,7 +111,7 @@ public class TraceServer implements Watc
     }
   }
   
-  class Receiver implements SpanReceiver.Iface {
+  class Receiver implements Iface {
     @Override
     public void span(RemoteSpan s) throws TException {
       String idString = Long.toHexString(s.traceId);
@@ -172,7 +173,7 @@ public class TraceServer implements Watc
     sock.bind(new InetSocketAddress(port));
     final TServerTransport transport = new TServerSocket(sock);
     TThreadPoolServer.Args options = new TThreadPoolServer.Args(transport);
-    options.processor(new SpanReceiver.Processor(new Receiver()));
+    options.processor(new Processor<Iface>(new Receiver()));
     server = new TThreadPoolServer(options);
     final InetSocketAddress address = new InetSocketAddress(hostname, sock.getLocalPort());
     registerInZooKeeper(AddressUtil.toString(address));

Modified: accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/util/Admin.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/util/Admin.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/util/Admin.java (original)
+++ accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/util/Admin.java Thu Jul  5 19:54:58 2012
@@ -24,6 +24,7 @@ import java.nio.ByteBuffer;
 
 import jline.ConsoleReader;
 
+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.impl.ClientExec;
@@ -119,10 +120,10 @@ public class Admin {
   }
   
   private static void stopServer(final AuthInfo credentials, final boolean tabletServersToo) throws AccumuloException, AccumuloSecurityException {
-    MasterClient.execute(HdfsZooInstance.getInstance(), new ClientExec<MasterClientService.Iface>() {
+    MasterClient.execute(HdfsZooInstance.getInstance(), new ClientExec<MasterClientService.Client>() {
       @Override
-      public void execute(MasterClientService.Iface client) throws Exception {
-        client.shutdown(null, credentials, tabletServersToo);
+      public void execute(MasterClientService.Client client) throws Exception {
+        client.shutdown(Tracer.traceInfo(), credentials, tabletServersToo);
       }
     });
   }
@@ -130,10 +131,10 @@ public class Admin {
   private static void stopTabletServer(String server, final boolean force) throws AccumuloException, AccumuloSecurityException {
     InetSocketAddress address = AddressUtil.parseAddress(server, Property.TSERV_CLIENTPORT);
     final String finalServer = org.apache.accumulo.core.util.AddressUtil.toString(address);
-    MasterClient.execute(HdfsZooInstance.getInstance(), new ClientExec<MasterClientService.Iface>() {
+    MasterClient.execute(HdfsZooInstance.getInstance(), new ClientExec<MasterClientService.Client>() {
       @Override
-      public void execute(MasterClientService.Iface client) throws Exception {
-        client.shutdownTabletServer(null, SecurityConstants.getSystemCredentials(), finalServer, force);
+      public void execute(MasterClientService.Client client) throws Exception {
+        client.shutdownTabletServer(Tracer.traceInfo(), SecurityConstants.getSystemCredentials(), finalServer, force);
       }
     });
   }

Modified: accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/util/TServerUtils.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/util/TServerUtils.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/util/TServerUtils.java (original)
+++ accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/util/TServerUtils.java Thu Jul  5 19:54:58 2012
@@ -20,7 +20,6 @@ import java.io.IOException;
 import java.lang.reflect.Field;
 import java.net.InetSocketAddress;
 import java.net.ServerSocket;
-import java.net.Socket;
 import java.net.UnknownHostException;
 import java.nio.channels.ServerSocketChannel;
 import java.util.Random;
@@ -43,10 +42,10 @@ import org.apache.thrift.TException;
 import org.apache.thrift.TProcessor;
 import org.apache.thrift.TProcessorFactory;
 import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.server.THsHaServer;
 import org.apache.thrift.server.TServer;
 import org.apache.thrift.server.TThreadPoolServer;
 import org.apache.thrift.transport.TNonblockingServerSocket;
-import org.apache.thrift.transport.TNonblockingSocket;
 import org.apache.thrift.transport.TServerTransport;
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
@@ -148,7 +147,12 @@ public class TServerUtils {
         metrics.add(ThriftMetrics.idle, (now - idleStart));
       }
       try {
-        return other.process(in, out);
+        try {
+          return other.process(in, out);
+        } catch (NullPointerException ex) {
+          // THRIFT-1447 - remove with thrift 0.9
+          return true;
+        }
       } finally {
         if (metrics.isEnabled()) {
           idleStart = System.currentTimeMillis();
@@ -173,34 +177,6 @@ public class TServerUtils {
     }
   }
   
-  public static class THsHaServer extends org.apache.thrift.server.THsHaServer {
-    public THsHaServer(Args args) {
-      super(args);
-    }
-    
-    protected Runnable getRunnable(FrameBuffer frameBuffer) {
-      return new Invocation(frameBuffer);
-    }
-    
-    private class Invocation implements Runnable {
-      
-      private final FrameBuffer frameBuffer;
-      
-      public Invocation(final FrameBuffer frameBuffer) {
-        this.frameBuffer = frameBuffer;
-      }
-      
-      public void run() {
-        if (frameBuffer.trans_ instanceof TNonblockingSocket) {
-          TNonblockingSocket tsock = (TNonblockingSocket) frameBuffer.trans_;
-          Socket sock = tsock.getSocketChannel().socket();
-          clientAddress.set(sock.getInetAddress().getHostAddress() + ":" + sock.getPort());
-        }
-        frameBuffer.invoke();
-      }
-    }
-  }
-  
   public static ServerPort startHsHaServer(int port, TProcessor processor, final String serverName, String threadName, final int numThreads,
       long timeBetweenThreadChecks) throws TTransportException {
     TNonblockingServerSocket transport = new TNonblockingServerSocket(port);

Modified: accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java (original)
+++ accumulo/branches/ACCUMULO-672/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java Thu Jul  5 19:54:58 2012
@@ -33,6 +33,8 @@ import java.util.concurrent.TimeUnit;
 
 import jline.ConsoleReader;
 
+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;
@@ -241,23 +243,23 @@ public class VerifyTabletAssignments {
       Range r = new Range(row, true, row2, false);
       batch.put(keyExtent.toThrift(), Collections.singletonList(r.toThrift()));
     }
-    
+    TInfo tinfo = Tracer.traceInfo();
     Map<String,Map<String,String>> emptyMapSMapSS = Collections.emptyMap();
     List<IterInfo> emptyListIterInfo = Collections.emptyList();
     List<TColumn> emptyListColumn = Collections.emptyList();
-    InitialMultiScan is = client.startMultiScan(null, st, batch, emptyListColumn, emptyListIterInfo, emptyMapSMapSS, Constants.NO_AUTHS.getAuthorizationsBB(),
+    InitialMultiScan is = client.startMultiScan(tinfo, st, batch, emptyListColumn, emptyListIterInfo, emptyMapSMapSS, Constants.NO_AUTHS.getAuthorizationsBB(),
         false);
     if (is.result.more) {
-      MultiScanResult result = client.continueMultiScan(null, is.scanID);
+      MultiScanResult result = client.continueMultiScan(tinfo, is.scanID);
       checkFailures(entry.getKey(), failures, result);
       
       while (result.more) {
-        result = client.continueMultiScan(null, is.scanID);
+        result = client.continueMultiScan(tinfo, is.scanID);
         checkFailures(entry.getKey(), failures, result);
       }
     }
     
-    client.closeMultiScan(null, is.scanID);
+    client.closeMultiScan(tinfo, is.scanID);
     
     ThriftUtil.returnClient((TServiceClient) client);
   }

Modified: accumulo/branches/ACCUMULO-672/trace/pom.xml
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/trace/pom.xml?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/trace/pom.xml (original)
+++ accumulo/branches/ACCUMULO-672/trace/pom.xml Thu Jul  5 19:54:58 2012
@@ -50,15 +50,19 @@
       <artifactId>libthrift</artifactId>
     </dependency>
 
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
-    </dependency>
 
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>zookeeper</artifactId>
     </dependency>
+    <dependency>
+    	<groupId>org.slf4j</groupId>
+    	<artifactId>slf4j-log4j12</artifactId>
+    </dependency>
+    <dependency>
+    	<groupId>org.slf4j</groupId>
+    	<artifactId>slf4j-api</artifactId>
+    </dependency>
   </dependencies>
 
 </project>

Modified: accumulo/branches/ACCUMULO-672/trace/src/test/java/org/apache/accumulo/cloudtrace/instrument/TracerTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/trace/src/test/java/org/apache/accumulo/cloudtrace/instrument/TracerTest.java?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/trace/src/test/java/org/apache/accumulo/cloudtrace/instrument/TracerTest.java (original)
+++ accumulo/branches/ACCUMULO-672/trace/src/test/java/org/apache/accumulo/cloudtrace/instrument/TracerTest.java Thu Jul  5 19:54:58 2012
@@ -35,6 +35,8 @@ import org.apache.accumulo.cloudtrace.in
 import org.apache.accumulo.cloudtrace.instrument.thrift.TraceWrap;
 import org.apache.accumulo.cloudtrace.thrift.TInfo;
 import org.apache.accumulo.cloudtrace.thrift.TestService;
+import org.apache.accumulo.cloudtrace.thrift.TestService.Iface;
+import org.apache.accumulo.cloudtrace.thrift.TestService.Processor;
 import org.apache.thrift.TException;
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.server.TServer;
@@ -153,7 +155,7 @@ public class TracerTest {
     TServerSocket transport = new TServerSocket(socket);
     transport.listen();
     TThreadPoolServer.Args args = new TThreadPoolServer.Args(transport);
-    args.processor(new TestService.Processor(TraceWrap.service(new Service())));
+    args.processor(new Processor<Iface>(TraceWrap.service(new Service())));
     final TServer tserver = new TThreadPoolServer(args);
     Thread t = new Thread() {
       public void run() {

Modified: accumulo/branches/ACCUMULO-672/trace/thrift.sh
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-672/trace/thrift.sh?rev=1357842&r1=1357841&r2=1357842&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-672/trace/thrift.sh (original)
+++ accumulo/branches/ACCUMULO-672/trace/thrift.sh Thu Jul  5 19:54:58 2012
@@ -20,18 +20,18 @@
 
 # Generates the cloudtrace thrift code. We're explicitly using thrift0.6 because other versions
 # are not compatible
-thrift0.6 -o target -gen java src/main/thrift/cloudtrace.thrift
+thrift -o target -gen java src/main/thrift/cloudtrace.thrift
 
 # For all generated thrift code, suppress all warnings
 find target/gen-java -name '*.java' -print | xargs sed -i.orig -e 's/public class /@SuppressWarnings("all") public class /'
 
 # Make a directory for said thrift code if does not already exist
-mkdir -p src/main/java/cloudtrace/thrift
+mkdir -p src/main/java/org/apache/accumulo/cloudtrace/thrift
 
 # For every file, move it with the appropriate path name IFF they are different
-for f in target/gen-java/cloudtrace/thrift/*
+for f in target/gen-java/org/apache/accumulo/cloudtrace/thrift/*
 do
-  DEST=src/main/java/cloudtrace/thrift/`basename $f`
+  DEST=src/main/java/org/apache/accumulo/cloudtrace/thrift/`basename $f`
   if ! cmp -s $f $DEST ; then
      echo cp $f $DEST
      cp $f $DEST