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 2013/04/30 20:33:09 UTC

svn commit: r1477758 [1/6] - in /accumulo/trunk: core/src/main/java/org/apache/accumulo/core/client/admin/ core/src/main/java/org/apache/accumulo/core/client/mock/ core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/ core/src/main/java/org/...

Author: ecn
Date: Tue Apr 30 18:33:06 2013
New Revision: 1477758

URL: http://svn.apache.org/r1477758
Log:
ACCUMULO-1335 commiting Corey Nolet's patch to add DiskUsage to the API

Added:
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/DiskUsage.java   (with props)
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/DiskUsage.java   (with props)
    accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/thrift/DiskUsage.java   (with props)
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/TableOperationsIT.java   (with props)
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/TabletServerIT.java   (with props)
Modified:
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/ActiveCompaction.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/IteratorConfig.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TIteratorSetting.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/TableDiskUsage.java
    accumulo/trunk/core/src/main/thrift/tabletserver.thrift
    accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java
    accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
    accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/thrift/AccumuloProxy.java
    accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ActiveCompaction.java
    accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ActiveScan.java
    accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/thrift/BatchScanOptions.java
    accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/thrift/IteratorSetting.java
    accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ScanOptions.java
    accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ScanResult.java
    accumulo/trunk/proxy/src/main/thrift/proxy.thrift
    accumulo/trunk/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
    accumulo/trunk/test/src/test/resources/log4j.properties

Added: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/DiskUsage.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/DiskUsage.java?rev=1477758&view=auto
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/DiskUsage.java (added)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/DiskUsage.java Tue Apr 30 18:33:06 2013
@@ -0,0 +1,67 @@
+/*
+ * 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.core.client.admin;
+
+import java.util.SortedSet;
+
+public class DiskUsage {
+  
+  protected final SortedSet<String> tables;
+  protected Long usage;
+  
+  public DiskUsage(SortedSet<String> tables, Long usage) {
+    this.tables = tables;
+    this.usage = usage;
+  }
+  
+  public SortedSet<String> getTables() {
+    return tables;
+  }
+  
+  public Long getUsage() {
+    return usage;
+  }
+  
+  @Override
+  public boolean equals(Object o) {
+    if (this == o)
+      return true;
+    if (!(o instanceof DiskUsage))
+      return false;
+    
+    DiskUsage diskUsage = (DiskUsage) o;
+    
+    if (tables != null ? !tables.equals(diskUsage.tables) : diskUsage.tables != null)
+      return false;
+    if (usage != null ? !usage.equals(diskUsage.usage) : diskUsage.usage != null)
+      return false;
+    
+    return true;
+  }
+  
+  @Override
+  public int hashCode() {
+    int result = tables != null ? tables.hashCode() : 0;
+    result = 31 * result + (usage != null ? usage.hashCode() : 0);
+    return result;
+  }
+  
+  @Override
+  public String toString() {
+    return "DiskUsage{" + "tables=" + tables + ", usage=" + usage + '}';
+  }
+}

Propchange: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/DiskUsage.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java?rev=1477758&r1=1477757&r2=1477758&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java Tue Apr 30 18:33:06 2013
@@ -24,6 +24,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedSet;
+import java.util.TreeSet;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -680,4 +681,17 @@ public interface TableOperations {
    * @since 1.5.0
    */
   public Map<String,Integer> listConstraints(String tableName) throws AccumuloException, TableNotFoundException;
+
+
+  /**
+   * Gets the number of bytes being used in the files for a set of tables
+   *
+   * @param tables
+   *          a set of tables
+   * @return  a list of disk usage objects containing linked table names and sizes
+   * @throws AccumuloException
+   * @throws AccumuloSecurityException
+   */
+  public List<DiskUsage> getDiskUsage(Set<String> tables) throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
+
 }

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java?rev=1477758&r1=1477757&r2=1477758&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java Tue Apr 30 18:33:06 2013
@@ -30,6 +30,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Random;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeMap;
@@ -45,6 +46,7 @@ import java.util.zip.ZipInputStream;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
@@ -538,7 +540,7 @@ public class TableOperationsImpl extends
     
     SortedSet<KeyExtent> tablets = new TreeSet<KeyExtent>();
     Map<KeyExtent,String> locations = new TreeMap<KeyExtent,String>();
