You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by vi...@apache.org on 2012/07/13 22:34:49 UTC

svn commit: r1361382 [15/16] - in /accumulo/branches/ACCUMULO-259: ./ bin/ conf/examples/1GB/native-standalone/ conf/examples/1GB/standalone/ conf/examples/2GB/native-standalone/ conf/examples/2GB/standalone/ conf/examples/3GB/native-standalone/ conf/e...

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java?rev=1361382&r1=1361381&r2=1361382&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java Fri Jul 13 20:34:44 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-259/server/src/main/java/org/apache/accumulo/server/util/ZooZap.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/util/ZooZap.java?rev=1361382&r1=1361381&r2=1361382&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/util/ZooZap.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/util/ZooZap.java Fri Jul 13 20:34:44 2012
@@ -115,7 +115,7 @@ public class ZooZap {
   }
   
   private static void printUsage() {
-    System.err.println("Usage : " + ZooZap.class.getName() + " [-verbose] [-tservers] [-master] [-loggers]");
+    System.err.println("Usage : " + ZooZap.class.getName() + " [-verbose] [-tservers] [-master] [-tracers]");
   }
   
 }

Added: accumulo/branches/ACCUMULO-259/server/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java?rev=1361382&view=auto
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java (added)
+++ accumulo/branches/ACCUMULO-259/server/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java Fri Jul 13 20:34:44 2012
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.server.security.handler;
+
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.SystemPermission;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.core.util.ByteArraySet;
+import org.apache.accumulo.server.security.handler.ZKSecurityTool;
+
+import junit.framework.TestCase;
+
+public class ZKAuthenticatorTest extends TestCase {
+  public void testPermissionIdConversions() {
+    for (SystemPermission s : SystemPermission.values())
+      assertTrue(s.equals(SystemPermission.getPermissionById(s.getId())));
+    
+    for (TablePermission s : TablePermission.values())
+      assertTrue(s.equals(TablePermission.getPermissionById(s.getId())));
+  }
+  
+  public void testAuthorizationConversion() {
+    ByteArraySet auths = new ByteArraySet();
+    for (int i = 0; i < 300; i += 3)
+      auths.add(Integer.toString(i).getBytes());
+    
+    Authorizations converted = new Authorizations(auths);
+    byte[] test = ZKSecurityTool.convertAuthorizations(converted);
+    Authorizations test2 = ZKSecurityTool.convertAuthorizations(test);
+    assertTrue(auths.size() == test2.size());
+    for (byte[] s : auths) {
+      assertTrue(test2.contains(s));
+    }
+  }
+  
+  public void testSystemConversion() {
+    Set<SystemPermission> perms = new TreeSet<SystemPermission>();
+    for (SystemPermission s : SystemPermission.values())
+      perms.add(s);
+    
+    Set<SystemPermission> converted = ZKSecurityTool.convertSystemPermissions(ZKSecurityTool.convertSystemPermissions(perms));
+    assertTrue(perms.size() == converted.size());
+    for (SystemPermission s : perms)
+      assertTrue(converted.contains(s));
+  }
+  
+  public void testTableConversion() {
+    Set<TablePermission> perms = new TreeSet<TablePermission>();
+    for (TablePermission s : TablePermission.values())
+      perms.add(s);
+    
+    Set<TablePermission> converted = ZKSecurityTool.convertTablePermissions(ZKSecurityTool.convertTablePermissions(perms));
+    assertTrue(perms.size() == converted.size());
+    for (TablePermission s : perms)
+      assertTrue(converted.contains(s));
+  }
+  
+  public void testEncryption() {
+    byte[] rawPass = "myPassword".getBytes();
+    byte[] storedBytes;
+    try {
+      storedBytes = ZKSecurityTool.createPass(rawPass);
+      assertTrue(ZKSecurityTool.checkPass(rawPass, storedBytes));
+    } catch (AccumuloException e) {
+      e.printStackTrace();
+      assertTrue(false);
+    }
+  }
+}

Propchange: accumulo/branches/ACCUMULO-259/src/
------------------------------------------------------------------------------
  Merged /accumulo/branches/1.4/src:r1343943-1349971,1349973-1351424,1351426-1354668,1354673-1356923,1358206
  Merged /accumulo/branches/1.3/src:r1354669
  Merged /accumulo/trunk:r1341000,1344302,1344358,1356400
  Merged /accumulo/trunk/src:r1355557-1361354
  Merged /accumulo/branches/1.4/src/src:r1339309-1342420,1343897-1343898,1343943-1349971,1349973-1351424,1351426-1354669,1354673-1356900,1358206
  Merged /accumulo/branches/ACCUMULO-672/src:r1357826,1357829,1357842,1357858,1358236,1359163

Modified: accumulo/branches/ACCUMULO-259/test/system/auto/TestUtils.py
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/test/system/auto/TestUtils.py?rev=1361382&r1=1361381&r2=1361382&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/test/system/auto/TestUtils.py (original)
+++ accumulo/branches/ACCUMULO-259/test/system/auto/TestUtils.py Fri Jul 13 20:34:44 2012
@@ -170,20 +170,10 @@ class TestUtilsMixin:
         self.pkill(host, 'Main master$', signal=signal.SIGHUP)
         self.cleanupAccumuloHandles()
 
-    def stop_logger(self, host):
-        self.pkill(host, 'Main logger$', signal=signal.SIGHUP)
-        # wait for it to stop
-        self.sleep(1)
-        self.cleanupAccumuloHandles(0.5)
-
     def start_tserver(self, host):
         return self.runOn(host,
                           [self.accumulo_sh(), 'tserver'])
 
-    def start_logger(self, host):
-        return self.runOn(host,
-                          [self.accumulo_sh(), 'logger'])
-
     def start_monitor(self, host):
         return self.runOn(host, [self.accumulo_sh(), 'monitor'])
 
@@ -247,7 +237,6 @@ class TestUtilsMixin:
                           'tserver.port.client': 39000 + FUZZ,
                           'master.port.client':  41000 + FUZZ,
                           'monitor.port.client': 50099,
-                          'logger.port.client':  44000 + FUZZ,
                           'gc.port.client':      45000 + FUZZ,
                           'logger.dir.walog': WALOG,
                           'general.classpaths' :General_CLASSPATH,
@@ -292,8 +281,6 @@ class TestUtilsMixin:
 
     def start_accumulo_procs(self, safeMode=None):
         self.accumuloHandles = [
-           self.start_logger(host) for host in self.hosts 
-           ] + [
            self.start_tserver(host) for host in self.hosts
            ] + [
            self.start_monitor(self.masterHost())
@@ -389,8 +376,6 @@ class TestUtilsMixin:
                  [self.accumulo_sh(), 'admin', '-u', ROOT,
                  '-p', ROOT_PASSWORD, 'stopAll'])
         self.waitForStop(handle, seconds)
