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/06/19 17:57:14 UTC

svn commit: r1494671 [2/6] - in /accumulo/branches/ACCUMULO-118: ./ assemble/ core/ core/src/main/java/org/apache/accumulo/core/ core/src/main/java/org/apache/accumulo/core/client/ core/src/main/java/org/apache/accumulo/core/client/admin/ core/src/main...

Modified: accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java?rev=1494671&r1=1494670&r2=1494671&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java (original)
+++ accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java Wed Jun 19 15:57:11 2013
@@ -3685,7 +3685,7 @@ import org.slf4j.LoggerFactory;
                   struct.columns = new ArrayList<org.apache.accumulo.core.data.thrift.TColumn>(_list106.size);
                   for (int _i107 = 0; _i107 < _list106.size; ++_i107)
                   {
-                    org.apache.accumulo.core.data.thrift.TColumn _elem108; // required
+                    org.apache.accumulo.core.data.thrift.TColumn _elem108; // optional
                     _elem108 = new org.apache.accumulo.core.data.thrift.TColumn();
                     _elem108.read(iprot);
                     struct.columns.add(_elem108);
@@ -3712,7 +3712,7 @@ import org.slf4j.LoggerFactory;
                   struct.ssiList = new ArrayList<org.apache.accumulo.core.data.thrift.IterInfo>(_list109.size);
                   for (int _i110 = 0; _i110 < _list109.size; ++_i110)
                   {
-                    org.apache.accumulo.core.data.thrift.IterInfo _elem111; // required
+                    org.apache.accumulo.core.data.thrift.IterInfo _elem111; // optional
                     _elem111 = new org.apache.accumulo.core.data.thrift.IterInfo();
                     _elem111.read(iprot);
                     struct.ssiList.add(_elem111);
@@ -3763,7 +3763,7 @@ import org.slf4j.LoggerFactory;
                   struct.authorizations = new ArrayList<ByteBuffer>(_list120.size);
                   for (int _i121 = 0; _i121 < _list120.size; ++_i121)
                   {
-                    ByteBuffer _elem122; // required
+                    ByteBuffer _elem122; // optional
                     _elem122 = iprot.readBinary();
                     struct.authorizations.add(_elem122);
                   }
@@ -4040,7 +4040,7 @@ import org.slf4j.LoggerFactory;
             struct.columns = new ArrayList<org.apache.accumulo.core.data.thrift.TColumn>(_list133.size);
             for (int _i134 = 0; _i134 < _list133.size; ++_i134)
             {
-              org.apache.accumulo.core.data.thrift.TColumn _elem135; // required
+              org.apache.accumulo.core.data.thrift.TColumn _elem135; // optional
               _elem135 = new org.apache.accumulo.core.data.thrift.TColumn();
               _elem135.read(iprot);
               struct.columns.add(_elem135);
@@ -4058,7 +4058,7 @@ import org.slf4j.LoggerFactory;
             struct.ssiList = new ArrayList<org.apache.accumulo.core.data.thrift.IterInfo>(_list136.size);
             for (int _i137 = 0; _i137 < _list136.size; ++_i137)
             {
-              org.apache.accumulo.core.data.thrift.IterInfo _elem138; // required
+              org.apache.accumulo.core.data.thrift.IterInfo _elem138; // optional
               _elem138 = new org.apache.accumulo.core.data.thrift.IterInfo();
               _elem138.read(iprot);
               struct.ssiList.add(_elem138);
@@ -4098,7 +4098,7 @@ import org.slf4j.LoggerFactory;
             struct.authorizations = new ArrayList<ByteBuffer>(_list147.size);
             for (int _i148 = 0; _i148 < _list147.size; ++_i148)
             {
-              ByteBuffer _elem149; // required
+              ByteBuffer _elem149; // optional
               _elem149 = iprot.readBinary();
               struct.authorizations.add(_elem149);
             }
@@ -7338,7 +7338,7 @@ import org.slf4j.LoggerFactory;
                       _val153 = new ArrayList<org.apache.accumulo.core.data.thrift.TRange>(_list154.size);
                       for (int _i155 = 0; _i155 < _list154.size; ++_i155)
                       {
-                        org.apache.accumulo.core.data.thrift.TRange _elem156; // required
+                        org.apache.accumulo.core.data.thrift.TRange _elem156; // optional
                         _elem156 = new org.apache.accumulo.core.data.thrift.TRange();
                         _elem156.read(iprot);
                         _val153.add(_elem156);
@@ -7361,7 +7361,7 @@ import org.slf4j.LoggerFactory;
                   struct.columns = new ArrayList<org.apache.accumulo.core.data.thrift.TColumn>(_list157.size);
                   for (int _i158 = 0; _i158 < _list157.size; ++_i158)
                   {
-                    org.apache.accumulo.core.data.thrift.TColumn _elem159; // required
+                    org.apache.accumulo.core.data.thrift.TColumn _elem159; // optional
                     _elem159 = new org.apache.accumulo.core.data.thrift.TColumn();
                     _elem159.read(iprot);
                     struct.columns.add(_elem159);
@@ -7380,7 +7380,7 @@ import org.slf4j.LoggerFactory;
                   struct.ssiList = new ArrayList<org.apache.accumulo.core.data.thrift.IterInfo>(_list160.size);
                   for (int _i161 = 0; _i161 < _list160.size; ++_i161)
                   {
-                    org.apache.accumulo.core.data.thrift.IterInfo _elem162; // required
+                    org.apache.accumulo.core.data.thrift.IterInfo _elem162; // optional
                     _elem162 = new org.apache.accumulo.core.data.thrift.IterInfo();
                     _elem162.read(iprot);
                     struct.ssiList.add(_elem162);
@@ -7431,7 +7431,7 @@ import org.slf4j.LoggerFactory;
                   struct.authorizations = new ArrayList<ByteBuffer>(_list171.size);
                   for (int _i172 = 0; _i172 < _list171.size; ++_i172)
                   {
-                    ByteBuffer _elem173; // required
+                    ByteBuffer _elem173; // optional
                     _elem173 = iprot.readBinary();
                     struct.authorizations.add(_elem173);
                   }
@@ -7698,7 +7698,7 @@ import org.slf4j.LoggerFactory;
                 _val191 = new ArrayList<org.apache.accumulo.core.data.thrift.TRange>(_list192.size);
                 for (int _i193 = 0; _i193 < _list192.size; ++_i193)
                 {
-                  org.apache.accumulo.core.data.thrift.TRange _elem194; // required
+                  org.apache.accumulo.core.data.thrift.TRange _elem194; // optional
                   _elem194 = new org.apache.accumulo.core.data.thrift.TRange();
                   _elem194.read(iprot);
                   _val191.add(_elem194);
@@ -7715,7 +7715,7 @@ import org.slf4j.LoggerFactory;
             struct.columns = new ArrayList<org.apache.accumulo.core.data.thrift.TColumn>(_list195.size);
             for (int _i196 = 0; _i196 < _list195.size; ++_i196)
             {
-              org.apache.accumulo.core.data.thrift.TColumn _elem197; // required
+              org.apache.accumulo.core.data.thrift.TColumn _elem197; // optional
               _elem197 = new org.apache.accumulo.core.data.thrift.TColumn();
               _elem197.read(iprot);
               struct.columns.add(_elem197);
@@ -7729,7 +7729,7 @@ import org.slf4j.LoggerFactory;
             struct.ssiList = new ArrayList<org.apache.accumulo.core.data.thrift.IterInfo>(_list198.size);
             for (int _i199 = 0; _i199 < _list198.size; ++_i199)
             {
-              org.apache.accumulo.core.data.thrift.IterInfo _elem200; // required
+              org.apache.accumulo.core.data.thrift.IterInfo _elem200; // optional
               _elem200 = new org.apache.accumulo.core.data.thrift.IterInfo();
               _elem200.read(iprot);
               struct.ssiList.add(_elem200);
@@ -7769,7 +7769,7 @@ import org.slf4j.LoggerFactory;
             struct.authorizations = new ArrayList<ByteBuffer>(_list209.size);
             for (int _i210 = 0; _i210 < _list209.size; ++_i210)
             {
-              ByteBuffer _elem211; // required
+              ByteBuffer _elem211; // optional
               _elem211 = iprot.readBinary();
               struct.authorizations.add(_elem211);
             }
@@ -11473,7 +11473,7 @@ import org.slf4j.LoggerFactory;
                   struct.mutations = new ArrayList<org.apache.accumulo.core.data.thrift.TMutation>(_list212.size);
                   for (int _i213 = 0; _i213 < _list212.size; ++_i213)
                   {
-                    org.apache.accumulo.core.data.thrift.TMutation _elem214; // required
+                    org.apache.accumulo.core.data.thrift.TMutation _elem214; // optional
                     _elem214 = new org.apache.accumulo.core.data.thrift.TMutation();
                     _elem214.read(iprot);
                     struct.mutations.add(_elem214);
@@ -11600,7 +11600,7 @@ import org.slf4j.LoggerFactory;
             struct.mutations = new ArrayList<org.apache.accumulo.core.data.thrift.TMutation>(_list217.size);
             for (int _i218 = 0; _i218 < _list217.size; ++_i218)
             {
-              org.apache.accumulo.core.data.thrift.TMutation _elem219; // required
+              org.apache.accumulo.core.data.thrift.TMutation _elem219; // optional
               _elem219 = new org.apache.accumulo.core.data.thrift.TMutation();
               _elem219.read(iprot);
               struct.mutations.add(_elem219);
@@ -15002,7 +15002,7 @@ import org.slf4j.LoggerFactory;
                   struct.success = new ArrayList<org.apache.accumulo.core.data.thrift.TKeyExtent>(_list240.size);
                   for (int _i241 = 0; _i241 < _list240.size; ++_i241)
                   {
-                    org.apache.accumulo.core.data.thrift.TKeyExtent _elem242; // required
+                    org.apache.accumulo.core.data.thrift.TKeyExtent _elem242; // optional
                     _elem242 = new org.apache.accumulo.core.data.thrift.TKeyExtent();
                     _elem242.read(iprot);
                     struct.success.add(_elem242);
@@ -15104,7 +15104,7 @@ import org.slf4j.LoggerFactory;
             struct.success = new ArrayList<org.apache.accumulo.core.data.thrift.TKeyExtent>(_list245.size);
             for (int _i246 = 0; _i246 < _list245.size; ++_i246)
             {
-              org.apache.accumulo.core.data.thrift.TKeyExtent _elem247; // required
+              org.apache.accumulo.core.data.thrift.TKeyExtent _elem247; // optional
               _elem247 = new org.apache.accumulo.core.data.thrift.TKeyExtent();
               _elem247.read(iprot);
               struct.success.add(_elem247);
@@ -22685,7 +22685,7 @@ import org.slf4j.LoggerFactory;
                   struct.success = new ArrayList<TabletStats>(_list248.size);
                   for (int _i249 = 0; _i249 < _list248.size; ++_i249)
                   {
-                    TabletStats _elem250; // required
+                    TabletStats _elem250; // optional
                     _elem250 = new TabletStats();
                     _elem250.read(iprot);
                     struct.success.add(_elem250);
@@ -22787,7 +22787,7 @@ import org.slf4j.LoggerFactory;
             struct.success = new ArrayList<TabletStats>(_list253.size);
             for (int _i254 = 0; _i254 < _list253.size; ++_i254)
             {
-              TabletStats _elem255; // required
+              TabletStats _elem255; // optional
               _elem255 = new TabletStats();
               _elem255.read(iprot);
               struct.success.add(_elem255);
@@ -26068,7 +26068,7 @@ import org.slf4j.LoggerFactory;
                   struct.success = new ArrayList<ActiveScan>(_list256.size);
                   for (int _i257 = 0; _i257 < _list256.size; ++_i257)
                   {
-                    ActiveScan _elem258; // required
+                    ActiveScan _elem258; // optional
                     _elem258 = new ActiveScan();
                     _elem258.read(iprot);
                     struct.success.add(_elem258);
@@ -26170,7 +26170,7 @@ import org.slf4j.LoggerFactory;
             struct.success = new ArrayList<ActiveScan>(_list261.size);
             for (int _i262 = 0; _i262 < _list261.size; ++_i262)
             {
-              ActiveScan _elem263; // required
+              ActiveScan _elem263; // optional
               _elem263 = new ActiveScan();
               _elem263.read(iprot);
               struct.success.add(_elem263);
@@ -27042,7 +27042,7 @@ import org.slf4j.LoggerFactory;
                   struct.success = new ArrayList<ActiveCompaction>(_list264.size);
                   for (int _i265 = 0; _i265 < _list264.size; ++_i265)
                   {
-                    ActiveCompaction _elem266; // required
+                    ActiveCompaction _elem266; // optional
                     _elem266 = new ActiveCompaction();
                     _elem266.read(iprot);
                     struct.success.add(_elem266);
@@ -27144,7 +27144,7 @@ import org.slf4j.LoggerFactory;
             struct.success = new ArrayList<ActiveCompaction>(_list269.size);
             for (int _i270 = 0; _i270 < _list269.size; ++_i270)
             {
-              ActiveCompaction _elem271; // required
+              ActiveCompaction _elem271; // optional
               _elem271 = new ActiveCompaction();
               _elem271.read(iprot);
               struct.success.add(_elem271);
@@ -27653,7 +27653,7 @@ import org.slf4j.LoggerFactory;
                   struct.filenames = new ArrayList<String>(_list272.size);
                   for (int _i273 = 0; _i273 < _list272.size; ++_i273)
                   {
-                    String _elem274; // required
+                    String _elem274; // optional
                     _elem274 = iprot.readString();
                     struct.filenames.add(_elem274);
                   }
@@ -27766,7 +27766,7 @@ import org.slf4j.LoggerFactory;
             struct.filenames = new ArrayList<String>(_list277.size);
             for (int _i278 = 0; _i278 < _list277.size; ++_i278)
             {
-              String _elem279; // required
+              String _elem279; // optional
               _elem279 = iprot.readString();
               struct.filenames.add(_elem279);
             }

Modified: accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/Merge.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/Merge.java?rev=1494671&r1=1494670&r2=1494671&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/Merge.java (original)
+++ accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/Merge.java Wed Jun 19 15:57:11 2013
@@ -21,7 +21,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.cli.ClientOnRequiredTable;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
@@ -121,7 +120,7 @@ public class Merge {
   
   public void mergomatic(Connector conn, String table, Text start, Text end, long goalSize, boolean force) throws MergeException {
     try {
-      if (table.equals(Constants.METADATA_TABLE_NAME)) {
+      if (table.equals(MetadataTable.NAME)) {
         throw new IllegalArgumentException("cannot merge tablets on the metadata table");
       }
       List<Size> sizes = new ArrayList<Size>();
@@ -212,13 +211,13 @@ public class Merge {
     Scanner scanner;
     try {
       tableId = Tables.getTableId(conn.getInstance(), tablename);
-      scanner = conn.createScanner(Constants.METADATA_TABLE_NAME, Authorizations.EMPTY);
+      scanner = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
     } catch (Exception e) {
       throw new MergeException(e);
     }
     scanner.setRange(new KeyExtent(new Text(tableId), end, start).toMetadataRange());
-    scanner.fetchColumnFamily(Constants.METADATA_DATAFILE_COLUMN_FAMILY);
-    Constants.METADATA_PREV_ROW_COLUMN.fetch(scanner);
+    scanner.fetchColumnFamily(MetadataTable.DATAFILE_COLUMN_FAMILY);
+    MetadataTable.PREV_ROW_COLUMN.fetch(scanner);
     final Iterator<Entry<Key,Value>> iterator = scanner.iterator();
     
     Iterator<Size> result = new Iterator<Size>() {
@@ -234,12 +233,12 @@ public class Merge {
         while (iterator.hasNext()) {
           Entry<Key,Value> entry = iterator.next();
           Key key = entry.getKey();
-          if (key.getColumnFamily().equals(Constants.METADATA_DATAFILE_COLUMN_FAMILY)) {
+          if (key.getColumnFamily().equals(MetadataTable.DATAFILE_COLUMN_FAMILY)) {
             String[] sizeEntries = new String(entry.getValue().get()).split(",");
             if (sizeEntries.length == 2) {
               tabletSize += Long.parseLong(sizeEntries[0]);
             }
-          } else if (Constants.METADATA_PREV_ROW_COLUMN.hasColumns(key)) {
+          } else if (MetadataTable.PREV_ROW_COLUMN.hasColumns(key)) {
             KeyExtent extent = new KeyExtent(key.getRow(), entry.getValue());
             return new Size(extent, tabletSize);
           }

Modified: accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/MetadataTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/MetadataTable.java?rev=1494671&r1=1494670&r2=1494671&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/MetadataTable.java (original)
+++ accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/MetadataTable.java Wed Jun 19 15:57:11 2013
@@ -26,7 +26,6 @@ import java.util.SortedMap;
 import java.util.SortedSet;
 import java.util.TreeMap;
 
-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.Instance;
@@ -44,6 +43,68 @@ import org.apache.accumulo.core.security
 import org.apache.hadoop.io.Text;
 
 public class MetadataTable {
+  
+  public static final String ID = "!0";
+  public static final String NAME = "!METADATA";
+  
+  /**
+   * Initial tablet directory name
+   */
+  public static final String TABLE_TABLET_LOCATION = "/table_info";
+  
+  /**
+   * Reserved keyspace is any row that begins with a tilde '~' character
+   */
+  public static final Key RESERVED_KEYSPACE_START_KEY = new Key(new Text(new byte[] {'~'}));
+  public static final String DELETE_FLAG_PREFIX = "~del";
+  public static final Range DELETES_KEYSPACE = new Range(new Key(new Text(DELETE_FLAG_PREFIX)), true, new Key(new Text("~dem")), false);
+  public static final String BLIP_FLAG_PREFIX = "~blip"; // BLIP = bulk load in progress
+  public static final Range BLIP_KEYSPACE = new Range(new Key(new Text(BLIP_FLAG_PREFIX)), true, new Key(new Text("~bliq")), false);
+  
+  public static final Text CURRENT_LOCATION_COLUMN_FAMILY = new Text("loc");
+  public static final Text FUTURE_LOCATION_COLUMN_FAMILY = new Text("future");
+  public static final Text LAST_LOCATION_COLUMN_FAMILY = new Text("last");
+  /**
+   * Temporary marker that indicates a tablet loaded a bulk file
+   */
+  public static final Text BULKFILE_COLUMN_FAMILY = new Text("loaded");
+  
+  /**
+   * Temporary marker that indicates a tablet was successfully cloned
+   */
+  public static final Text CLONED_COLUMN_FAMILY = new Text("!cloned");
+  
+  /**
+   * This needs to sort after all other column families for that tablet, because the {@link #PREV_ROW_COLUMN} sits in this and that needs to sort last because
+   * the {@link SimpleGarbageCollector} relies on this.
+   */
+  public static final Text TABLET_COLUMN_FAMILY = new Text("~tab");
+  
+  /**
+   * README : very important that prevRow sort last to avoid race conditions between garbage collector and split this needs to sort after everything else for
+   * that tablet
+   */
+  public static final ColumnFQ PREV_ROW_COLUMN = new ColumnFQ(TABLET_COLUMN_FAMILY, new Text("~pr"));
+  public static final ColumnFQ OLD_PREV_ROW_COLUMN = new ColumnFQ(TABLET_COLUMN_FAMILY, new Text("oldprevrow"));
+  public static final ColumnFQ SPLIT_RATIO_COLUMN = new ColumnFQ(TABLET_COLUMN_FAMILY, new Text("splitRatio"));
+  
+  public static final Text SERVER_COLUMN_FAMILY = new Text("srv");
+  public static final ColumnFQ DIRECTORY_COLUMN = new ColumnFQ(SERVER_COLUMN_FAMILY, new Text("dir"));
+  public static final ColumnFQ TIME_COLUMN = new ColumnFQ(SERVER_COLUMN_FAMILY, new Text("time"));
+  public static final ColumnFQ FLUSH_COLUMN = new ColumnFQ(SERVER_COLUMN_FAMILY, new Text("flush"));
+  public static final ColumnFQ COMPACT_COLUMN = new ColumnFQ(SERVER_COLUMN_FAMILY, new Text("compact"));
+  public static final ColumnFQ LOCK_COLUMN = new ColumnFQ(SERVER_COLUMN_FAMILY, new Text("lock"));
+  
+  public static final Text DATAFILE_COLUMN_FAMILY = new Text("file");
+  public static final Text SCANFILE_COLUMN_FAMILY = new Text("scan");
+  public static final Text LOG_COLUMN_FAMILY = new Text("log");
+  public static final Text CHOPPED_COLUMN_FAMILY = new Text("chopped");
+  public static final ColumnFQ CHOPPED_COLUMN = new ColumnFQ(CHOPPED_COLUMN_FAMILY, new Text("chopped"));
+  
+  public static final Range NON_ROOT_KEYSPACE = new Range(new Key(KeyExtent.getMetadataEntry(new Text(ID), null)).followingKey(PartialKey.ROW), true,
+      RESERVED_KEYSPACE_START_KEY, false);
+  public static final Range KEYSPACE = new Range(new Key(new Text(ID)), true, RESERVED_KEYSPACE_START_KEY, false);
+  
   public static class DataFileValue {
     private long size;
     private long numEntries;
@@ -153,12 +214,12 @@ public class MetadataTable {
       colq = key.getColumnQualifier(colq);
       
       // interpret the row id as a key extent
-      if (colf.equals(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY) || colf.equals(Constants.METADATA_FUTURE_LOCATION_COLUMN_FAMILY)) {
+      if (colf.equals(CURRENT_LOCATION_COLUMN_FAMILY) || colf.equals(FUTURE_LOCATION_COLUMN_FAMILY)) {
         if (location != null) {
           throw new IllegalStateException("Tablet has multiple locations : " + lastRowFromKey);
         }
         location = new Text(val.toString());
-      } else if (Constants.METADATA_PREV_ROW_COLUMN.equals(colf, colq)) {
+      } else if (PREV_ROW_COLUMN.equals(colf, colq)) {
         prevRow = new Value(val);
       }
       
@@ -209,11 +270,10 @@ public class MetadataTable {
       SortedSet<KeyExtent> tablets) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     String tableId = isTid ? table : Tables.getNameToIdMap(instance).get(table);
     
-    Scanner scanner = instance.getConnector(credentials.getPrincipal(), CredentialHelper.extractToken(credentials)).createScanner(
-        Constants.METADATA_TABLE_NAME, Authorizations.EMPTY);
+    Scanner scanner = instance.getConnector(credentials.getPrincipal(), CredentialHelper.extractToken(credentials)).createScanner(NAME, Authorizations.EMPTY);
     
-    Constants.METADATA_PREV_ROW_COLUMN.fetch(scanner);
-    scanner.fetchColumnFamily(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY);
+    PREV_ROW_COLUMN.fetch(scanner);
+    scanner.fetchColumnFamily(CURRENT_LOCATION_COLUMN_FAMILY);
     
     // position at first entry in metadata table for given table
     KeyExtent ke = new KeyExtent(new Text(tableId), new Text(), null);
@@ -249,11 +309,11 @@ public class MetadataTable {
       if (!(new KeyExtent(entry.getKey().getRow(), (Text) null)).getTableId().toString().equals(tableId))
         break;
       
-      if (Constants.METADATA_PREV_ROW_COLUMN.equals(colf, colq)) {
+      if (PREV_ROW_COLUMN.equals(colf, colq)) {
         currentKeyExtent = new KeyExtent(entry.getKey().getRow(), entry.getValue());
         tablets.add(currentKeyExtent);
         haveExtent = true;
-      } else if (colf.equals(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY)) {
+      } else if (colf.equals(CURRENT_LOCATION_COLUMN_FAMILY)) {
         location = entry.getValue().toString();
         haveLocation = true;
       }

Modified: accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FlushCommand.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FlushCommand.java?rev=1494671&r1=1494670&r2=1494671&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FlushCommand.java (original)
+++ accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FlushCommand.java Wed Jun 19 15:57:11 2013
@@ -16,10 +16,10 @@
  */
 package org.apache.accumulo.core.util.shell.commands;
 
-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.TableNotFoundException;
+import org.apache.accumulo.core.util.MetadataTable;
 import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
@@ -41,9 +41,9 @@ public class FlushCommand extends TableO
   protected void doTableOp(final Shell shellState, final String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     shellState.getConnector().tableOperations().flush(tableName, startRow, endRow, wait);
     Shell.log.info("Flush of table " + tableName + (wait ? " completed." : " initiated..."));
-    if (tableName.equals(Constants.METADATA_TABLE_NAME)) {
-      Shell.log.info("  May need to flush " + Constants.METADATA_TABLE_NAME + " table multiple times.");
-      Shell.log.info("  Flushing " + Constants.METADATA_TABLE_NAME + " causes writes to itself and");
+    if (tableName.equals(MetadataTable.NAME)) {
+      Shell.log.info("  May need to flush " + MetadataTable.NAME + " table multiple times.");
+      Shell.log.info("  Flushing " + MetadataTable.NAME + " causes writes to itself and");
       Shell.log.info("  minor compactions, which also cause writes to itself.");
       Shell.log.info("  Check the monitor web page and give it time to settle.");
     }

Modified: accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GetSplitsCommand.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GetSplitsCommand.java?rev=1494671&r1=1494670&r2=1494671&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GetSplitsCommand.java (original)
+++ accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GetSplitsCommand.java Wed Jun 19 15:57:11 2013
@@ -22,7 +22,6 @@ import java.security.NoSuchAlgorithmExce
 import java.util.Iterator;
 import java.util.Map.Entry;
 
-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.Scanner;
@@ -32,6 +31,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.MetadataTable;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.core.util.format.BinaryFormatter;
 import org.apache.accumulo.core.util.shell.Shell;
@@ -69,15 +69,15 @@ public class GetSplitsCommand extends Co
           p.print(encode(encode, row));
         }
       } else {
-        final Scanner scanner = shellState.getConnector().createScanner(Constants.METADATA_TABLE_NAME, Authorizations.EMPTY);
-        Constants.METADATA_PREV_ROW_COLUMN.fetch(scanner);
+        final Scanner scanner = shellState.getConnector().createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+        MetadataTable.PREV_ROW_COLUMN.fetch(scanner);
         final Text start = new Text(shellState.getConnector().tableOperations().tableIdMap().get(tableName));
         final Text end = new Text(start);
         end.append(new byte[] {'<'}, 0, 1);
         scanner.setRange(new Range(start, end));
         for (Iterator<Entry<Key,Value>> iterator = scanner.iterator(); iterator.hasNext();) {
           final Entry<Key,Value> next = iterator.next();
-          if (Constants.METADATA_PREV_ROW_COLUMN.hasColumns(next.getKey())) {
+          if (MetadataTable.PREV_ROW_COLUMN.hasColumns(next.getKey())) {
             KeyExtent extent = new KeyExtent(next.getKey().getRow(), next.getValue());
             final String pr = encode(encode, extent.getPrevEndRow());
             final String er = encode(encode, extent.getEndRow());

Modified: accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OfflineCommand.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OfflineCommand.java?rev=1494671&r1=1494670&r2=1494671&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OfflineCommand.java (original)
+++ accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OfflineCommand.java Wed Jun 19 15:57:11 2013
@@ -16,10 +16,10 @@
  */
 package org.apache.accumulo.core.util.shell.commands;
 
-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.TableNotFoundException;
+import org.apache.accumulo.core.util.MetadataTable;
 import org.apache.accumulo.core.util.shell.Shell;
 
 public class OfflineCommand extends TableOperation {
@@ -29,8 +29,8 @@ public class OfflineCommand extends Tabl
   }
   
   protected void doTableOp(final Shell shellState, final String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    if (tableName.equals(Constants.METADATA_TABLE_NAME)) {
-      Shell.log.info("  You cannot take the " + Constants.METADATA_TABLE_NAME + " offline.");
+    if (tableName.equals(MetadataTable.NAME)) {
+      Shell.log.info("  You cannot take the " + MetadataTable.NAME + " offline.");
     } else {
       Shell.log.info("Attempting to begin taking " + tableName + " offline");
       shellState.getConnector().tableOperations().offline(tableName);

Modified: accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OnlineCommand.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OnlineCommand.java?rev=1494671&r1=1494670&r2=1494671&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OnlineCommand.java (original)
+++ accumulo/branches/ACCUMULO-118/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OnlineCommand.java Wed Jun 19 15:57:11 2013
@@ -16,10 +16,10 @@
  */
 package org.apache.accumulo.core.util.shell.commands;
 
-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.TableNotFoundException;
+import org.apache.accumulo.core.util.MetadataTable;
 import org.apache.accumulo.core.util.shell.Shell;
 
 public class OnlineCommand extends TableOperation {
@@ -29,8 +29,8 @@ public class OnlineCommand extends Table
   }
   
   protected void doTableOp(final Shell shellState, final String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    if (tableName.equals(Constants.METADATA_TABLE_NAME)) {
-      Shell.log.info("  The " + Constants.METADATA_TABLE_NAME + " is always online.");
+    if (tableName.equals(MetadataTable.NAME)) {
+      Shell.log.info("  The " + MetadataTable.NAME + " is always online.");
     } else {
       Shell.log.info("Attempting to begin bringing " + tableName + " online");
       shellState.getConnector().tableOperations().online(tableName);

Modified: accumulo/branches/ACCUMULO-118/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java?rev=1494671&r1=1494670&r2=1494671&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java (original)
+++ accumulo/branches/ACCUMULO-118/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java Wed Jun 19 15:57:11 2013
@@ -31,7 +31,6 @@ import java.util.TreeMap;
 
 import junit.framework.TestCase;
 
-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;
@@ -51,12 +50,13 @@ import org.apache.accumulo.core.data.Val
 import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.util.MetadataTable;
 import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.core.util.RootTable;
 import org.apache.hadoop.io.Text;
 
 public class TabletLocatorImplTest extends TestCase {
   
-  private static final KeyExtent RTE = Constants.ROOT_TABLET_EXTENT;
-  private static final KeyExtent MTE = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), null, RTE.getEndRow());
+  private static final KeyExtent RTE = RootTable.ROOT_TABLET_EXTENT;
+  private static final KeyExtent MTE = new KeyExtent(new Text(MetadataTable.ID), null, RTE.getEndRow());
   private static TCredentials credential = null;
   
   static KeyExtent nke(String t, String er, String per) {
@@ -138,7 +138,7 @@ public class TabletLocatorImplTest exten
     TestInstance testInstance = new TestInstance("instance1", "tserver1");
     
     RootTabletLocator rtl = new RootTabletLocator(testInstance);
-    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rtl, ttlo);
+    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(MetadataTable.ID), rtl, ttlo);
     TabletLocatorImpl tab1TabletCache = new TabletLocatorImpl(new Text(table), rootTabletCache, ttlo);
     
     setLocation(tservers, rootTabLoc, RTE, MTE, metaTabLoc);
@@ -606,11 +606,11 @@ public class TabletLocatorImplTest exten
     if (location != null) {
       if (instance == null)
         instance = "";
-      Key lk = new Key(mr, Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY, new Text(instance));
+      Key lk = new Key(mr, MetadataTable.CURRENT_LOCATION_COLUMN_FAMILY, new Text(instance));
       tabletData.put(lk, new Value(location.getBytes()));
     }
     
-    Key pk = new Key(mr, Constants.METADATA_PREV_ROW_COLUMN.getColumnFamily(), Constants.METADATA_PREV_ROW_COLUMN.getColumnQualifier());
+    Key pk = new Key(mr, MetadataTable.PREV_ROW_COLUMN.getColumnFamily(), MetadataTable.PREV_ROW_COLUMN.getColumnQualifier());
     tabletData.put(pk, per);
   }
   
@@ -648,7 +648,7 @@ public class TabletLocatorImplTest exten
     TestInstance testInstance = new TestInstance("instance1", "tserver1");
     
     RootTabletLocator rtl = new RootTabletLocator(testInstance);
-    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rtl, ttlo);
+    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(MetadataTable.ID), rtl, ttlo);
     TabletLocatorImpl tab1TabletCache = new TabletLocatorImpl(new Text("tab1"), rootTabletCache, ttlo);
     
     locateTabletTest(tab1TabletCache, "r1", null, null, credential);
@@ -726,8 +726,8 @@ public class TabletLocatorImplTest exten
     locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver3", credential);
     
     // simulate the !METADATA table splitting
-    KeyExtent mte1 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), tab1e21.getMetadataEntry(), RTE.getEndRow());
-    KeyExtent mte2 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), null, tab1e21.getMetadataEntry());
+    KeyExtent mte1 = new KeyExtent(new Text(MetadataTable.ID), tab1e21.getMetadataEntry(), RTE.getEndRow());
+    KeyExtent mte2 = new KeyExtent(new Text(MetadataTable.ID), null, tab1e21.getMetadataEntry());
     
     setLocation(tservers, "tserver4", RTE, mte1, "tserver5");
     setLocation(tservers, "tserver4", RTE, mte2, "tserver6");
@@ -765,8 +765,8 @@ public class TabletLocatorImplTest exten
     locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credential);
     
     // simulate a hole in the !METADATA table, caused by a partial split
-    KeyExtent mte11 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), tab1e1.getMetadataEntry(), RTE.getEndRow());
-    KeyExtent mte12 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), tab1e21.getMetadataEntry(), tab1e1.getMetadataEntry());
+    KeyExtent mte11 = new KeyExtent(new Text(MetadataTable.ID), tab1e1.getMetadataEntry(), RTE.getEndRow());
+    KeyExtent mte12 = new KeyExtent(new Text(MetadataTable.ID), tab1e21.getMetadataEntry(), tab1e1.getMetadataEntry());
     deleteServer(tservers, "tserver10");
     setLocation(tservers, "tserver4", RTE, mte12, "tserver10");
     setLocation(tservers, "tserver10", mte12, tab1e21, "tserver12");
@@ -1172,15 +1172,15 @@ public class TabletLocatorImplTest exten
   
   public void testBug1() throws Exception {
     // a bug that occurred while running continuous ingest
-    KeyExtent mte1 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("0;0bc"), RTE.getEndRow());
-    KeyExtent mte2 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), null, new Text("0;0bc"));
+    KeyExtent mte1 = new KeyExtent(new Text(MetadataTable.ID), new Text("0;0bc"), RTE.getEndRow());
+    KeyExtent mte2 = new KeyExtent(new Text(MetadataTable.ID), null, new Text("0;0bc"));
     
     TServers tservers = new TServers();
     TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
     TestInstance testInstance = new TestInstance("instance1", "tserver1");
     
     RootTabletLocator rtl = new RootTabletLocator(testInstance);
-    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rtl, ttlo);
+    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(MetadataTable.ID), rtl, ttlo);
     TabletLocatorImpl tab0TabletCache = new TabletLocatorImpl(new Text("0"), rootTabletCache, ttlo);
     
     setLocation(tservers, "tserver1", RTE, mte1, "tserver2");
@@ -1199,15 +1199,15 @@ public class TabletLocatorImplTest exten
   
   public void testBug2() throws Exception {
     // a bug that occurred while running a functional test
-    KeyExtent mte1 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("~"), RTE.getEndRow());
-    KeyExtent mte2 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), null, new Text("~"));
+    KeyExtent mte1 = new KeyExtent(new Text(MetadataTable.ID), new Text("~"), RTE.getEndRow());
+    KeyExtent mte2 = new KeyExtent(new Text(MetadataTable.ID), null, new Text("~"));
     
     TServers tservers = new TServers();
     TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
     TestInstance testInstance = new TestInstance("instance1", "tserver1");
     
     RootTabletLocator rtl = new RootTabletLocator(testInstance);
-    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rtl, ttlo);
+    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(MetadataTable.ID), rtl, ttlo);
     TabletLocatorImpl tab0TabletCache = new TabletLocatorImpl(new Text("0"), rootTabletCache, ttlo);
     
     setLocation(tservers, "tserver1", RTE, mte1, "tserver2");
@@ -1224,11 +1224,11 @@ public class TabletLocatorImplTest exten
   
   // this test reproduces a problem where empty metadata tablets, that were created by user tablets being merged away, caused locating tablets to fail
   public void testBug3() throws Exception {
-    KeyExtent mte1 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("1;c"), RTE.getEndRow());
-    KeyExtent mte2 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("1;f"), new Text("1;c"));
-    KeyExtent mte3 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("1;j"), new Text("1;f"));
-    KeyExtent mte4 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), new Text("1;r"), new Text("1;j"));
-    KeyExtent mte5 = new KeyExtent(new Text(Constants.METADATA_TABLE_ID), null, new Text("1;r"));
+    KeyExtent mte1 = new KeyExtent(new Text(MetadataTable.ID), new Text("1;c"), RTE.getEndRow());
+    KeyExtent mte2 = new KeyExtent(new Text(MetadataTable.ID), new Text("1;f"), new Text("1;c"));
+    KeyExtent mte3 = new KeyExtent(new Text(MetadataTable.ID), new Text("1;j"), new Text("1;f"));
+    KeyExtent mte4 = new KeyExtent(new Text(MetadataTable.ID), new Text("1;r"), new Text("1;j"));
+    KeyExtent mte5 = new KeyExtent(new Text(MetadataTable.ID), null, new Text("1;r"));
     
     KeyExtent ke1 = new KeyExtent(new Text("1"), null, null);
     
@@ -1238,7 +1238,7 @@ public class TabletLocatorImplTest exten
     
     RootTabletLocator rtl = new RootTabletLocator(testInstance);
     
-    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rtl, ttlo);
+    TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(MetadataTable.ID), rtl, ttlo);
     TabletLocatorImpl tab0TabletCache = new TabletLocatorImpl(new Text("1"), rootTabletCache, ttlo);
     
     setLocation(tservers, "tserver1", RTE, mte1, "tserver2");

Propchange: accumulo/branches/ACCUMULO-118/examples/
------------------------------------------------------------------------------
  Merged /accumulo/trunk/examples:r1494234-1494649
  Merged /accumulo/branches/1.5/examples:r1494361,1494372

Propchange: accumulo/branches/ACCUMULO-118/fate/src/main/java/org/apache/accumulo/fate/ZooStore.java
------------------------------------------------------------------------------
  Merged /accumulo/branches/1.5/fate/src/main/java/org/apache/accumulo/fate/ZooStore.java:r1494361,1494372
  Merged /accumulo/trunk/fate/src/main/java/org/apache/accumulo/fate/ZooStore.java:r1494234-1494649

Propchange: accumulo/branches/ACCUMULO-118/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooSession.java
------------------------------------------------------------------------------
  Merged /accumulo/branches/1.5/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooSession.java:r1494361,1494372
  Merged /accumulo/trunk/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooSession.java:r1494234-1494649

Modified: accumulo/branches/ACCUMULO-118/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java?rev=1494671&r1=1494670&r2=1494671&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java (original)
+++ accumulo/branches/ACCUMULO-118/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java Wed Jun 19 15:57:11 2013
@@ -194,12 +194,6 @@ public class MiniAccumuloCluster {
    */
   public MiniAccumuloCluster(MiniAccumuloConfig config) throws IOException {
 
-    if (config.getDir().exists() && !config.getDir().isDirectory())
-      throw new IllegalArgumentException("Must pass in directory, " + config.getDir() + " is a file");
-
-    if (config.getDir().exists() && config.getDir().list().length != 0)
-      throw new IllegalArgumentException("Directory " + config.getDir() + " is not empty");
-
     this.config = config.initialize();
 
     config.getConfDir().mkdirs();

Modified: accumulo/branches/ACCUMULO-118/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java?rev=1494671&r1=1494670&r2=1494671&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java (original)
+++ accumulo/branches/ACCUMULO-118/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java Wed Jun 19 15:57:11 2013
@@ -71,6 +71,14 @@ public class MiniAccumuloConfig {
    * Set directories and fully populate site config
    */
   MiniAccumuloConfig initialize() {
+    
+    //Sanity checks
+    if (this.getDir().exists() && !this.getDir().isDirectory())
+      throw new IllegalArgumentException("Must pass in directory, " + this.getDir() + " is a file");
+
+    if (this.getDir().exists() && this.getDir().list().length != 0)
+      throw new IllegalArgumentException("Directory " + this.getDir() + " is not empty");
+    
     if (!initialized) {
       libDir = new File(dir, "lib");
       confDir = new File(dir, "conf");

Propchange: accumulo/branches/ACCUMULO-118/pom.xml
------------------------------------------------------------------------------
  Merged /accumulo/branches/1.5/pom.xml:r1494361,1494372
  Merged /accumulo/trunk/pom.xml:r1494234-1494649

Propchange: accumulo/branches/ACCUMULO-118/proxy/README
------------------------------------------------------------------------------
  Merged /accumulo/trunk/proxy/README:r1494234-1494649
  Merged /accumulo/branches/1.5/proxy/README:r1494361,1494372

Modified: accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/AccumuloProxy.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/AccumuloProxy.java?rev=1494671&r1=1494670&r2=1494671&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/AccumuloProxy.java (original)
+++ accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/AccumuloProxy.java Wed Jun 19 15:57:11 2013
@@ -10056,7 +10056,7 @@ import org.slf4j.LoggerFactory;
                   struct.splits = new HashSet<ByteBuffer>(2*_set132.size);
                   for (int _i133 = 0; _i133 < _set132.size; ++_i133)
                   {
-                    ByteBuffer _elem134; // required
+                    ByteBuffer _elem134; // optional
                     _elem134 = iprot.readBinary();
                     struct.splits.add(_elem134);
                   }
@@ -10167,7 +10167,7 @@ import org.slf4j.LoggerFactory;
             struct.splits = new HashSet<ByteBuffer>(2*_set137.size);
             for (int _i138 = 0; _i138 < _set137.size; ++_i138)
             {
-              ByteBuffer _elem139; // required
+              ByteBuffer _elem139; // optional
               _elem139 = iprot.readBinary();
               struct.splits.add(_elem139);
             }
@@ -11322,7 +11322,7 @@ import org.slf4j.LoggerFactory;
                   struct.scopes = new HashSet<IteratorScope>(2*_set140.size);
                   for (int _i141 = 0; _i141 < _set140.size; ++_i141)
                   {
-                    IteratorScope _elem142; // required
+                    IteratorScope _elem142; // optional
                     _elem142 = IteratorScope.findByValue(iprot.readI32());
                     struct.scopes.add(_elem142);
                   }
@@ -11449,7 +11449,7 @@ import org.slf4j.LoggerFactory;
             struct.scopes = new HashSet<IteratorScope>(2*_set145.size);
             for (int _i146 = 0; _i146 < _set145.size; ++_i146)
             {
-              IteratorScope _elem147; // required
+              IteratorScope _elem147; // optional
               _elem147 = IteratorScope.findByValue(iprot.readI32());
               struct.scopes.add(_elem147);
             }
@@ -12604,7 +12604,7 @@ import org.slf4j.LoggerFactory;
                   struct.scopes = new HashSet<IteratorScope>(2*_set148.size);
                   for (int _i149 = 0; _i149 < _set148.size; ++_i149)
                   {
-                    IteratorScope _elem150; // required
+                    IteratorScope _elem150; // optional
                     _elem150 = IteratorScope.findByValue(iprot.readI32());
                     struct.scopes.add(_elem150);
                   }
@@ -12731,7 +12731,7 @@ import org.slf4j.LoggerFactory;
             struct.scopes = new HashSet<IteratorScope>(2*_set153.size);
             for (int _i154 = 0; _i154 < _set153.size; ++_i154)
             {
-              IteratorScope _elem155; // required
+              IteratorScope _elem155; // optional
               _elem155 = IteratorScope.findByValue(iprot.readI32());
               struct.scopes.add(_elem155);
             }
@@ -14910,7 +14910,7 @@ import org.slf4j.LoggerFactory;
                   struct.propertiesToExclude = new HashSet<String>(2*_set160.size);
                   for (int _i161 = 0; _i161 < _set160.size; ++_i161)
                   {
-                    String _elem162; // required
+                    String _elem162; // optional
                     _elem162 = iprot.readString();
                     struct.propertiesToExclude.add(_elem162);
                   }
@@ -15090,7 +15090,7 @@ import org.slf4j.LoggerFactory;
             struct.propertiesToExclude = new HashSet<String>(2*_set171.size);
             for (int _i172 = 0; _i172 < _set171.size; ++_i172)
             {
-              String _elem173; // required
+              String _elem173; // optional
               _elem173 = iprot.readString();
               struct.propertiesToExclude.add(_elem173);
             }
@@ -16600,7 +16600,7 @@ import org.slf4j.LoggerFactory;
                   struct.iterators = new ArrayList<IteratorSetting>(_list174.size);
                   for (int _i175 = 0; _i175 < _list174.size; ++_i175)
                   {
-                    IteratorSetting _elem176; // required
+                    IteratorSetting _elem176; // optional
                     _elem176 = new IteratorSetting();
                     _elem176.read(iprot);
                     struct.iterators.add(_elem176);
@@ -16776,7 +16776,7 @@ import org.slf4j.LoggerFactory;
             struct.iterators = new ArrayList<IteratorSetting>(_list179.size);
             for (int _i180 = 0; _i180 < _list179.size; ++_i180)
             {
-              IteratorSetting _elem181; // required
+              IteratorSetting _elem181; // optional
               _elem181 = new IteratorSetting();
               _elem181.read(iprot);
               struct.iterators.add(_elem181);
@@ -24772,7 +24772,7 @@ import org.slf4j.LoggerFactory;
                   struct.tables = new HashSet<String>(2*_set182.size);
                   for (int _i183 = 0; _i183 < _set182.size; ++_i183)
                   {
-                    String _elem184; // required
+                    String _elem184; // optional
                     _elem184 = iprot.readString();
                     struct.tables.add(_elem184);
                   }
@@ -24868,7 +24868,7 @@ import org.slf4j.LoggerFactory;
             struct.tables = new HashSet<String>(2*_set187.size);
             for (int _i188 = 0; _i188 < _set187.size; ++_i188)
             {
-              String _elem189; // required
+              String _elem189; // optional
               _elem189 = iprot.readString();
               struct.tables.add(_elem189);
             }
@@ -25424,7 +25424,7 @@ import org.slf4j.LoggerFactory;
                   struct.success = new ArrayList<DiskUsage>(_list190.size);
                   for (int _i191 = 0; _i191 < _list190.size; ++_i191)
                   {
-                    DiskUsage _elem192; // required
+                    DiskUsage _elem192; // optional
                     _elem192 = new DiskUsage();
                     _elem192.read(iprot);
                     struct.success.add(_elem192);
@@ -25566,7 +25566,7 @@ import org.slf4j.LoggerFactory;
             struct.success = new ArrayList<DiskUsage>(_list195.size);
             for (int _i196 = 0; _i196 < _list195.size; ++_i196)
             {
-              DiskUsage _elem197; // required
+              DiskUsage _elem197; // optional
               _elem197 = new DiskUsage();
               _elem197.read(iprot);
               struct.success.add(_elem197);
@@ -26620,7 +26620,7 @@ import org.slf4j.LoggerFactory;
                       _val201 = new HashSet<String>(2*_set202.size);
                       for (int _i203 = 0; _i203 < _set202.size; ++_i203)
                       {
-                        String _elem204; // required
+                        String _elem204; // optional
                         _elem204 = iprot.readString();
                         _val201.add(_elem204);
                       }
@@ -26788,7 +26788,7 @@ import org.slf4j.LoggerFactory;
                 _val212 = new HashSet<String>(2*_set213.size);
                 for (int _i214 = 0; _i214 < _set213.size; ++_i214)
                 {
-                  String _elem215; // required
+                  String _elem215; // optional
                   _elem215 = iprot.readString();
                   _val212.add(_elem215);
                 }
@@ -28986,7 +28986,7 @@ import org.slf4j.LoggerFactory;
                   struct.auths = new HashSet<ByteBuffer>(2*_set216.size);
                   for (int _i217 = 0; _i217 < _set216.size; ++_i217)
                   {
-                    ByteBuffer _elem218; // required
+                    ByteBuffer _elem218; // optional
                     _elem218 = iprot.readBinary();
                     struct.auths.add(_elem218);
                   }
@@ -29169,7 +29169,7 @@ import org.slf4j.LoggerFactory;
             struct.auths = new HashSet<ByteBuffer>(2*_set221.size);
             for (int _i222 = 0; _i222 < _set221.size; ++_i222)
             {
-              ByteBuffer _elem223; // required
+              ByteBuffer _elem223; // optional
               _elem223 = iprot.readBinary();
               struct.auths.add(_elem223);
             }
@@ -34613,7 +34613,7 @@ import org.slf4j.LoggerFactory;
                   struct.success = new ArrayList<ByteBuffer>(_list234.size);
                   for (int _i235 = 0; _i235 < _list234.size; ++_i235)
                   {
-                    ByteBuffer _elem236; // required
+                    ByteBuffer _elem236; // optional
                     _elem236 = iprot.readBinary();
                     struct.success.add(_elem236);
                   }
@@ -34754,7 +34754,7 @@ import org.slf4j.LoggerFactory;
             struct.success = new ArrayList<ByteBuffer>(_list239.size);
             for (int _i240 = 0; _i240 < _list239.size; ++_i240)
             {
-              ByteBuffer _elem241; // required
+              ByteBuffer _elem241; // optional
               _elem241 = iprot.readBinary();
               struct.success.add(_elem241);
             }
@@ -35459,7 +35459,7 @@ import org.slf4j.LoggerFactory;
                   struct.success = new HashSet<String>(2*_set242.size);
                   for (int _i243 = 0; _i243 < _set242.size; ++_i243)
                   {
-                    String _elem244; // required
+                    String _elem244; // optional
                     _elem244 = iprot.readString();
                     struct.success.add(_elem244);
                   }
@@ -35540,7 +35540,7 @@ import org.slf4j.LoggerFactory;
             struct.success = new HashSet<String>(2*_set247.size);
             for (int _i248 = 0; _i248 < _set247.size; ++_i248)
             {
-              String _elem249; // required
+              String _elem249; // optional
               _elem249 = iprot.readString();
               struct.success.add(_elem249);
             }
@@ -36578,7 +36578,7 @@ import org.slf4j.LoggerFactory;
                       _val253 = new HashSet<IteratorScope>(2*_set254.size);
                       for (int _i255 = 0; _i255 < _set254.size; ++_i255)
                       {
-                        IteratorScope _elem256; // required
+                        IteratorScope _elem256; // optional
                         _elem256 = IteratorScope.findByValue(iprot.readI32());
                         _val253.add(_elem256);
                       }
@@ -36746,7 +36746,7 @@ import org.slf4j.LoggerFactory;
                 _val264 = new HashSet<IteratorScope>(2*_set265.size);
                 for (int _i266 = 0; _i266 < _set265.size; ++_i266)
                 {
-                  IteratorScope _elem267; // required
+                  IteratorScope _elem267; // optional
                   _elem267 = IteratorScope.findByValue(iprot.readI32());
                   _val264.add(_elem267);
                 }
@@ -42963,7 +42963,7 @@ import org.slf4j.LoggerFactory;
                   struct.scopes = new HashSet<IteratorScope>(2*_set278.size);
                   for (int _i279 = 0; _i279 < _set278.size; ++_i279)
                   {
-                    IteratorScope _elem280; // required
+                    IteratorScope _elem280; // optional
                     _elem280 = IteratorScope.findByValue(iprot.readI32());
                     struct.scopes.add(_elem280);
                   }
@@ -43089,7 +43089,7 @@ import org.slf4j.LoggerFactory;
             struct.scopes = new HashSet<IteratorScope>(2*_set283.size);
             for (int _i284 = 0; _i284 < _set283.size; ++_i284)
             {
-              IteratorScope _elem285; // required
+              IteratorScope _elem285; // optional
               _elem285 = IteratorScope.findByValue(iprot.readI32());
               struct.scopes.add(_elem285);
             }
@@ -46524,7 +46524,7 @@ import org.slf4j.LoggerFactory;
                       _val289 = new HashSet<String>(2*_set290.size);
                       for (int _i291 = 0; _i291 < _set290.size; ++_i291)
                       {
-                        String _elem292; // required
+                        String _elem292; // optional
                         _elem292 = iprot.readString();
                         _val289.add(_elem292);
                       }
@@ -46662,7 +46662,7 @@ import org.slf4j.LoggerFactory;
                 _val300 = new HashSet<String>(2*_set301.size);
                 for (int _i302 = 0; _i302 < _set301.size; ++_i302)
                 {
-                  String _elem303; // required
+                  String _elem303; // optional
                   _elem303 = iprot.readString();
                   _val300.add(_elem303);
                 }
@@ -49674,7 +49674,7 @@ import org.slf4j.LoggerFactory;
                   struct.success = new HashSet<Range>(2*_set304.size);
                   for (int _i305 = 0; _i305 < _set304.size; ++_i305)
                   {
-                    Range _elem306; // required
+                    Range _elem306; // optional
                     _elem306 = new Range();
                     _elem306.read(iprot);
                     struct.success.add(_elem306);
@@ -49816,7 +49816,7 @@ import org.slf4j.LoggerFactory;
             struct.success = new HashSet<Range>(2*_set309.size);
             for (int _i310 = 0; _i310 < _set309.size; ++_i310)
             {
-              Range _elem311; // required
+              Range _elem311; // optional
               _elem311 = new Range();
               _elem311.read(iprot);
               struct.success.add(_elem311);
@@ -54625,7 +54625,7 @@ import org.slf4j.LoggerFactory;
                   struct.success = new ArrayList<ActiveScan>(_list322.size);
                   for (int _i323 = 0; _i323 < _list322.size; ++_i323)
                   {
-                    ActiveScan _elem324; // required
+                    ActiveScan _elem324; // optional
                     _elem324 = new ActiveScan();
                     _elem324.read(iprot);
                     struct.success.add(_elem324);
@@ -54747,7 +54747,7 @@ import org.slf4j.LoggerFactory;
             struct.success = new ArrayList<ActiveScan>(_list327.size);
             for (int _i328 = 0; _i328 < _list327.size; ++_i328)
             {
-              ActiveScan _elem329; // required
+              ActiveScan _elem329; // optional
               _elem329 = new ActiveScan();
               _elem329.read(iprot);
               struct.success.add(_elem329);
@@ -55702,7 +55702,7 @@ import org.slf4j.LoggerFactory;
                   struct.success = new ArrayList<ActiveCompaction>(_list330.size);
                   for (int _i331 = 0; _i331 < _list330.size; ++_i331)
                   {
-                    ActiveCompaction _elem332; // required
+                    ActiveCompaction _elem332; // optional
                     _elem332 = new ActiveCompaction();
                     _elem332.read(iprot);
                     struct.success.add(_elem332);
@@ -55824,7 +55824,7 @@ import org.slf4j.LoggerFactory;
             struct.success = new ArrayList<ActiveCompaction>(_list335.size);
             for (int _i336 = 0; _i336 < _list335.size; ++_i336)
             {
-              ActiveCompaction _elem337; // required
+              ActiveCompaction _elem337; // optional
               _elem337 = new ActiveCompaction();
               _elem337.read(iprot);
               struct.success.add(_elem337);
@@ -58497,7 +58497,7 @@ import org.slf4j.LoggerFactory;
                   struct.success = new ArrayList<String>(_list358.size);
                   for (int _i359 = 0; _i359 < _list358.size; ++_i359)
                   {
-                    String _elem360; // required
+                    String _elem360; // optional
                     _elem360 = iprot.readString();
                     struct.success.add(_elem360);
                   }
@@ -58578,7 +58578,7 @@ import org.slf4j.LoggerFactory;
             struct.success = new ArrayList<String>(_list363.size);
             for (int _i364 = 0; _i364 < _list363.size; ++_i364)
             {
-              String _elem365; // required
+              String _elem365; // optional
               _elem365 = iprot.readString();
               struct.success.add(_elem365);
             }
@@ -63341,7 +63341,7 @@ import org.slf4j.LoggerFactory;
                   struct.authorizations = new HashSet<ByteBuffer>(2*_set376.size);
                   for (int _i377 = 0; _i377 < _set376.size; ++_i377)
                   {
-                    ByteBuffer _elem378; // required
+                    ByteBuffer _elem378; // optional
                     _elem378 = iprot.readBinary();
                     struct.authorizations.add(_elem378);
                   }
@@ -63452,7 +63452,7 @@ import org.slf4j.LoggerFactory;
             struct.authorizations = new HashSet<ByteBuffer>(2*_set381.size);
             for (int _i382 = 0; _i382 < _set381.size; ++_i382)
             {
-              ByteBuffer _elem383; // required
+              ByteBuffer _elem383; // optional
               _elem383 = iprot.readBinary();
               struct.authorizations.add(_elem383);
             }
@@ -67847,7 +67847,7 @@ import org.slf4j.LoggerFactory;
                   struct.success = new ArrayList<ByteBuffer>(_list384.size);
                   for (int _i385 = 0; _i385 < _list384.size; ++_i385)
                   {
-                    ByteBuffer _elem386; // required
+                    ByteBuffer _elem386; // optional
                     _elem386 = iprot.readBinary();
                     struct.success.add(_elem386);
                   }
@@ -67968,7 +67968,7 @@ import org.slf4j.LoggerFactory;
             struct.success = new ArrayList<ByteBuffer>(_list389.size);
             for (int _i390 = 0; _i390 < _list389.size; ++_i390)
             {
-              ByteBuffer _elem391; // required
+              ByteBuffer _elem391; // optional
               _elem391 = iprot.readBinary();
               struct.success.add(_elem391);
             }
@@ -73659,7 +73659,7 @@ import org.slf4j.LoggerFactory;
                   struct.success = new HashSet<String>(2*_set392.size);
                   for (int _i393 = 0; _i393 < _set392.size; ++_i393)
                   {
-                    String _elem394; // required
+                    String _elem394; // optional
                     _elem394 = iprot.readString();
                     struct.success.add(_elem394);
                   }
@@ -73800,7 +73800,7 @@ import org.slf4j.LoggerFactory;
             struct.success = new HashSet<String>(2*_set397.size);
             for (int _i398 = 0; _i398 < _set397.size; ++_i398)
             {
-              String _elem399; // required
+              String _elem399; // optional
               _elem399 = iprot.readString();
               struct.success.add(_elem399);
             }
@@ -82735,7 +82735,7 @@ import org.slf4j.LoggerFactory;
                       _val403 = new ArrayList<ColumnUpdate>(_list404.size);
                       for (int _i405 = 0; _i405 < _list404.size; ++_i405)
                       {
-                        ColumnUpdate _elem406; // required
+                        ColumnUpdate _elem406; // optional
                         _elem406 = new ColumnUpdate();
                         _elem406.read(iprot);
                         _val403.add(_elem406);
@@ -82874,7 +82874,7 @@ import org.slf4j.LoggerFactory;
                 _val414 = new ArrayList<ColumnUpdate>(_list415.size);
                 for (int _i416 = 0; _i416 < _list415.size; ++_i416)
                 {
-                  ColumnUpdate _elem417; // required
+                  ColumnUpdate _elem417; // optional
                   _elem417 = new ColumnUpdate();
                   _elem417.read(iprot);
                   _val414.add(_elem417);
@@ -85198,7 +85198,7 @@ import org.slf4j.LoggerFactory;
                       _val421 = new ArrayList<ColumnUpdate>(_list422.size);
                       for (int _i423 = 0; _i423 < _list422.size; ++_i423)
                       {
-                        ColumnUpdate _elem424; // required
+                        ColumnUpdate _elem424; // optional
                         _elem424 = new ColumnUpdate();
                         _elem424.read(iprot);
                         _val421.add(_elem424);
@@ -85322,7 +85322,7 @@ import org.slf4j.LoggerFactory;
                 _val432 = new ArrayList<ColumnUpdate>(_list433.size);
                 for (int _i434 = 0; _i434 < _list433.size; ++_i434)
                 {
-                  ColumnUpdate _elem435; // required
+                  ColumnUpdate _elem435; // optional
                   _elem435 = new ColumnUpdate();
                   _elem435.read(iprot);
                   _val432.add(_elem435);

Modified: accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ActiveCompaction.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ActiveCompaction.java?rev=1494671&r1=1494670&r2=1494671&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ActiveCompaction.java (original)
+++ accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ActiveCompaction.java Wed Jun 19 15:57:11 2013
@@ -1116,7 +1116,7 @@ import org.slf4j.LoggerFactory;
                 struct.inputFiles = new ArrayList<String>(_list106.size);
                 for (int _i107 = 0; _i107 < _list106.size; ++_i107)
                 {
-                  String _elem108; // required
+                  String _elem108; // optional
                   _elem108 = iprot.readString();
                   struct.inputFiles.add(_elem108);
                 }
@@ -1182,7 +1182,7 @@ import org.slf4j.LoggerFactory;
                 struct.iterators = new ArrayList<IteratorSetting>(_list109.size);
                 for (int _i110 = 0; _i110 < _list109.size; ++_i110)
                 {
-                  IteratorSetting _elem111; // required
+                  IteratorSetting _elem111; // optional
                   _elem111 = new IteratorSetting();
                   _elem111.read(iprot);
                   struct.iterators.add(_elem111);
@@ -1379,7 +1379,7 @@ import org.slf4j.LoggerFactory;
           struct.inputFiles = new ArrayList<String>(_list116.size);
           for (int _i117 = 0; _i117 < _list116.size; ++_i117)
           {
-            String _elem118; // required
+            String _elem118; // optional
             _elem118 = iprot.readString();
             struct.inputFiles.add(_elem118);
           }
@@ -1416,7 +1416,7 @@ import org.slf4j.LoggerFactory;
           struct.iterators = new ArrayList<IteratorSetting>(_list119.size);
           for (int _i120 = 0; _i120 < _list119.size; ++_i120)
           {
-            IteratorSetting _elem121; // required
+            IteratorSetting _elem121; // optional
             _elem121 = new IteratorSetting();
             _elem121.read(iprot);
             struct.iterators.add(_elem121);

Modified: accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ActiveScan.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ActiveScan.java?rev=1494671&r1=1494670&r2=1494671&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ActiveScan.java (original)
+++ accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ActiveScan.java Wed Jun 19 15:57:11 2013
@@ -1267,7 +1267,7 @@ import org.slf4j.LoggerFactory;
                 struct.columns = new ArrayList<Column>(_list82.size);
                 for (int _i83 = 0; _i83 < _list82.size; ++_i83)
                 {
-                  Column _elem84; // required
+                  Column _elem84; // optional
                   _elem84 = new Column();
                   _elem84.read(iprot);
                   struct.columns.add(_elem84);
@@ -1286,7 +1286,7 @@ import org.slf4j.LoggerFactory;
                 struct.iterators = new ArrayList<IteratorSetting>(_list85.size);
                 for (int _i86 = 0; _i86 < _list85.size; ++_i86)
                 {
-                  IteratorSetting _elem87; // required
+                  IteratorSetting _elem87; // optional
                   _elem87 = new IteratorSetting();
                   _elem87.read(iprot);
                   struct.iterators.add(_elem87);
@@ -1305,7 +1305,7 @@ import org.slf4j.LoggerFactory;
                 struct.authorizations = new ArrayList<ByteBuffer>(_list88.size);
                 for (int _i89 = 0; _i89 < _list88.size; ++_i89)
                 {
-                  ByteBuffer _elem90; // required
+                  ByteBuffer _elem90; // optional
                   _elem90 = iprot.readBinary();
                   struct.authorizations.add(_elem90);
                 }
@@ -1551,7 +1551,7 @@ import org.slf4j.LoggerFactory;
           struct.columns = new ArrayList<Column>(_list97.size);
           for (int _i98 = 0; _i98 < _list97.size; ++_i98)
           {
-            Column _elem99; // required
+            Column _elem99; // optional
             _elem99 = new Column();
             _elem99.read(iprot);
             struct.columns.add(_elem99);
@@ -1565,7 +1565,7 @@ import org.slf4j.LoggerFactory;
           struct.iterators = new ArrayList<IteratorSetting>(_list100.size);
           for (int _i101 = 0; _i101 < _list100.size; ++_i101)
           {
-            IteratorSetting _elem102; // required
+            IteratorSetting _elem102; // optional
             _elem102 = new IteratorSetting();
             _elem102.read(iprot);
             struct.iterators.add(_elem102);
@@ -1579,7 +1579,7 @@ import org.slf4j.LoggerFactory;
           struct.authorizations = new ArrayList<ByteBuffer>(_list103.size);
           for (int _i104 = 0; _i104 < _list103.size; ++_i104)
           {
-            ByteBuffer _elem105; // required
+            ByteBuffer _elem105; // optional
             _elem105 = iprot.readBinary();
             struct.authorizations.add(_elem105);
           }

Modified: accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/BatchScanOptions.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/BatchScanOptions.java?rev=1494671&r1=1494670&r2=1494671&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/BatchScanOptions.java (original)
+++ accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/BatchScanOptions.java Wed Jun 19 15:57:11 2013
@@ -724,7 +724,7 @@ import org.slf4j.LoggerFactory;
                 struct.authorizations = new HashSet<ByteBuffer>(2*_set50.size);
                 for (int _i51 = 0; _i51 < _set50.size; ++_i51)
                 {
-                  ByteBuffer _elem52; // required
+                  ByteBuffer _elem52; // optional
                   _elem52 = iprot.readBinary();
                   struct.authorizations.add(_elem52);
                 }
@@ -742,7 +742,7 @@ import org.slf4j.LoggerFactory;
                 struct.ranges = new ArrayList<Range>(_list53.size);
                 for (int _i54 = 0; _i54 < _list53.size; ++_i54)
                 {
-                  Range _elem55; // required
+                  Range _elem55; // optional
                   _elem55 = new Range();
                   _elem55.read(iprot);
                   struct.ranges.add(_elem55);
@@ -761,7 +761,7 @@ import org.slf4j.LoggerFactory;
                 struct.columns = new ArrayList<ScanColumn>(_list56.size);
                 for (int _i57 = 0; _i57 < _list56.size; ++_i57)
                 {
-                  ScanColumn _elem58; // required
+                  ScanColumn _elem58; // optional
                   _elem58 = new ScanColumn();
                   _elem58.read(iprot);
                   struct.columns.add(_elem58);
@@ -780,7 +780,7 @@ import org.slf4j.LoggerFactory;
                 struct.iterators = new ArrayList<IteratorSetting>(_list59.size);
                 for (int _i60 = 0; _i60 < _list59.size; ++_i60)
                 {
-                  IteratorSetting _elem61; // required
+                  IteratorSetting _elem61; // optional
                   _elem61 = new IteratorSetting();
                   _elem61.read(iprot);
                   struct.iterators.add(_elem61);
@@ -961,7 +961,7 @@ import org.slf4j.LoggerFactory;
           struct.authorizations = new HashSet<ByteBuffer>(2*_set70.size);
           for (int _i71 = 0; _i71 < _set70.size; ++_i71)
           {
-            ByteBuffer _elem72; // required
+            ByteBuffer _elem72; // optional
             _elem72 = iprot.readBinary();
             struct.authorizations.add(_elem72);
           }
@@ -974,7 +974,7 @@ import org.slf4j.LoggerFactory;
           struct.ranges = new ArrayList<Range>(_list73.size);
           for (int _i74 = 0; _i74 < _list73.size; ++_i74)
           {
-            Range _elem75; // required
+            Range _elem75; // optional
             _elem75 = new Range();
             _elem75.read(iprot);
             struct.ranges.add(_elem75);
@@ -988,7 +988,7 @@ import org.slf4j.LoggerFactory;
           struct.columns = new ArrayList<ScanColumn>(_list76.size);
           for (int _i77 = 0; _i77 < _list76.size; ++_i77)
           {
-            ScanColumn _elem78; // required
+            ScanColumn _elem78; // optional
             _elem78 = new ScanColumn();
             _elem78.read(iprot);
             struct.columns.add(_elem78);
@@ -1002,7 +1002,7 @@ import org.slf4j.LoggerFactory;
           struct.iterators = new ArrayList<IteratorSetting>(_list79.size);
           for (int _i80 = 0; _i80 < _list79.size; ++_i80)
           {
-            IteratorSetting _elem81; // required
+            IteratorSetting _elem81; // optional
             _elem81 = new IteratorSetting();
             _elem81.read(iprot);
             struct.iterators.add(_elem81);

Modified: accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/DiskUsage.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/DiskUsage.java?rev=1494671&r1=1494670&r2=1494671&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/DiskUsage.java (original)
+++ accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/DiskUsage.java Wed Jun 19 15:57:11 2013
@@ -436,7 +436,7 @@ import org.slf4j.LoggerFactory;
                 struct.tables = new ArrayList<String>(_list0.size);
                 for (int _i1 = 0; _i1 < _list0.size; ++_i1)
                 {
-                  String _elem2; // required
+                  String _elem2; // optional
                   _elem2 = iprot.readString();
                   struct.tables.add(_elem2);
                 }
@@ -534,7 +534,7 @@ import org.slf4j.LoggerFactory;
           struct.tables = new ArrayList<String>(_list5.size);
           for (int _i6 = 0; _i6 < _list5.size; ++_i6)
           {
-            String _elem7; // required
+            String _elem7; // optional
             _elem7 = iprot.readString();
             struct.tables.add(_elem7);
           }

Modified: accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ScanOptions.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ScanOptions.java?rev=1494671&r1=1494670&r2=1494671&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ScanOptions.java (original)
+++ accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ScanOptions.java Wed Jun 19 15:57:11 2013
@@ -707,7 +707,7 @@ import org.slf4j.LoggerFactory;
                 struct.authorizations = new HashSet<ByteBuffer>(2*_set26.size);
                 for (int _i27 = 0; _i27 < _set26.size; ++_i27)
                 {
-                  ByteBuffer _elem28; // required
+                  ByteBuffer _elem28; // optional
                   _elem28 = iprot.readBinary();
                   struct.authorizations.add(_elem28);
                 }
@@ -734,7 +734,7 @@ import org.slf4j.LoggerFactory;
                 struct.columns = new ArrayList<ScanColumn>(_list29.size);
                 for (int _i30 = 0; _i30 < _list29.size; ++_i30)
                 {
-                  ScanColumn _elem31; // required
+                  ScanColumn _elem31; // optional
                   _elem31 = new ScanColumn();
                   _elem31.read(iprot);
                   struct.columns.add(_elem31);
@@ -753,7 +753,7 @@ import org.slf4j.LoggerFactory;
                 struct.iterators = new ArrayList<IteratorSetting>(_list32.size);
                 for (int _i33 = 0; _i33 < _list32.size; ++_i33)
                 {
-                  IteratorSetting _elem34; // required
+                  IteratorSetting _elem34; // optional
                   _elem34 = new IteratorSetting();
                   _elem34.read(iprot);
                   struct.iterators.add(_elem34);
@@ -921,7 +921,7 @@ import org.slf4j.LoggerFactory;
           struct.authorizations = new HashSet<ByteBuffer>(2*_set41.size);
           for (int _i42 = 0; _i42 < _set41.size; ++_i42)
           {
-            ByteBuffer _elem43; // required
+            ByteBuffer _elem43; // optional
             _elem43 = iprot.readBinary();
             struct.authorizations.add(_elem43);
           }
@@ -939,7 +939,7 @@ import org.slf4j.LoggerFactory;
           struct.columns = new ArrayList<ScanColumn>(_list44.size);
           for (int _i45 = 0; _i45 < _list44.size; ++_i45)
           {
-            ScanColumn _elem46; // required
+            ScanColumn _elem46; // optional
             _elem46 = new ScanColumn();
             _elem46.read(iprot);
             struct.columns.add(_elem46);
@@ -953,7 +953,7 @@ import org.slf4j.LoggerFactory;
           struct.iterators = new ArrayList<IteratorSetting>(_list47.size);
           for (int _i48 = 0; _i48 < _list47.size; ++_i48)
           {
-            IteratorSetting _elem49; // required
+            IteratorSetting _elem49; // optional
             _elem49 = new IteratorSetting();
             _elem49.read(iprot);
             struct.iterators.add(_elem49);

Modified: accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ScanResult.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ScanResult.java?rev=1494671&r1=1494670&r2=1494671&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ScanResult.java (original)
+++ accumulo/branches/ACCUMULO-118/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ScanResult.java Wed Jun 19 15:57:11 2013
@@ -436,7 +436,7 @@ import org.slf4j.LoggerFactory;
                 struct.results = new ArrayList<KeyValue>(_list8.size);
                 for (int _i9 = 0; _i9 < _list8.size; ++_i9)
                 {
-                  KeyValue _elem10; // required
+                  KeyValue _elem10; // optional
                   _elem10 = new KeyValue();
                   _elem10.read(iprot);
                   struct.results.add(_elem10);
@@ -535,7 +535,7 @@ import org.slf4j.LoggerFactory;
           struct.results = new ArrayList<KeyValue>(_list13.size);
           for (int _i14 = 0; _i14 < _list13.size; ++_i14)
           {
-            KeyValue _elem15; // required
+            KeyValue _elem15; // optional
             _elem15 = new KeyValue();
             _elem15.read(iprot);
             struct.results.add(_elem15);

Modified: accumulo/branches/ACCUMULO-118/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java?rev=1494671&r1=1494670&r2=1494671&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java (original)
+++ accumulo/branches/ACCUMULO-118/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java Wed Jun 19 15:57:11 2013
@@ -40,7 +40,6 @@ import java.util.Set;
 import java.util.TreeMap;
 import java.util.UUID;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -51,6 +50,7 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.user.SummingCombiner;
 import org.apache.accumulo.core.iterators.user.VersioningIterator;
+import org.apache.accumulo.core.util.MetadataTable;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.examples.simple.constraints.NumericValueConstraint;
 import org.apache.accumulo.minicluster.MiniAccumuloCluster;
@@ -1111,7 +1111,7 @@ public class SimpleTest {
     ScanOptions opt = new ScanOptions();
     opt.range = new Range(start, true, end, false);
     opt.columns = Collections.singletonList(new ScanColumn(s2bb("file")));
-    String scanner = client.createScanner(creds, Constants.METADATA_TABLE_NAME, opt);
+    String scanner = client.createScanner(creds, MetadataTable.NAME, opt);
     int result = 0;
     while (true) {
       ScanResult more = client.nextK(scanner, 100);

Propchange: accumulo/branches/ACCUMULO-118/server/
------------------------------------------------------------------------------
  Merged /accumulo/branches/1.5/server:r1494361,1494372
  Merged /accumulo/trunk/server:r1494234-1494649

Modified: accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/ServerConstants.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/ServerConstants.java?rev=1494671&r1=1494670&r2=1494671&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/ServerConstants.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/ServerConstants.java Wed Jun 19 15:57:11 2013
@@ -16,11 +16,10 @@
  */
 package org.apache.accumulo.server;
 
-import static org.apache.accumulo.core.Constants.METADATA_TABLE_ID;
-import static org.apache.accumulo.core.Constants.ZROOT_TABLET;
-
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.core.util.MetadataTable;
+import org.apache.accumulo.core.util.RootTable;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -83,10 +82,10 @@ public class ServerConstants {
   }
   
   public static String[] getMetadataTableDirs() {
-    return prefix(getTablesDirs(), METADATA_TABLE_ID);
+    return prefix(getTablesDirs(), MetadataTable.ID);
   }
   
   public static String getRootTabletDir() {
-    return prefix(getMetadataTableDirs(), ZROOT_TABLET)[0];
+    return prefix(getMetadataTableDirs(), RootTable.ZROOT_TABLET)[0];
   }
 }

Modified: accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java?rev=1494671&r1=1494670&r2=1494671&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java Wed Jun 19 15:57:11 2013
@@ -32,7 +32,6 @@ import java.util.concurrent.ExecutorServ
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
-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.Instance;
@@ -56,6 +55,7 @@ import org.apache.accumulo.core.security
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.LoggingRunnable;
+import org.apache.accumulo.core.util.MetadataTable;
 import org.apache.accumulo.core.util.NamingThreadFactory;
 import org.apache.accumulo.core.util.StopWatch;
 import org.apache.accumulo.core.util.ThriftUtil;
@@ -281,7 +281,7 @@ public class BulkImporter {
     log.debug("BULK IMPORT TIMING STATISTICS");
     log.debug(String.format("Examine map files    : %,10.2f secs %6.2f%s", timer.getSecs(Timers.EXAMINE_MAP_FILES), 100.0 * timer.get(Timers.EXAMINE_MAP_FILES)
         / timer.get(Timers.TOTAL), "%"));
-    log.debug(String.format("Query %-14s : %,10.2f secs %6.2f%s", Constants.METADATA_TABLE_NAME, timer.getSecs(Timers.QUERY_METADATA),
+    log.debug(String.format("Query %-14s : %,10.2f secs %6.2f%s", MetadataTable.NAME, timer.getSecs(Timers.QUERY_METADATA),
         100.0 * timer.get(Timers.QUERY_METADATA) / timer.get(Timers.TOTAL), "%"));
     log.debug(String.format("Import Map Files     : %,10.2f secs %6.2f%s", timer.getSecs(Timers.IMPORT_MAP_FILES), 100.0 * timer.get(Timers.IMPORT_MAP_FILES)
         / timer.get(Timers.TOTAL), "%"));