-
+    
     while (true) {
       try {
         tablets.clear();
@@ -552,11 +554,11 @@ public class TableOperationsImpl extends
         if (!Tables.exists(instance, tableId)) {
           throw new TableNotFoundException(tableId, tableName, null);
         }
-
+        
         if (t instanceof RuntimeException && t.getCause() instanceof AccumuloSecurityException) {
           throw (AccumuloSecurityException) t.getCause();
         }
-
+        
         log.info(t.getMessage() + " ... retrying ...");
         UtilWaitThread.sleep(3000);
       }
@@ -579,7 +581,7 @@ public class TableOperationsImpl extends
       throw new RuntimeException(e);
     }
   }
-
+  
   /**
    * @param tableName
    *          the name of the table
@@ -621,7 +623,7 @@ public class TableOperationsImpl extends
       throw new RuntimeException(e);
     }
   }
-
+  
   /**
    * Delete a table
    * 
@@ -1096,7 +1098,7 @@ public class TableOperationsImpl extends
         throw new AccumuloException("Bulk import directory " + failPath + " is a file");
       throw new AccumuloException("Bulk import failure directory " + failPath + " is not empty");
     }
-
+    
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes()), ByteBuffer.wrap(dirPath.toString().getBytes()),
         ByteBuffer.wrap(failPath.toString().getBytes()), ByteBuffer.wrap((setTime + "").getBytes()));
     Map<String,String> opts = new HashMap<String,String>();
@@ -1193,6 +1195,40 @@ public class TableOperationsImpl extends
     return FindMax.findMax(scanner, startRow, startInclusive, endRow, endInclusive);
   }
   
+  @Override
+  public List<DiskUsage> getDiskUsage(Set<String> tables) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    
+    Random rand = new Random();
+    
+    Connector conn = instance.getConnector(credentials.getPrincipal(), CredentialHelper.extractToken(credentials));
+    List<String> tservers = conn.instanceOperations().getTabletServers();
+
+    List<Integer> triedServers = new ArrayList<Integer>();
+    List<org.apache.accumulo.core.tabletserver.thrift.DiskUsage> diskUsages = null;
+    while (diskUsages == null && triedServers.size() < tservers.size()) {
+      try {
+        int randServer = rand.nextInt(tservers.size());
+        while (triedServers.contains(randServer))
+          randServer = rand.nextInt();
+        TabletClientService.Client client = ThriftUtil.getTServerClient(tservers.get(randServer), instance.getConfiguration());
+        diskUsages = client.getDiskUsage(tables, credentials);
+      } catch (ThriftTableOperationException e) {
+        throw new TableNotFoundException(e.getTableId(), e.getTableName(), e.getDescription(), e.getCause());
+      } catch (ThriftSecurityException e) {
+        throw new AccumuloSecurityException(e.getUser(), e.getCode());
+      } catch (TException e) {
+        // If failure was not related to above exceptions, we should choose a new server and try again
+      }
+    }
+
+    List<DiskUsage> finalUsages = new ArrayList<DiskUsage>();
+    for (org.apache.accumulo.core.tabletserver.thrift.DiskUsage diskUsage : diskUsages) {
+      finalUsages.add(new DiskUsage(new TreeSet<String>(diskUsage.getTables()), diskUsage.getUsage()));
+    }
+    
+    return finalUsages;
+  }
+  
   public static Map<String,String> getExportedProps(FileSystem fs, Path path) throws IOException {
     HashMap<String,String> props = new HashMap<String,String>();
     

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java?rev=1477758&r1=1477757&r2=1477758&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java Tue Apr 30 18:33:06 2013
@@ -18,6 +18,7 @@ package org.apache.accumulo.core.client.
 
 import java.io.DataInputStream;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -34,6 +35,7 @@ import org.apache.accumulo.core.client.A
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.DiskUsage;
 import org.apache.accumulo.core.client.admin.FindMax;
 import org.apache.accumulo.core.client.admin.TableOperationsHelper;
 import org.apache.accumulo.core.client.admin.TimeType;
@@ -298,7 +300,20 @@ public class MockTableOperations extends
     }
     return result;
   }
-  
+
+  @Override
+  public List<DiskUsage> getDiskUsage(Set<String> tables) throws AccumuloException, AccumuloSecurityException {
+
+    List<DiskUsage> diskUsages = new ArrayList<DiskUsage>();
+    for(String table : tables) {
+      TreeSet<String> tree = new TreeSet<String>();
+      tree.add(table);
+      diskUsages.add(new DiskUsage(tree, 1l));
+    }
+
+    return diskUsages;
+  }
+
   @Override
   public void merge(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     if (!exists(tableName))

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/ActiveCompaction.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/ActiveCompaction.java?rev=1477758&r1=1477757&r2=1477758&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/ActiveCompaction.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/ActiveCompaction.java Tue Apr 30 18:33:06 2013
@@ -1227,13 +1227,13 @@ import org.slf4j.LoggerFactory;
           case 3: // INPUT_FILES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list52 = iprot.readListBegin();
-                struct.inputFiles = new ArrayList<String>(_list52.size);
-                for (int _i53 = 0; _i53 < _list52.size; ++_i53)
+                org.apache.thrift.protocol.TList _list60 = iprot.readListBegin();
+                struct.inputFiles = new ArrayList<String>(_list60.size);
+                for (int _i61 = 0; _i61 < _list60.size; ++_i61)
                 {
-                  String _elem54; // required
-                  _elem54 = iprot.readString();
-                  struct.inputFiles.add(_elem54);
+                  String _elem62; // required
+                  _elem62 = iprot.readString();
+                  struct.inputFiles.add(_elem62);
                 }
                 iprot.readListEnd();
               }
@@ -1293,14 +1293,14 @@ import org.slf4j.LoggerFactory;
           case 10: // SSI_LIST
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list55 = iprot.readListBegin();
-                struct.ssiList = new ArrayList<org.apache.accumulo.core.data.thrift.IterInfo>(_list55.size);
-                for (int _i56 = 0; _i56 < _list55.size; ++_i56)
+                org.apache.thrift.protocol.TList _list63 = iprot.readListBegin();
+                struct.ssiList = new ArrayList<org.apache.accumulo.core.data.thrift.IterInfo>(_list63.size);
+                for (int _i64 = 0; _i64 < _list63.size; ++_i64)
                 {
-                  org.apache.accumulo.core.data.thrift.IterInfo _elem57; // required
-                  _elem57 = new org.apache.accumulo.core.data.thrift.IterInfo();
-                  _elem57.read(iprot);
-                  struct.ssiList.add(_elem57);
+                  org.apache.accumulo.core.data.thrift.IterInfo _elem65; // required
+                  _elem65 = new org.apache.accumulo.core.data.thrift.IterInfo();
+                  _elem65.read(iprot);
+                  struct.ssiList.add(_elem65);
                 }
                 iprot.readListEnd();
               }
@@ -1312,27 +1312,27 @@ import org.slf4j.LoggerFactory;
           case 11: // SSIO
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map58 = iprot.readMapBegin();
-                struct.ssio = new HashMap<String,Map<String,String>>(2*_map58.size);
-                for (int _i59 = 0; _i59 < _map58.size; ++_i59)
+                org.apache.thrift.protocol.TMap _map66 = iprot.readMapBegin();
+                struct.ssio = new HashMap<String,Map<String,String>>(2*_map66.size);
+                for (int _i67 = 0; _i67 < _map66.size; ++_i67)
                 {
-                  String _key60; // required
-                  Map<String,String> _val61; // required
-                  _key60 = iprot.readString();
+                  String _key68; // required
+                  Map<String,String> _val69; // required
+                  _key68 = iprot.readString();
                   {
-                    org.apache.thrift.protocol.TMap _map62 = iprot.readMapBegin();
-                    _val61 = new HashMap<String,String>(2*_map62.size);
-                    for (int _i63 = 0; _i63 < _map62.size; ++_i63)
+                    org.apache.thrift.protocol.TMap _map70 = iprot.readMapBegin();
+                    _val69 = new HashMap<String,String>(2*_map70.size);
+                    for (int _i71 = 0; _i71 < _map70.size; ++_i71)
                     {
-                      String _key64; // required
-                      String _val65; // required
-                      _key64 = iprot.readString();
-                      _val65 = iprot.readString();
-                      _val61.put(_key64, _val65);
+                      String _key72; // required
+                      String _val73; // required
+                      _key72 = iprot.readString();
+                      _val73 = iprot.readString();
+                      _val69.put(_key72, _val73);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.ssio.put(_key60, _val61);
+                  struct.ssio.put(_key68, _val69);
                 }
                 iprot.readMapEnd();
               }
@@ -1368,9 +1368,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(INPUT_FILES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.inputFiles.size()));
-          for (String _iter66 : struct.inputFiles)
+          for (String _iter74 : struct.inputFiles)
           {
-            oprot.writeString(_iter66);
+            oprot.writeString(_iter74);
           }
           oprot.writeListEnd();
         }
@@ -1406,9 +1406,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(SSI_LIST_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.ssiList.size()));
-          for (org.apache.accumulo.core.data.thrift.IterInfo _iter67 : struct.ssiList)
+          for (org.apache.accumulo.core.data.thrift.IterInfo _iter75 : struct.ssiList)
           {
-            _iter67.write(oprot);
+            _iter75.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1418,15 +1418,15 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(SSIO_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.ssio.size()));
-          for (Map.Entry<String, Map<String,String>> _iter68 : struct.ssio.entrySet())
+          for (Map.Entry<String, Map<String,String>> _iter76 : struct.ssio.entrySet())
           {
-            oprot.writeString(_iter68.getKey());
+            oprot.writeString(_iter76.getKey());
             {
-              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, _iter68.getValue().size()));
-              for (Map.Entry<String, String> _iter69 : _iter68.getValue().entrySet())
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, _iter76.getValue().size()));
+              for (Map.Entry<String, String> _iter77 : _iter76.getValue().entrySet())
               {
-                oprot.writeString(_iter69.getKey());
-                oprot.writeString(_iter69.getValue());
+                oprot.writeString(_iter77.getKey());
+                oprot.writeString(_iter77.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -1496,9 +1496,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetInputFiles()) {
         {
           oprot.writeI32(struct.inputFiles.size());
-          for (String _iter70 : struct.inputFiles)
+          for (String _iter78 : struct.inputFiles)
           {
-            oprot.writeString(_iter70);
+            oprot.writeString(_iter78);
           }
         }
       }
@@ -1523,24 +1523,24 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetSsiList()) {
         {
           oprot.writeI32(struct.ssiList.size());
-          for (org.apache.accumulo.core.data.thrift.IterInfo _iter71 : struct.ssiList)
+          for (org.apache.accumulo.core.data.thrift.IterInfo _iter79 : struct.ssiList)
           {
-            _iter71.write(oprot);
+            _iter79.write(oprot);
           }
         }
       }
       if (struct.isSetSsio()) {
         {
           oprot.writeI32(struct.ssio.size());
-          for (Map.Entry<String, Map<String,String>> _iter72 : struct.ssio.entrySet())
+          for (Map.Entry<String, Map<String,String>> _iter80 : struct.ssio.entrySet())
           {
-            oprot.writeString(_iter72.getKey());
+            oprot.writeString(_iter80.getKey());
             {
-              oprot.writeI32(_iter72.getValue().size());
-              for (Map.Entry<String, String> _iter73 : _iter72.getValue().entrySet())
+              oprot.writeI32(_iter80.getValue().size());
+              for (Map.Entry<String, String> _iter81 : _iter80.getValue().entrySet())
               {
-                oprot.writeString(_iter73.getKey());
-                oprot.writeString(_iter73.getValue());
+                oprot.writeString(_iter81.getKey());
+                oprot.writeString(_iter81.getValue());
               }
             }
           }
@@ -1563,13 +1563,13 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list74 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.inputFiles = new ArrayList<String>(_list74.size);
-          for (int _i75 = 0; _i75 < _list74.size; ++_i75)
+          org.apache.thrift.protocol.TList _list82 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.inputFiles = new ArrayList<String>(_list82.size);
+          for (int _i83 = 0; _i83 < _list82.size; ++_i83)
           {
-            String _elem76; // required
-            _elem76 = iprot.readString();
-            struct.inputFiles.add(_elem76);
+            String _elem84; // required
+            _elem84 = iprot.readString();
+            struct.inputFiles.add(_elem84);
           }
         }
         struct.setInputFilesIsSet(true);
@@ -1600,40 +1600,40 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(9)) {
         {
-          org.apache.thrift.protocol.TList _list77 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.ssiList = new ArrayList<org.apache.accumulo.core.data.thrift.IterInfo>(_list77.size);
-          for (int _i78 = 0; _i78 < _list77.size; ++_i78)
+          org.apache.thrift.protocol.TList _list85 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.ssiList = new ArrayList<org.apache.accumulo.core.data.thrift.IterInfo>(_list85.size);
+          for (int _i86 = 0; _i86 < _list85.size; ++_i86)
           {
-            org.apache.accumulo.core.data.thrift.IterInfo _elem79; // required
-            _elem79 = new org.apache.accumulo.core.data.thrift.IterInfo();
-            _elem79.read(iprot);
-            struct.ssiList.add(_elem79);
+            org.apache.accumulo.core.data.thrift.IterInfo _elem87; // required
+            _elem87 = new org.apache.accumulo.core.data.thrift.IterInfo();
+            _elem87.read(iprot);
+            struct.ssiList.add(_elem87);
           }
         }
         struct.setSsiListIsSet(true);
       }
       if (incoming.get(10)) {
         {
-          org.apache.thrift.protocol.TMap _map80 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
-          struct.ssio = new HashMap<String,Map<String,String>>(2*_map80.size);
-          for (int _i81 = 0; _i81 < _map80.size; ++_i81)
+          org.apache.thrift.protocol.TMap _map88 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+          struct.ssio = new HashMap<String,Map<String,String>>(2*_map88.size);
+          for (int _i89 = 0; _i89 < _map88.size; ++_i89)
           {
-            String _key82; // required
-            Map<String,String> _val83; // required
-            _key82 = iprot.readString();
+            String _key90; // required
+            Map<String,String> _val91; // required
+            _key90 = iprot.readString();
             {
-              org.apache.thrift.protocol.TMap _map84 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-              _val83 = new HashMap<String,String>(2*_map84.size);
-              for (int _i85 = 0; _i85 < _map84.size; ++_i85)
+              org.apache.thrift.protocol.TMap _map92 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+              _val91 = new HashMap<String,String>(2*_map92.size);
+              for (int _i93 = 0; _i93 < _map92.size; ++_i93)
               {
-                String _key86; // required
-                String _val87; // required
-                _key86 = iprot.readString();
-                _val87 = iprot.readString();
-                _val83.put(_key86, _val87);
+                String _key94; // required
+                String _val95; // required
+                _key94 = iprot.readString();
+                _val95 = iprot.readString();
+                _val91.put(_key94, _val95);
               }
             }
-            struct.ssio.put(_key82, _val83);
+            struct.ssio.put(_key90, _val91);
           }
         }
         struct.setSsioIsSet(true);

Added: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/DiskUsage.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/DiskUsage.java?rev=1477758&view=auto
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/DiskUsage.java (added)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/DiskUsage.java Tue Apr 30 18:33:06 2013
@@ -0,0 +1,552 @@
+/*
+ * 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.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.accumulo.core.tabletserver.thrift;
+
+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 org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+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 DiskUsage implements org.apache.thrift.TBase<DiskUsage, DiskUsage._Fields>, java.io.Serializable, Cloneable {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DiskUsage");
+
+  private static final org.apache.thrift.protocol.TField TABLES_FIELD_DESC = new org.apache.thrift.protocol.TField("tables", org.apache.thrift.protocol.TType.LIST, (short)1);
+  private static final org.apache.thrift.protocol.TField USAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("usage", org.apache.thrift.protocol.TType.I64, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new DiskUsageStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new DiskUsageTupleSchemeFactory());
+  }
+
+  public List<String> tables; // required
+  public long usage; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    TABLES((short)1, "tables"),
+    USAGE((short)2, "usage");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TABLES
+          return TABLES;
+        case 2: // USAGE
+          return USAGE;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __USAGE_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TABLES, new org.apache.thrift.meta_data.FieldMetaData("tables", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.USAGE, new org.apache.thrift.meta_data.FieldMetaData("usage", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(DiskUsage.class, metaDataMap);
+  }
+
+  public DiskUsage() {
+  }
+
+  public DiskUsage(
+    List<String> tables,
+    long usage)
+  {
+    this();
+    this.tables = tables;
+    this.usage = usage;
+    setUsageIsSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public DiskUsage(DiskUsage other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetTables()) {
+      List<String> __this__tables = new ArrayList<String>();
+      for (String other_element : other.tables) {
+        __this__tables.add(other_element);
+      }
+      this.tables = __this__tables;
+    }
+    this.usage = other.usage;
+  }
+
+  public DiskUsage deepCopy() {
+    return new DiskUsage(this);
+  }
+
+  @Override
+  public void clear() {
+    this.tables = null;
+    setUsageIsSet(false);
+    this.usage = 0;
+  }
+
+  public int getTablesSize() {
+    return (this.tables == null) ? 0 : this.tables.size();
+  }
+
+  public java.util.Iterator<String> getTablesIterator() {
+    return (this.tables == null) ? null : this.tables.iterator();
+  }
+
+  public void addToTables(String elem) {
+    if (this.tables == null) {
+      this.tables = new ArrayList<String>();
+    }
+    this.tables.add(elem);
+  }
+
+  public List<String> getTables() {
+    return this.tables;
+  }
+
+  public DiskUsage setTables(List<String> tables) {
+    this.tables = tables;
+    return this;
+  }
+
+  public void unsetTables() {
+    this.tables = null;
+  }
+
+  /** Returns true if field tables is set (has been assigned a value) and false otherwise */
+  public boolean isSetTables() {
+    return this.tables != null;
+  }
+
+  public void setTablesIsSet(boolean value) {
+    if (!value) {
+      this.tables = null;
+    }
+  }
+
+  public long getUsage() {
+    return this.usage;
+  }
+
+  public DiskUsage setUsage(long usage) {
+    this.usage = usage;
+    setUsageIsSet(true);
+    return this;
+  }
+
+  public void unsetUsage() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __USAGE_ISSET_ID);
+  }
+
+  /** Returns true if field usage is set (has been assigned a value) and false otherwise */
+  public boolean isSetUsage() {
+    return EncodingUtils.testBit(__isset_bitfield, __USAGE_ISSET_ID);
+  }
+
+  public void setUsageIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __USAGE_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TABLES:
+      if (value == null) {
+        unsetTables();
+      } else {
+        setTables((List<String>)value);
+      }
+      break;
+
+    case USAGE:
+      if (value == null) {
+        unsetUsage();
+      } else {
+        setUsage((Long)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TABLES:
+      return getTables();
+
+    case USAGE:
+      return Long.valueOf(getUsage());
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TABLES:
+      return isSetTables();
+    case USAGE:
+      return isSetUsage();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof DiskUsage)
+      return this.equals((DiskUsage)that);
+    return false;
+  }
+
+  public boolean equals(DiskUsage that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_tables = true && this.isSetTables();
+    boolean that_present_tables = true && that.isSetTables();
+    if (this_present_tables || that_present_tables) {
+      if (!(this_present_tables && that_present_tables))
+        return false;
+      if (!this.tables.equals(that.tables))
+        return false;
+    }
+
+    boolean this_present_usage = true;
+    boolean that_present_usage = true;
+    if (this_present_usage || that_present_usage) {
+      if (!(this_present_usage && that_present_usage))
+        return false;
+      if (this.usage != that.usage)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    return 0;
+  }
+
+  public int compareTo(DiskUsage other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    DiskUsage typedOther = (DiskUsage)other;
+
+    lastComparison = Boolean.valueOf(isSetTables()).compareTo(typedOther.isSetTables());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTables()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tables, typedOther.tables);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetUsage()).compareTo(typedOther.isSetUsage());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetUsage()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.usage, typedOther.usage);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("DiskUsage(");
+    boolean first = true;
+
+    sb.append("tables:");
+    if (this.tables == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.tables);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("usage:");
+    sb.append(this.usage);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  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_bitfield = 0;
+      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 DiskUsageStandardSchemeFactory implements SchemeFactory {
+    public DiskUsageStandardScheme getScheme() {
+      return new DiskUsageStandardScheme();
+    }
+  }
+
+  private static class DiskUsageStandardScheme extends StandardScheme<DiskUsage> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, DiskUsage 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: // TABLES
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list52 = iprot.readListBegin();
+                struct.tables = new ArrayList<String>(_list52.size);
+                for (int _i53 = 0; _i53 < _list52.size; ++_i53)
+                {
+                  String _elem54; // required
+                  _elem54 = iprot.readString();
+                  struct.tables.add(_elem54);
+                }
+                iprot.readListEnd();
+              }
+              struct.setTablesIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // USAGE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.usage = iprot.readI64();
+              struct.setUsageIsSet(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, DiskUsage struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.tables != null) {
+        oprot.writeFieldBegin(TABLES_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tables.size()));
+          for (String _iter55 : struct.tables)
+          {
+            oprot.writeString(_iter55);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(USAGE_FIELD_DESC);
+      oprot.writeI64(struct.usage);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class DiskUsageTupleSchemeFactory implements SchemeFactory {
+    public DiskUsageTupleScheme getScheme() {
+      return new DiskUsageTupleScheme();
+    }
+  }
+
+  private static class DiskUsageTupleScheme extends TupleScheme<DiskUsage> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, DiskUsage struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetTables()) {
+        optionals.set(0);
+      }
+      if (struct.isSetUsage()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetTables()) {
+        {
+          oprot.writeI32(struct.tables.size());
+          for (String _iter56 : struct.tables)
+          {
+            oprot.writeString(_iter56);
+          }
+        }
+      }
+      if (struct.isSetUsage()) {
+        oprot.writeI64(struct.usage);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, DiskUsage struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TList _list57 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.tables = new ArrayList<String>(_list57.size);
+          for (int _i58 = 0; _i58 < _list57.size; ++_i58)
+          {
+            String _elem59; // required
+            _elem59 = iprot.readString();
+            struct.tables.add(_elem59);
+          }
+        }
+        struct.setTablesIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.usage = iprot.readI64();
+        struct.setUsageIsSet(true);
+      }
+    }
+  }
+
+}
+