-        for host in self.hosts:
-            self.stop_logger(host)
         self.stop_monitor(self.masterHost())
         self.cleanupAccumuloHandles()
         # give everyone a couple seconds to completely stop

Modified: accumulo/branches/ACCUMULO-259/test/system/auto/simple/shutdown.py
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/test/system/auto/simple/shutdown.py?rev=1361382&r1=1361381&r2=1361382&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/test/system/auto/simple/shutdown.py (original)
+++ accumulo/branches/ACCUMULO-259/test/system/auto/simple/shutdown.py Fri Jul 13 20:34:44 2012
@@ -19,9 +19,10 @@ import logging
 import unittest
 import sleep
 import signal
+import socket
 from subprocess import PIPE
 
-from TestUtils import TestUtilsMixin, ROOT, ROOT_PASSWORD
+from TestUtils import TestUtilsMixin, ROOT, ROOT_PASSWORD, FUZZ
 from simple.readwrite import SunnyDayTest
 
 log = logging.getLogger('test.auto')
@@ -76,14 +77,24 @@ class ShutdownDuringDeleteTable(TestUtil
         handle.stdin.write(dt)
         self.shutdown_accumulo()
 
-class ShutdownDuringStart(TestUtilsMixin, unittest.TestCase):
+class AdminStopDuringStart(TestUtilsMixin, unittest.TestCase):
 
     order = SunnyDayTest.order + 1
     
     def runTest(self):
-        self.hosts = self.options.hosts
         self.clean_accumulo(self.masterHost())
         self.start_accumulo()
+        handle = self.runOn(self.masterHost(),
+                            [self.accumulo_sh(),'admin','stop', socket.getfqdn() + ":%d" % (39000 + FUZZ)])
+
+class AdminStop(SunnyDayTest):
+
+    order = SunnyDayTest.order + 1
+    
+    def runTest(self):
+        self.waitForStop(self.ingester, self.waitTime())
+        handle = self.runOn(self.masterHost(),
+                            [self.accumulo_sh(),'admin','stop', socket.getfqdn() + ":%d" % (39000 + FUZZ)])
         self.shutdown_accumulo()
 
 def suite():
@@ -92,5 +103,6 @@ def suite():
     result.addTest(ShutdownDuringQuery())
     result.addTest(ShutdownDuringDelete())
     result.addTest(ShutdownDuringDeleteTable())
-    result.addTest(ShutdownDuringStart())
+    result.addTest(AdminStopDuringStart())
+    result.addTest(AdminStop())
     return result

Modified: accumulo/branches/ACCUMULO-259/test/system/auto/simple/split.py
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/test/system/auto/simple/split.py?rev=1361382&r1=1361381&r2=1361382&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/test/system/auto/simple/split.py (original)
+++ accumulo/branches/ACCUMULO-259/test/system/auto/simple/split.py Fri Jul 13 20:34:44 2012
@@ -38,7 +38,8 @@ class TabletShouldSplit(SunnyDayTest):
         })
     tableSettings = SunnyDayTest.tableSettings.copy()
     tableSettings['test_ingest'] = { 
-    	'table.split.threshold': '5K',
+    	'table.split.threshold': '256K',
+      'table.file.compress.blocksize': '1K',
         }
     def runTest(self):
 
@@ -65,10 +66,10 @@ class TabletShouldSplit(SunnyDayTest):
                     lines.append(line)
         # check that the row values aren't always whole rows, but something shorter
         for line in lines:
-            if len(line) != len(lines[0]):
-                break
+          if len(line) != 14:
+            break
         else:
-            self.fail("The split points are not being shortened")
+          self.fail("The split points are not being shortened")
 
         self.assert_(len(lines) > 10)
 

Modified: accumulo/branches/ACCUMULO-259/test/system/auto/simple/wal.py
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/test/system/auto/simple/wal.py?rev=1361382&r1=1361381&r2=1361382&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/test/system/auto/simple/wal.py (original)
+++ accumulo/branches/ACCUMULO-259/test/system/auto/simple/wal.py Fri Jul 13 20:34:44 2012
@@ -60,65 +60,8 @@ class WriteAheadLog(SunnyDayTest):
                            waitTime)
           self.shutdown_accumulo()
 
-class DiskFailure(SunnyDayTest):
-
-     order = 25
-
-     settings = SunnyDayTest.settings.copy()
-   
-     # compact frequently
-     settings['tserver.port.search'] = 'true'
-     settings['tserver.memory.maps.max'] = '200K'
-     settings['tserver.compaction.major.delay'] = 1
-     settings['tserver.logger.timeout'] = '5s'
-
-     def start_accumulo_procs(self, safeMode=None):
-          log.info("Starting normal accumulo")
-          SunnyDayTest.start_accumulo_procs(self, safeMode)
-          log.info("Starting victim logger")
-          libpath = '%s/test/system/auto/fake_disk_failure.so' % ACCUMULO_HOME
-          os.environ['LD_PRELOAD'] = libpath
-          os.environ['DYLD_INSERT_LIBRARIES'] = libpath
-          os.environ['DYLD_FORCE_FLAT_NAMESPACE'] = 'true'
-          stop = self.start_logger(self.masterHost())
-          del os.environ['LD_PRELOAD']
-          del os.environ['DYLD_FORCE_FLAT_NAMESPACE']
-          del os.environ['DYLD_INSERT_LIBRARIES']
-          self.flagFile = os.getenv("HOME") + "/HOLD_IO_%d" % stop.pid
-          self.sleep(5)
-          
-     def runTest(self):
-          self.sleep(3)
-          waitTime = self.waitTime()
-          log.info("Waiting for ingest to stop")
-          self.waitForStop(self.ingester, waitTime)
-
-          log.info("Starting fake disk failure for logger")
-          fp = open(self.flagFile, "w+")
-          fp.close()
-          self.ingester = self.ingest(self.masterHost(),
-                                      self.options.rows,
-                                      self.options.rows,
-                                      size=self.options.size)
-          self.waitForStop(self.ingester, waitTime)
-          
-          log.info("Verifying Ingestion")
-          self.waitForStop(self.verify(self.masterHost(),
-                                       self.options.rows * 2,
-                                       size=self.options.size),
-                           waitTime)
-
-     def tearDown(self):
-          SunnyDayTest.tearDown(self)
-          try:
-               os.unlink(self.flagFile)
-          except:
-               pass
-          
-
 def suite():
      result = unittest.TestSuite()
      result.addTest(WriteAheadLog())
-     result.addTest(DiskFailure())
      return result
 

Modified: accumulo/branches/ACCUMULO-259/test/system/auto/stress/restart.py
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/test/system/auto/stress/restart.py?rev=1361382&r1=1361381&r2=1361382&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/test/system/auto/stress/restart.py (original)
+++ accumulo/branches/ACCUMULO-259/test/system/auto/stress/restart.py Fri Jul 13 20:34:44 2012
@@ -186,18 +186,6 @@ class ShutdownSplitter(TabletShouldSplit
                         os.path.join(ACCUMULO_HOME,'logs') ])
             )
 