Propchange: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/DiskUsage.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/IteratorConfig.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/IteratorConfig.java?rev=1477758&r1=1477757&r2=1477758&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/IteratorConfig.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/IteratorConfig.java Tue Apr 30 18:33:06 2013
@@ -355,14 +355,14 @@ import org.slf4j.LoggerFactory;
           case 1: // ITERATORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list98 = iprot.readListBegin();
-                struct.iterators = new ArrayList<TIteratorSetting>(_list98.size);
-                for (int _i99 = 0; _i99 < _list98.size; ++_i99)
+                org.apache.thrift.protocol.TList _list106 = iprot.readListBegin();
+                struct.iterators = new ArrayList<TIteratorSetting>(_list106.size);
+                for (int _i107 = 0; _i107 < _list106.size; ++_i107)
                 {
-                  TIteratorSetting _elem100; // required
-                  _elem100 = new TIteratorSetting();
-                  _elem100.read(iprot);
-                  struct.iterators.add(_elem100);
+                  TIteratorSetting _elem108; // required
+                  _elem108 = new TIteratorSetting();
+                  _elem108.read(iprot);
+                  struct.iterators.add(_elem108);
                 }
                 iprot.readListEnd();
               }
@@ -390,9 +390,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(ITERATORS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.iterators.size()));
-          for (TIteratorSetting _iter101 : struct.iterators)
+          for (TIteratorSetting _iter109 : struct.iterators)
           {
-            _iter101.write(oprot);
+            _iter109.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -423,9 +423,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetIterators()) {
         {
           oprot.writeI32(struct.iterators.size());
-          for (TIteratorSetting _iter102 : struct.iterators)
+          for (TIteratorSetting _iter110 : struct.iterators)
           {
-            _iter102.write(oprot);
+            _iter110.write(oprot);
           }
         }
       }