-class RestartLoggerLate(KilledTabletServerTest):
-
-    def runTest(self):
-        self.waitForStop(self.ingester, 30)
-        self.stop_tserver(self.hosts[0])
-        self.stop_logger(self.hosts[0])
-        self.start_tserver(self.hosts[0])
-        self.sleep(15)
-        self.start_logger(self.hosts[0])
-        self.waitForStop(self.verify(self.masterHost(), self.options.rows), 100)
-        
-
 def suite():
     result = unittest.TestSuite()
     result.addTest(ShutdownSplitter())
@@ -207,6 +195,5 @@ def suite():
     result.addTest(RestartMasterTest())
     result.addTest(RestartMasterSplitterTest())
     result.addTest(KilledTabletServerTest())
-    result.addTest(RestartLoggerLate())
     result.addTest(KilledTabletServerSplitTest())
     return result

Modified: accumulo/branches/ACCUMULO-259/test/system/randomwalk/conf/modules/Concurrent.xml
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/test/system/randomwalk/conf/modules/Concurrent.xml?rev=1361382&r1=1361381&r2=1361382&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/test/system/randomwalk/conf/modules/Concurrent.xml (original)
+++ accumulo/branches/ACCUMULO-259/test/system/randomwalk/conf/modules/Concurrent.xml Fri Jul 13 20:34:44 2012
@@ -44,6 +44,8 @@
   <edge id="ct.ChangePermissions" weight="1000"/>
   <edge id="ct.CheckPermission" weight="1000"/>
   <edge id="ct.CheckBalance" weight="1000"/>
+  <edge id="ct.StopTabletServer" weight="1000"/>
+  <edge id="ct.StartAll" weight="1000"/>
   <edge id="END" weight="1"/>
 </node>
 
@@ -132,4 +134,12 @@
   <edge id="dummy.ToAll" weight="1"/>
 </node>
 
+<node id="ct.StopTabletServer">
+  <edge id="dummy.ToAll" weight="1"/>
+</node>
+
+<node id="ct.StartAll">
+  <edge id="dummy.ToAll" weight="1"/>
+</node>
+
 </module>

Modified: accumulo/branches/ACCUMULO-259/test/system/test4/bulk_import_test.sh
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/test/system/test4/bulk_import_test.sh?rev=1361382&r1=1361381&r2=1361382&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/test/system/test4/bulk_import_test.sh (original)
+++ accumulo/branches/ACCUMULO-259/test/system/test4/bulk_import_test.sh Fri Jul 13 20:34:44 2012
@@ -19,11 +19,11 @@ hadoop dfs -rmr /testmf
 
 echo "creating first set of map files"
 
-../../../bin/accumulo org.apache.accumulo.server.test.TestIngest -mapFile /testmf/mf01 -timestamp 1 -size 50 -random 56 1000000 0 1 &
-../../../bin/accumulo org.apache.accumulo.server.test.TestIngest -mapFile /testmf/mf02 -timestamp 1 -size 50 -random 56 1000000 1000000 1 &
-../../../bin/accumulo org.apache.accumulo.server.test.TestIngest -mapFile /testmf/mf03 -timestamp 1 -size 50 -random 56 1000000 2000000 1 &
-../../../bin/accumulo org.apache.accumulo.server.test.TestIngest -mapFile /testmf/mf04 -timestamp 1 -size 50 -random 56 1000000 3000000 1 &
-../../../bin/accumulo org.apache.accumulo.server.test.TestIngest -mapFile /testmf/mf05 -timestamp 1 -size 50 -random 56 1000000 4000000 1 &
+../../../bin/accumulo org.apache.accumulo.server.test.TestIngest -rfile /testmf/mf01 -timestamp 1 -size 50 -random 56 1000000 0 1 &
+../../../bin/accumulo org.apache.accumulo.server.test.TestIngest -rfile /testmf/mf02 -timestamp 1 -size 50 -random 56 1000000 1000000 1 &
+../../../bin/accumulo org.apache.accumulo.server.test.TestIngest -rfile /testmf/mf03 -timestamp 1 -size 50 -random 56 1000000 2000000 1 &
+../../../bin/accumulo org.apache.accumulo.server.test.TestIngest -rfile /testmf/mf04 -timestamp 1 -size 50 -random 56 1000000 3000000 1 &
+../../../bin/accumulo org.apache.accumulo.server.test.TestIngest -rfile /testmf/mf05 -timestamp 1 -size 50 -random 56 1000000 4000000 1 &
 
 wait
 
@@ -46,11 +46,11 @@ hadoop dfs -rmr /testmf
 
 echo "creating second set of map files"
 
-../../../bin/accumulo org.apache.accumulo.server.test.TestIngest -mapFile /testmf/mf01 -timestamp 2 -size 50 -random 57 1000000 0 1 &
-../../../bin/accumulo org.apache.accumulo.server.test.TestIngest -mapFile /testmf/mf02 -timestamp 2 -size 50 -random 57 1000000 1000000 1 &
-../../../bin/accumulo org.apache.accumulo.server.test.TestIngest -mapFile /testmf/mf03 -timestamp 2 -size 50 -random 57 1000000 2000000 1 &
-../../../bin/accumulo org.apache.accumulo.server.test.TestIngest -mapFile /testmf/mf04 -timestamp 2 -size 50 -random 57 1000000 3000000 1 &
-../../../bin/accumulo org.apache.accumulo.server.test.TestIngest -mapFile /testmf/mf05 -timestamp 2 -size 50 -random 57 1000000 4000000 1 &
+../../../bin/accumulo org.apache.accumulo.server.test.TestIngest -rfile /testmf/mf01 -timestamp 2 -size 50 -random 57 1000000 0 1 &
+../../../bin/accumulo org.apache.accumulo.server.test.TestIngest -rfile /testmf/mf02 -timestamp 2 -size 50 -random 57 1000000 1000000 1 &
+../../../bin/accumulo org.apache.accumulo.server.test.TestIngest -rfile /testmf/mf03 -timestamp 2 -size 50 -random 57 1000000 2000000 1 &
+../../../bin/accumulo org.apache.accumulo.server.test.TestIngest -rfile /testmf/mf04 -timestamp 2 -size 50 -random 57 1000000 3000000 1 &
+../../../bin/accumulo org.apache.accumulo.server.test.TestIngest -rfile /testmf/mf05 -timestamp 2 -size 50 -random 57 1000000 4000000 1 &
 
 wait
 

Modified: accumulo/branches/ACCUMULO-259/trace/pom.xml
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/trace/pom.xml?rev=1361382&r1=1361381&r2=1361382&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/trace/pom.xml (original)
+++ accumulo/branches/ACCUMULO-259/trace/pom.xml Fri Jul 13 20:34:44 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-259/trace/src/main/java/org/apache/accumulo/cloudtrace/instrument/thrift/TraceWrap.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/trace/src/main/java/org/apache/accumulo/cloudtrace/instrument/thrift/TraceWrap.java?rev=1361382&r1=1361381&r2=1361382&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/trace/src/main/java/org/apache/accumulo/cloudtrace/instrument/thrift/TraceWrap.java (original)
+++ accumulo/branches/ACCUMULO-259/trace/src/main/java/org/apache/accumulo/cloudtrace/instrument/thrift/TraceWrap.java Fri Jul 13 20:34:44 2012
@@ -25,6 +25,7 @@ import org.apache.accumulo.cloudtrace.in
 import org.apache.accumulo.cloudtrace.instrument.Trace;
 import org.apache.accumulo.cloudtrace.instrument.Tracer;
 import org.apache.accumulo.cloudtrace.thrift.TInfo;
+import org.apache.log4j.Logger;
 
 
 /**

Modified: accumulo/branches/ACCUMULO-259/trace/src/main/java/org/apache/accumulo/cloudtrace/thrift/RemoteSpan.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/trace/src/main/java/org/apache/accumulo/cloudtrace/thrift/RemoteSpan.java?rev=1361382&r1=1361381&r2=1361382&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/trace/src/main/java/org/apache/accumulo/cloudtrace/thrift/RemoteSpan.java (original)
+++ accumulo/branches/ACCUMULO-259/trace/src/main/java/org/apache/accumulo/cloudtrace/thrift/RemoteSpan.java Fri Jul 13 20:34:44 2012
@@ -1,12 +1,33 @@
 /**
- * Autogenerated by Thrift
+ * Autogenerated by Thrift Compiler (0.8.0)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
  */
 package org.apache.accumulo.cloudtrace.thrift;
 
-public class RemoteSpan implements org.apache.thrift.TBase<RemoteSpan, RemoteSpan._Fields>, java.io.Serializable, Cloneable {
-private static final long serialVersionUID = 1L;
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("all") public class RemoteSpan implements org.apache.thrift.TBase<RemoteSpan, RemoteSpan._Fields>, java.io.Serializable, Cloneable {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RemoteSpan");
 
   private static final org.apache.thrift.protocol.TField SENDER_FIELD_DESC = new org.apache.thrift.protocol.TField("sender", org.apache.thrift.protocol.TType.STRING, (short)1);
@@ -19,15 +40,21 @@ private static final long serialVersionU
   private static final org.apache.thrift.protocol.TField DESCRIPTION_FIELD_DESC = new org.apache.thrift.protocol.TField("description", org.apache.thrift.protocol.TType.STRING, (short)8);
   private static final org.apache.thrift.protocol.TField DATA_FIELD_DESC = new org.apache.thrift.protocol.TField("data", org.apache.thrift.protocol.TType.MAP, (short)9);
 
-  public String sender;
-  public String svc;
-  public long traceId;
-  public long spanId;
-  public long parentId;
-  public long start;
-  public long stop;
-  public String description;
-  public java.util.Map<String,String> data;
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new RemoteSpanStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new RemoteSpanTupleSchemeFactory());
+  }
+
+  public String sender; // required
+  public String svc; // required
+  public long traceId; // required
+  public long spanId; // required
+  public long parentId; // required
+  public long start; // required
+  public long stop; // required
+  public String description; // required
+  public Map<String,String> data; // required
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -41,10 +68,10 @@ private static final long serialVersionU
     DESCRIPTION((short)8, "description"),
     DATA((short)9, "data");
 
-    private static final java.util.Map<String, _Fields> byName = new java.util.HashMap<String, _Fields>();
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
     static {
-      for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
         byName.put(field.getFieldName(), field);
       }
     }
@@ -117,11 +144,10 @@ private static final long serialVersionU
   private static final int __PARENTID_ISSET_ID = 2;
   private static final int __START_ISSET_ID = 3;
   private static final int __STOP_ISSET_ID = 4;