@@ -437,14 +437,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list103 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.iterators = new ArrayList<TIteratorSetting>(_list103.size);
-          for (int _i104 = 0; _i104 < _list103.size; ++_i104)
+          org.apache.thrift.protocol.TList _list111 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.iterators = new ArrayList<TIteratorSetting>(_list111.size);
+          for (int _i112 = 0; _i112 < _list111.size; ++_i112)
           {
-            TIteratorSetting _elem105; // required
-            _elem105 = new TIteratorSetting();
-            _elem105.read(iprot);
-            struct.iterators.add(_elem105);
+            TIteratorSetting _elem113; // required
+            _elem113 = new TIteratorSetting();
+            _elem113.read(iprot);
+            struct.iterators.add(_elem113);
           }
         }
         struct.setIteratorsIsSet(true);

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TIteratorSetting.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TIteratorSetting.java?rev=1477758&r1=1477757&r2=1477758&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TIteratorSetting.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TIteratorSetting.java Tue Apr 30 18:33:06 2013
@@ -615,15 +615,15 @@ import org.slf4j.LoggerFactory;
           case 4: // PROPERTIES
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map88 = iprot.readMapBegin();
-                struct.properties = new HashMap<String,String>(2*_map88.size);
-                for (int _i89 = 0; _i89 < _map88.size; ++_i89)
+                org.apache.thrift.protocol.TMap _map96 = iprot.readMapBegin();
+                struct.properties = new HashMap<String,String>(2*_map96.size);
+                for (int _i97 = 0; _i97 < _map96.size; ++_i97)
                 {
-                  String _key90; // required
-                  String _val91; // required
-                  _key90 = iprot.readString();
-                  _val91 = iprot.readString();
-                  struct.properties.put(_key90, _val91);
+                  String _key98; // required
+                  String _val99; // required
+                  _key98 = iprot.readString();
+                  _val99 = iprot.readString();
+                  struct.properties.put(_key98, _val99);
                 }
                 iprot.readMapEnd();
               }