-  private java.util.BitSet __isset_bit_vector = new java.util.BitSet(5);
-
-  public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  private BitSet __isset_bit_vector = new BitSet(5);
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
-    java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
     tmpMap.put(_Fields.SENDER, new org.apache.thrift.meta_data.FieldMetaData("sender", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.SVC, new org.apache.thrift.meta_data.FieldMetaData("svc", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -142,7 +168,7 @@ private static final long serialVersionU
         new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
-    metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(RemoteSpan.class, metaDataMap);
   }
 
@@ -158,7 +184,7 @@ private static final long serialVersionU
     long start,
     long stop,
     String description,
-    java.util.Map<String,String> data)
+    Map<String,String> data)
   {
     this();
     this.sender = sender;
@@ -198,8 +224,8 @@ private static final long serialVersionU
       this.description = other.description;
     }
     if (other.isSetData()) {
-      java.util.Map<String,String> __this__data = new java.util.HashMap<String,String>();
-      for (java.util.Map.Entry<String, String> other_element : other.data.entrySet()) {
+      Map<String,String> __this__data = new HashMap<String,String>();
+      for (Map.Entry<String, String> other_element : other.data.entrySet()) {
 
         String other_element_key = other_element.getKey();
         String other_element_value = other_element.getValue();
@@ -429,16 +455,16 @@ private static final long serialVersionU
 
   public void putToData(String key, String val) {
     if (this.data == null) {
-      this.data = new java.util.HashMap<String,String>();
+      this.data = new HashMap<String,String>();
     }
     this.data.put(key, val);
   }
 
-  public java.util.Map<String,String> getData() {
+  public Map<String,String> getData() {
     return this.data;
   }
 
-  public RemoteSpan setData(java.util.Map<String,String> data) {
+  public RemoteSpan setData(Map<String,String> data) {
     this.data = data;
     return this;
   }
@@ -528,9 +554,7 @@ private static final long serialVersionU
       if (value == null) {
         unsetData();
       } else {
-      @SuppressWarnings("unchecked")
-      java.util.Map<String,String> x = (java.util.Map<String,String>)value;
-        setData(x);
+        setData((Map<String,String>)value);
       }
       break;
 
@@ -546,19 +570,19 @@ private static final long serialVersionU
       return getSvc();
 
     case TRACE_ID:
-      return new Long(getTraceId());
+      return Long.valueOf(getTraceId());
 
     case SPAN_ID:
-      return new Long(getSpanId());
+      return Long.valueOf(getSpanId());
 
     case PARENT_ID:
-      return new Long(getParentId());
+      return Long.valueOf(getParentId());
 
     case START:
-      return new Long(getStart());
+      return Long.valueOf(getStart());
 
     case STOP:
-      return new Long(getStop());
+      return Long.valueOf(getStop());
 
     case DESCRIPTION:
       return getDescription();
@@ -807,155 +831,11 @@ private static final long serialVersionU
   }
 
   public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-    org.apache.thrift.protocol.TField field;
-    iprot.readStructBegin();
-    while (true)
-    {
-      field = iprot.readFieldBegin();
-      if (field.type == org.apache.thrift.protocol.TType.STOP) { 
-        break;
-      }
-      switch (field.id) {
-        case 1: // SENDER
-          if (field.type == org.apache.thrift.protocol.TType.STRING) {
-            this.sender = iprot.readString();
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 2: // SVC
-          if (field.type == org.apache.thrift.protocol.TType.STRING) {
-            this.svc = iprot.readString();
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 3: // TRACE_ID
-          if (field.type == org.apache.thrift.protocol.TType.I64) {
-            this.traceId = iprot.readI64();
-            setTraceIdIsSet(true);
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 4: // SPAN_ID
-          if (field.type == org.apache.thrift.protocol.TType.I64) {
-            this.spanId = iprot.readI64();
-            setSpanIdIsSet(true);
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 5: // PARENT_ID
-          if (field.type == org.apache.thrift.protocol.TType.I64) {
-            this.parentId = iprot.readI64();
-            setParentIdIsSet(true);
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 6: // START
-          if (field.type == org.apache.thrift.protocol.TType.I64) {
-            this.start = iprot.readI64();
-            setStartIsSet(true);
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 7: // STOP
-          if (field.type == org.apache.thrift.protocol.TType.I64) {
-            this.stop = iprot.readI64();
-            setStopIsSet(true);
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 8: // DESCRIPTION
-          if (field.type == org.apache.thrift.protocol.TType.STRING) {
-            this.description = iprot.readString();
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 9: // DATA
-          if (field.type == org.apache.thrift.protocol.TType.MAP) {
-            {
-              org.apache.thrift.protocol.TMap _map0 = iprot.readMapBegin();
-              this.data = new java.util.HashMap<String,String>(2*_map0.size);
-              for (int _i1 = 0; _i1 < _map0.size; ++_i1)
-              {
-                String _key2;
-                String _val3;
-                _key2 = iprot.readString();
-                _val3 = iprot.readString();
-                this.data.put(_key2, _val3);
-              }
-              iprot.readMapEnd();
-            }
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        default:
-          org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-      }
-      iprot.readFieldEnd();
-    }
-    iprot.readStructEnd();
-
-    // check for required fields of primitive type, which can't be checked in the validate method
-    validate();
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
   }
 
   public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
-    validate();
-
-    oprot.writeStructBegin(STRUCT_DESC);
-    if (this.sender != null) {
-      oprot.writeFieldBegin(SENDER_FIELD_DESC);
-      oprot.writeString(this.sender);
-      oprot.writeFieldEnd();
-    }
-    if (this.svc != null) {
-      oprot.writeFieldBegin(SVC_FIELD_DESC);
-      oprot.writeString(this.svc);
-      oprot.writeFieldEnd();
-    }
-    oprot.writeFieldBegin(TRACE_ID_FIELD_DESC);
-    oprot.writeI64(this.traceId);
-    oprot.writeFieldEnd();
-    oprot.writeFieldBegin(SPAN_ID_FIELD_DESC);
-    oprot.writeI64(this.spanId);
-    oprot.writeFieldEnd();
-    oprot.writeFieldBegin(PARENT_ID_FIELD_DESC);
-    oprot.writeI64(this.parentId);
-    oprot.writeFieldEnd();
-    oprot.writeFieldBegin(START_FIELD_DESC);
-    oprot.writeI64(this.start);
-    oprot.writeFieldEnd();
-    oprot.writeFieldBegin(STOP_FIELD_DESC);
-    oprot.writeI64(this.stop);
-    oprot.writeFieldEnd();
-    if (this.description != null) {
-      oprot.writeFieldBegin(DESCRIPTION_FIELD_DESC);
-      oprot.writeString(this.description);
-      oprot.writeFieldEnd();
-    }
-    if (this.data != null) {
-      oprot.writeFieldBegin(DATA_FIELD_DESC);
-      {
-        oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, this.data.size()));
-        for (java.util.Map.Entry<String, String> _iter4 : this.data.entrySet())
-        {
-          oprot.writeString(_iter4.getKey());
-          oprot.writeString(_iter4.getValue());
-        }
-        oprot.writeMapEnd();
-      }
-      oprot.writeFieldEnd();
-    }
-    oprot.writeFieldStop();
-    oprot.writeStructEnd();
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
   }
 
   @Override
@@ -1033,12 +913,308 @@ private static final long serialVersionU
   private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
     try {
       // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
-      __isset_bit_vector = new java.util.BitSet(1);
+      __isset_bit_vector = new BitSet(1);
       read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
     } catch (org.apache.thrift.TException te) {
       throw new java.io.IOException(te);
     }
   }
 
+  private static class RemoteSpanStandardSchemeFactory implements SchemeFactory {
+    public RemoteSpanStandardScheme getScheme() {
+      return new RemoteSpanStandardScheme();
+    }
+  }
+
+  private static class RemoteSpanStandardScheme extends StandardScheme<RemoteSpan> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, RemoteSpan struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // SENDER
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.sender = iprot.readString();
+              struct.setSenderIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // SVC
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.svc = iprot.readString();
+              struct.setSvcIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // TRACE_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.traceId = iprot.readI64();
+              struct.setTraceIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // SPAN_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.spanId = iprot.readI64();
+              struct.setSpanIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // PARENT_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.parentId = iprot.readI64();
+              struct.setParentIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // START
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.start = iprot.readI64();
+              struct.setStartIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // STOP
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.stop = iprot.readI64();
+              struct.setStopIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 8: // DESCRIPTION
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.description = iprot.readString();
+              struct.setDescriptionIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 9: // DATA
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map0 = iprot.readMapBegin();
+                struct.data = new HashMap<String,String>(2*_map0.size);
+                for (int _i1 = 0; _i1 < _map0.size; ++_i1)
+                {
+                  String _key2; // required
+                  String _val3; // required
+                  _key2 = iprot.readString();
+                  _val3 = iprot.readString();
+                  struct.data.put(_key2, _val3);
+                }
+                iprot.readMapEnd();
+              }
+              struct.setDataIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+
+      // check for required fields of primitive type, which can't be checked in the validate method
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, RemoteSpan struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.sender != null) {
+        oprot.writeFieldBegin(SENDER_FIELD_DESC);
+        oprot.writeString(struct.sender);
+        oprot.writeFieldEnd();
+      }
+      if (struct.svc != null) {
+        oprot.writeFieldBegin(SVC_FIELD_DESC);
+        oprot.writeString(struct.svc);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(TRACE_ID_FIELD_DESC);
+      oprot.writeI64(struct.traceId);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(SPAN_ID_FIELD_DESC);
+      oprot.writeI64(struct.spanId);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(PARENT_ID_FIELD_DESC);
+      oprot.writeI64(struct.parentId);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(START_FIELD_DESC);
+      oprot.writeI64(struct.start);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(STOP_FIELD_DESC);
+      oprot.writeI64(struct.stop);
+      oprot.writeFieldEnd();
+      if (struct.description != null) {
+        oprot.writeFieldBegin(DESCRIPTION_FIELD_DESC);
+        oprot.writeString(struct.description);
+        oprot.writeFieldEnd();
+      }
+      if (struct.data != null) {
+        oprot.writeFieldBegin(DATA_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.data.size()));
+          for (Map.Entry<String, String> _iter4 : struct.data.entrySet())
+          {
+            oprot.writeString(_iter4.getKey());
+            oprot.writeString(_iter4.getValue());
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class RemoteSpanTupleSchemeFactory implements SchemeFactory {
+    public RemoteSpanTupleScheme getScheme() {
+      return new RemoteSpanTupleScheme();
+    }
+  }
+
+  private static class RemoteSpanTupleScheme extends TupleScheme<RemoteSpan> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, RemoteSpan struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetSender()) {
+        optionals.set(0);
+      }
+      if (struct.isSetSvc()) {
+        optionals.set(1);
+      }
+      if (struct.isSetTraceId()) {
+        optionals.set(2);
+      }
+      if (struct.isSetSpanId()) {
+        optionals.set(3);
+      }
+      if (struct.isSetParentId()) {
+        optionals.set(4);
+      }
+      if (struct.isSetStart()) {
+        optionals.set(5);
+      }
+      if (struct.isSetStop()) {
+        optionals.set(6);
+      }
+      if (struct.isSetDescription()) {
+        optionals.set(7);
+      }
+      if (struct.isSetData()) {
+        optionals.set(8);
+      }
+      oprot.writeBitSet(optionals, 9);
+      if (struct.isSetSender()) {
+        oprot.writeString(struct.sender);
+      }
+      if (struct.isSetSvc()) {
+        oprot.writeString(struct.svc);
+      }
+      if (struct.isSetTraceId()) {
+        oprot.writeI64(struct.traceId);
+      }
+      if (struct.isSetSpanId()) {
+        oprot.writeI64(struct.spanId);
+      }
+      if (struct.isSetParentId()) {
+        oprot.writeI64(struct.parentId);
+      }
+      if (struct.isSetStart()) {
+        oprot.writeI64(struct.start);
+      }
+      if (struct.isSetStop()) {
+        oprot.writeI64(struct.stop);
+      }
+      if (struct.isSetDescription()) {
+        oprot.writeString(struct.description);
+      }
+      if (struct.isSetData()) {
+        {
+          oprot.writeI32(struct.data.size());
+          for (Map.Entry<String, String> _iter5 : struct.data.entrySet())
+          {
+            oprot.writeString(_iter5.getKey());
+            oprot.writeString(_iter5.getValue());
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, RemoteSpan struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(9);
+      if (incoming.get(0)) {
+        struct.sender = iprot.readString();
+        struct.setSenderIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.svc = iprot.readString();
+        struct.setSvcIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.traceId = iprot.readI64();
+        struct.setTraceIdIsSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.spanId = iprot.readI64();
+        struct.setSpanIdIsSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.parentId = iprot.readI64();
+        struct.setParentIdIsSet(true);
+      }
+      if (incoming.get(5)) {
+        struct.start = iprot.readI64();
+        struct.setStartIsSet(true);
+      }
+      if (incoming.get(6)) {
+        struct.stop = iprot.readI64();
+        struct.setStopIsSet(true);
+      }
+      if (incoming.get(7)) {
+        struct.description = iprot.readString();
+        struct.setDescriptionIsSet(true);
+      }
+      if (incoming.get(8)) {
+        {
+          org.apache.thrift.protocol.TMap _map6 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.data = new HashMap<String,String>(2*_map6.size);
+          for (int _i7 = 0; _i7 < _map6.size; ++_i7)
+          {
+            String _key8; // required
+            String _val9; // required
+            _key8 = iprot.readString();
+            _val9 = iprot.readString();
+            struct.data.put(_key8, _val9);
+          }
+        }
+        struct.setDataIsSet(true);
+      }
+    }
+  }
+
 }
 

Modified: accumulo/branches/ACCUMULO-259/trace/src/main/java/org/apache/accumulo/cloudtrace/thrift/SpanReceiver.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/trace/src/main/java/org/apache/accumulo/cloudtrace/thrift/SpanReceiver.java?rev=1361382&r1=1361381&r2=1361382&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/trace/src/main/java/org/apache/accumulo/cloudtrace/thrift/SpanReceiver.java (original)
+++ accumulo/branches/ACCUMULO-259/trace/src/main/java/org/apache/accumulo/cloudtrace/thrift/SpanReceiver.java Fri Jul 13 20:34:44 2012
@@ -1,11 +1,33 @@
 /**
- * Autogenerated by Thrift
+ * Autogenerated by Thrift Compiler (0.8.0)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
  */
 package org.apache.accumulo.cloudtrace.thrift;
 
-public class SpanReceiver {
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("all") public class SpanReceiver {
 
   public interface Iface {
 
@@ -19,7 +41,7 @@ public class SpanReceiver {
 
   }
 
-  public static class Client implements org.apache.thrift.TServiceClient, Iface {
+  public static class Client extends org.apache.thrift.TServiceClient implements Iface {
     public static class Factory implements org.apache.thrift.TServiceClientFactory<Client> {
       public Factory() {}
       public Client getClient(org.apache.thrift.protocol.TProtocol prot) {
@@ -32,28 +54,11 @@ public class SpanReceiver {
 
     public Client(org.apache.thrift.protocol.TProtocol prot)
     {
-      this(prot, prot);
-    }
-
-    public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot)
-    {
-      iprot_ = iprot;
-      oprot_ = oprot;
+      super(prot, prot);
     }
 
-    protected org.apache.thrift.protocol.TProtocol iprot_;
-    protected org.apache.thrift.protocol.TProtocol oprot_;
-
-    protected int seqid_;
-
-    public org.apache.thrift.protocol.TProtocol getInputProtocol()
-    {
-      return this.iprot_;
-    }
-
-    public org.apache.thrift.protocol.TProtocol getOutputProtocol()
-    {
-      return this.oprot_;
+    public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
+      super(iprot, oprot);
     }
 
     public void span(RemoteSpan span) throws org.apache.thrift.TException
@@ -63,12 +68,9 @@ public class SpanReceiver {
 
     public void send_span(RemoteSpan span) throws org.apache.thrift.TException
     {
-      oprot_.writeMessageBegin(new org.apache.thrift.protocol.TMessage("span", org.apache.thrift.protocol.TMessageType.CALL, ++seqid_));
       span_args args = new span_args();
       args.setSpan(span);
-      args.write(oprot_);
-      oprot_.writeMessageEnd();
-      oprot_.getTransport().flush();
+      sendBase("span", args);
     }
 
   }
@@ -91,15 +93,13 @@ public class SpanReceiver {
 
     public void span(RemoteSpan span, org.apache.thrift.async.AsyncMethodCallback<span_call> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      span_call method_call = new span_call(span, resultHandler, this, protocolFactory, transport);
-      this.currentMethod = method_call;
-      manager.call(method_call);
+      span_call method_call = new span_call(span, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
     }
 
-    @SuppressWarnings("rawtypes")
     public static class span_call extends org.apache.thrift.async.TAsyncMethodCall {
       private RemoteSpan span;
-      @SuppressWarnings("unchecked")
       public span_call(RemoteSpan span, org.apache.thrift.async.AsyncMethodCallback<span_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, true);
         this.span = span;
@@ -117,83 +117,66 @@ public class SpanReceiver {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
       }
     }
 
   }
 
-  public static class Processor implements org.apache.thrift.TProcessor {
-    private static final org.slf4j.Logger LOGGER = org.slf4j.LoggerFactory.getLogger(Processor.class.getName());
-    public Processor(Iface iface)
-    {
-      iface_ = iface;
-      processMap_.put("span", new span());
+  public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
+    private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName());
+    public Processor(I iface) {
+      super(iface, getProcessMap(new HashMap<String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
     }
 
-    protected static interface ProcessFunction {
-      public void process(int seqid, org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException;
+    protected Processor(I iface, Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
+      super(iface, getProcessMap(processMap));
     }
 
-    private Iface iface_;
-    protected final java.util.HashMap<String,ProcessFunction> processMap_ = new java.util.HashMap<String,ProcessFunction>();
+    private static <I extends Iface> Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> getProcessMap(Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
+      processMap.put("span", new span());
+      return processMap;
+    }
 
-    public boolean process(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException
-    {
-      org.apache.thrift.protocol.TMessage msg = iprot.readMessageBegin();
-      ProcessFunction fn = processMap_.get(msg.name);
-      if (fn == null) {
-        org.apache.thrift.protocol.TProtocolUtil.skip(iprot, org.apache.thrift.protocol.TType.STRUCT);
-        iprot.readMessageEnd();
-        org.apache.thrift.TApplicationException x = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.UNKNOWN_METHOD, "Invalid method name: '"+msg.name+"'");
-        oprot.writeMessageBegin(new org.apache.thrift.protocol.TMessage(msg.name, org.apache.thrift.protocol.TMessageType.EXCEPTION, msg.seqid));
-        x.write(oprot);
-        oprot.writeMessageEnd();
-        oprot.getTransport().flush();
-        return true;
+    private static class span<I extends Iface> extends org.apache.thrift.ProcessFunction<I, span_args> {
+      public span() {
+        super("span");
       }
-      fn.process(msg.seqid, iprot, oprot);
-      return true;
-    }
 
-    private class span implements ProcessFunction {
-      public void process(int seqid, org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException
-      {
-        span_args args = new span_args();
-        try {
-          args.read(iprot);
-        } catch (org.apache.thrift.protocol.TProtocolException e) {
-          iprot.readMessageEnd();
-          org.apache.thrift.TApplicationException x = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.PROTOCOL_ERROR, e.getMessage());
-          oprot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("span", org.apache.thrift.protocol.TMessageType.EXCEPTION, seqid));
-          x.write(oprot);
-          oprot.writeMessageEnd();
-          oprot.getTransport().flush();
-          return;
-        }
-        iprot.readMessageEnd();
-        iface_.span(args.span);
-        return;
+      protected span_args getEmptyArgsInstance() {
+        return new span_args();
+      }
+
+      protected org.apache.thrift.TBase getResult(I iface, span_args args) throws org.apache.thrift.TException {
+        iface.span(args.span);
+        return null;
       }
     }
 
   }
 
   public static class span_args implements org.apache.thrift.TBase<span_args, span_args._Fields>, java.io.Serializable, Cloneable   {
-private static final long serialVersionUID = 1L;
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("span_args");
 
     private static final org.apache.thrift.protocol.TField SPAN_FIELD_DESC = new org.apache.thrift.protocol.TField("span", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    public RemoteSpan span;
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new span_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new span_argsTupleSchemeFactory());
+    }
+
+    public RemoteSpan span; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       SPAN((short)1, "span");
 
-      private static final java.util.Map<String, _Fields> byName = new java.util.HashMap<String, _Fields>();
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
       static {
-        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
           byName.put(field.getFieldName(), field);
         }
       }
@@ -245,13 +228,12 @@ private static final long serialVersionU
     }
 
     // isset id assignments
-
-    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
-      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.SPAN, new org.apache.thrift.meta_data.FieldMetaData("span", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, RemoteSpan.class)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(span_args.class, metaDataMap);
     }
 
@@ -398,56 +380,25 @@ private static final long serialVersionU
     }
 
     public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-      org.apache.thrift.protocol.TField field;
-      iprot.readStructBegin();
-      while (true)
-      {
-        field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
-          break;
-        }
-        switch (field.id) {
-          case 1: // SPAN
-            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
-              this.span = new RemoteSpan();
-              this.span.read(iprot);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
-          default:
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-        }
-        iprot.readFieldEnd();
-      }
-      iprot.readStructEnd();
-
-      // check for required fields of primitive type, which can't be checked in the validate method
-      validate();
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
     }
 
     public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
-      validate();
-
-      oprot.writeStructBegin(STRUCT_DESC);
-      if (this.span != null) {
-        oprot.writeFieldBegin(SPAN_FIELD_DESC);
-        this.span.write(oprot);
-        oprot.writeFieldEnd();
-      }
-      oprot.writeFieldStop();
-      oprot.writeStructEnd();
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
     }
 
     @Override
     public String toString() {
       StringBuilder sb = new StringBuilder("span_args(");
+      boolean first = true;
+
       sb.append("span:");
       if (this.span == null) {
         sb.append("null");
       } else {
         sb.append(this.span);
       }
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -472,6 +423,92 @@ private static final long serialVersionU
       }
     }
 
+    private static class span_argsStandardSchemeFactory implements SchemeFactory {
+      public span_argsStandardScheme getScheme() {
+        return new span_argsStandardScheme();
+      }
+    }
+
+    private static class span_argsStandardScheme extends StandardScheme<span_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, span_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // SPAN
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.span = new RemoteSpan();
+                struct.span.read(iprot);
+                struct.setSpanIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, span_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.span != null) {
+          oprot.writeFieldBegin(SPAN_FIELD_DESC);
+          struct.span.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class span_argsTupleSchemeFactory implements SchemeFactory {
+      public span_argsTupleScheme getScheme() {
+        return new span_argsTupleScheme();
+      }
+    }
+
+    private static class span_argsTupleScheme extends TupleScheme<span_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, span_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetSpan()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSpan()) {
+          struct.span.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, span_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.span = new RemoteSpan();
+          struct.span.read(iprot);
+          struct.setSpanIsSet(true);
+        }
+      }
+    }
+
   }
 
 }

Modified: accumulo/branches/ACCUMULO-259/trace/src/main/java/org/apache/accumulo/cloudtrace/thrift/TInfo.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/trace/src/main/java/org/apache/accumulo/cloudtrace/thrift/TInfo.java?rev=1361382&r1=1361381&r2=1361382&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/trace/src/main/java/org/apache/accumulo/cloudtrace/thrift/TInfo.java (original)
+++ accumulo/branches/ACCUMULO-259/trace/src/main/java/org/apache/accumulo/cloudtrace/thrift/TInfo.java Fri Jul 13 20:34:44 2012
@@ -1,29 +1,56 @@
 /**
- * Autogenerated by Thrift
+ * Autogenerated by Thrift Compiler (0.8.0)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
  */
 package org.apache.accumulo.cloudtrace.thrift;
 
-public class TInfo implements org.apache.thrift.TBase<TInfo, TInfo._Fields>, java.io.Serializable, Cloneable {
-private static final long serialVersionUID = 1L;
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("all") public class TInfo implements org.apache.thrift.TBase<TInfo, TInfo._Fields>, java.io.Serializable, Cloneable {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TInfo");
 
   private static final org.apache.thrift.protocol.TField TRACE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("traceId", org.apache.thrift.protocol.TType.I64, (short)1);
   private static final org.apache.thrift.protocol.TField PARENT_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("parentId", org.apache.thrift.protocol.TType.I64, (short)2);
 
-  public long traceId;
-  public long parentId;
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TInfoStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TInfoTupleSchemeFactory());
+  }
+
+  public long traceId; // required
+  public long parentId; // required
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     TRACE_ID((short)1, "traceId"),
     PARENT_ID((short)2, "parentId");
 
-    private static final java.util.Map<String, _Fields> byName = new java.util.HashMap<String, _Fields>();
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
     static {
-      for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
         byName.put(field.getFieldName(), field);
       }
     }
@@ -79,16 +106,15 @@ private static final long serialVersionU
   // isset id assignments
   private static final int __TRACEID_ISSET_ID = 0;
   private static final int __PARENTID_ISSET_ID = 1;
-  private java.util.BitSet __isset_bit_vector = new java.util.BitSet(2);
-
-  public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  private BitSet __isset_bit_vector = new BitSet(2);
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
-    java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
     tmpMap.put(_Fields.TRACE_ID, new org.apache.thrift.meta_data.FieldMetaData("traceId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     tmpMap.put(_Fields.PARENT_ID, new org.apache.thrift.meta_data.FieldMetaData("parentId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
-    metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TInfo.class, metaDataMap);
   }
 
@@ -198,10 +224,10 @@ private static final long serialVersionU
   public Object getFieldValue(_Fields field) {
     switch (field) {
     case TRACE_ID:
-      return new Long(getTraceId());
+      return Long.valueOf(getTraceId());
 
     case PARENT_ID:
-      return new Long(getParentId());
+      return Long.valueOf(getParentId());
 
     }
     throw new IllegalStateException();
@@ -297,54 +323,11 @@ private static final long serialVersionU
   }
 
   public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-    org.apache.thrift.protocol.TField field;
-    iprot.readStructBegin();
-    while (true)
-    {
-      field = iprot.readFieldBegin();
-      if (field.type == org.apache.thrift.protocol.TType.STOP) { 
-        break;
-      }
-      switch (field.id) {
-        case 1: // TRACE_ID
-          if (field.type == org.apache.thrift.protocol.TType.I64) {
-            this.traceId = iprot.readI64();
-            setTraceIdIsSet(true);
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 2: // PARENT_ID
-          if (field.type == org.apache.thrift.protocol.TType.I64) {
-            this.parentId = iprot.readI64();
-            setParentIdIsSet(true);
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        default:
-          org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-      }
-      iprot.readFieldEnd();
-    }
-    iprot.readStructEnd();
-
-    // check for required fields of primitive type, which can't be checked in the validate method
-    validate();
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
   }
 
   public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
-    validate();
-
-    oprot.writeStructBegin(STRUCT_DESC);
-    oprot.writeFieldBegin(TRACE_ID_FIELD_DESC);
-    oprot.writeI64(this.traceId);
-    oprot.writeFieldEnd();
-    oprot.writeFieldBegin(PARENT_ID_FIELD_DESC);
-    oprot.writeI64(this.parentId);
-    oprot.writeFieldEnd();
-    oprot.writeFieldStop();
-    oprot.writeStructEnd();
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
   }
 
   @Override
@@ -378,12 +361,115 @@ private static final long serialVersionU
   private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
     try {
       // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
-      __isset_bit_vector = new java.util.BitSet(1);
+      __isset_bit_vector = new BitSet(1);
       read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
     } catch (org.apache.thrift.TException te) {
       throw new java.io.IOException(te);
     }
   }
 
+  private static class TInfoStandardSchemeFactory implements SchemeFactory {
+    public TInfoStandardScheme getScheme() {
+      return new TInfoStandardScheme();
+    }
+  }
+
+  private static class TInfoStandardScheme extends StandardScheme<TInfo> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TInfo struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TRACE_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.traceId = iprot.readI64();
+              struct.setTraceIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // PARENT_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.parentId = iprot.readI64();
+              struct.setParentIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+
+      // check for required fields of primitive type, which can't be checked in the validate method
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TInfo struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(TRACE_ID_FIELD_DESC);
+      oprot.writeI64(struct.traceId);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(PARENT_ID_FIELD_DESC);
+      oprot.writeI64(struct.parentId);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TInfoTupleSchemeFactory implements SchemeFactory {
+    public TInfoTupleScheme getScheme() {
+      return new TInfoTupleScheme();
+    }
+  }
+
+  private static class TInfoTupleScheme extends TupleScheme<TInfo> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetTraceId()) {
+        optionals.set(0);
+      }
+      if (struct.isSetParentId()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetTraceId()) {
+        oprot.writeI64(struct.traceId);
+      }
+      if (struct.isSetParentId()) {
+        oprot.writeI64(struct.parentId);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        struct.traceId = iprot.readI64();
+        struct.setTraceIdIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.parentId = iprot.readI64();
+        struct.setParentIdIsSet(true);
+      }
+    }
+  }
+
 }