@@ -664,10 +664,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PROPERTIES_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.properties.size()));
-          for (Map.Entry<String, String> _iter92 : struct.properties.entrySet())
+          for (Map.Entry<String, String> _iter100 : struct.properties.entrySet())
           {
-            oprot.writeString(_iter92.getKey());
-            oprot.writeString(_iter92.getValue());
+            oprot.writeString(_iter100.getKey());
+            oprot.writeString(_iter100.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -716,10 +716,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetProperties()) {
         {
           oprot.writeI32(struct.properties.size());
-          for (Map.Entry<String, String> _iter93 : struct.properties.entrySet())
+          for (Map.Entry<String, String> _iter101 : struct.properties.entrySet())
           {
-            oprot.writeString(_iter93.getKey());
-            oprot.writeString(_iter93.getValue());
+            oprot.writeString(_iter101.getKey());
+            oprot.writeString(_iter101.getValue());
           }
         }
       }
@@ -743,15 +743,15 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(3)) {
         {
-          org.apache.thrift.protocol.TMap _map94 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.properties = new HashMap<String,String>(2*_map94.size);
-          for (int _i95 = 0; _i95 < _map94.size; ++_i95)
+          org.apache.thrift.protocol.TMap _map102 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.properties = new HashMap<String,String>(2*_map102.size);
+          for (int _i103 = 0; _i103 < _map102.size; ++_i103)
           {
-            String _key96; // required
-            String _val97; // required
-            _key96 = iprot.readString();
-            _val97 = iprot.readString();
-            struct.properties.put(_key96, _val97);
+            String _key104; // required
+            String _val105; // required
+            _key104 = iprot.readString();
+            _val105 = iprot.readString();
+            struct.properties.put(_key104, _val105);
           }
         }
         struct.setPropertiesIsSet(true);