You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2013/12/28 02:03:14 UTC

[1/7] ACCUMULO-1965 Fix exception handling for namespaces

Updated Branches:
  refs/heads/1.6.0-SNAPSHOT c41900edc -> f35e3f472


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java b/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
index c4adb64..8354d56 100644
--- a/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
@@ -14,17 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.accumulo.test;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.EnumSet;
-import java.util.HashSet;
 import java.util.Iterator;
+import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Random;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
@@ -32,344 +39,503 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 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.MutationsRejectedException;
+import org.apache.accumulo.core.client.NamespaceExistsException;
 import org.apache.accumulo.core.client.NamespaceNotEmptyException;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.NamespaceOperations;
+import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.core.client.impl.thrift.TableOperation;
+import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
+import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+import org.apache.accumulo.core.client.security.SecurityErrorCode;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.Filter;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.iterators.user.VersioningIterator;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.examples.simple.constraints.NumericValueConstraint;
 import org.apache.accumulo.test.functional.SimpleMacIT;
+import org.apache.hadoop.io.Text;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 
 public class NamespacesIT extends SimpleMacIT {
 
-  Random random = new Random();
+  private Connector c;
+  private String namespace;
+
+  @Before
+  public void setUpConnectorAndNamespace() throws Exception {
+    c = getConnector();
+    namespace = "ns_" + getTableNames(1)[0];
+  }
+
+  @After
+  public void removeTablesAndNamespaces() throws Exception {
+    for (String t : c.tableOperations().list())
+      if (!Tables.qualify(t).getFirst().equals(Namespaces.ACCUMULO_NAMESPACE))
+        c.tableOperations().delete(t);
+    assertEquals(2, c.tableOperations().list().size());
+    for (String n : c.namespaceOperations().list())
+      if (!n.equals(Namespaces.ACCUMULO_NAMESPACE) && !n.equals(Namespaces.DEFAULT_NAMESPACE))
+        c.namespaceOperations().delete(n);
+    assertEquals(2, c.namespaceOperations().list().size());
+    for (String u : c.securityOperations().listLocalUsers())
+      if (!"root".equals(u))
+        c.securityOperations().dropLocalUser(u);
+    assertEquals(1, c.securityOperations().listLocalUsers().size());
+  }
 
-  /**
-   * This test creates a table without specifying a namespace. In this case, it puts the table into the default namespace.
-   */
   @Test
-  public void testDefaultNamespace() throws Exception {
-    String tableName = "test";
-    Connector c = getConnector();
+  public void checkBuiltInNamespaces() throws Exception {
+    assertTrue(c.namespaceOperations().exists(Namespaces.DEFAULT_NAMESPACE));
+    assertTrue(c.namespaceOperations().exists(Namespaces.ACCUMULO_NAMESPACE));
+  }
 
-    assertTrue(c.namespaceOperations().exists(Constants.DEFAULT_NAMESPACE));
+  @Test
+  public void createTableInDefaultNamespace() throws Exception {
+    String tableName = getTableNames(1)[0];
     c.tableOperations().create(tableName);
     assertTrue(c.tableOperations().exists(tableName));
   }
 
-  /**
-   * This test creates a new namespace "testing" and a table "testing.table1" which puts "table1" into the "testing" namespace. Then we create "testing.table2"
-   * which creates "table2" and puts it into "testing" as well. Then we make sure that you can't delete a namespace with tables in it, and then we delete the
-   * tables and delete the namespace.
-   */
-  @Test
-  public void testCreateAndDeleteNamespace() throws Exception {
-    String namespace = "testing";
-    String tableName1 = namespace + ".table1";
-    String tableName2 = namespace + ".table2";
-
-    Connector c = getConnector();
+  // TODO enable after ACCUMULO-2079 is fixed
+  @Ignore
+  @Test(expected = AccumuloSecurityException.class)
+  public void createTableInAccumuloNamespace() throws Exception {
+    String tableName = Namespaces.ACCUMULO_NAMESPACE + "." + getTableNames(1)[0];
+    assertFalse(c.tableOperations().exists(tableName));
+    c.tableOperations().create(tableName); // should fail
+  }
 
-    c.namespaceOperations().create(namespace);
-    assertTrue(c.namespaceOperations().exists(namespace));
+  @Test(expected = AccumuloSecurityException.class)
+  public void deleteDefaultNamespace() throws Exception {
+    c.namespaceOperations().delete(Namespaces.DEFAULT_NAMESPACE); // should fail
+  }
 
-    c.tableOperations().create(tableName1);
-    assertTrue(c.tableOperations().exists(tableName1));
+  @Test(expected = AccumuloSecurityException.class)
+  public void deleteAccumuloNamespace() throws Exception {
+    c.namespaceOperations().delete(Namespaces.ACCUMULO_NAMESPACE); // should fail
+  }
 
-    c.tableOperations().create(tableName2);
-    assertTrue(c.tableOperations().exists(tableName2));
+  @Test
+  public void createTableInMissingNamespace() throws Exception {
+    String t = namespace + ".1";
+    assertFalse(c.namespaceOperations().exists(namespace));
+    assertFalse(c.tableOperations().exists(t));
+    try {
+      c.tableOperations().create(t);
+      fail();
+    } catch (AccumuloException e) {
+      assertEquals(NamespaceNotFoundException.class.getName(), e.getCause().getClass().getName());
+      assertFalse(c.namespaceOperations().exists(namespace));
+      assertFalse(c.tableOperations().exists(t));
+    }
+  }
 
-    // deleting
+  @Test
+  public void createAndDeleteNamespace() throws Exception {
+    String t1 = namespace + ".1";
+    String t2 = namespace + ".2";
+    assertFalse(c.namespaceOperations().exists(namespace));
+    assertFalse(c.tableOperations().exists(t1));
+    assertFalse(c.tableOperations().exists(t2));
     try {
-      // can't delete a namespace with tables in it
       c.namespaceOperations().delete(namespace);
-      fail();
-    } catch (NamespaceNotEmptyException e) {
-      // ignore, supposed to happen
+    } catch (NamespaceNotFoundException e) {}
+    try {
+      c.tableOperations().delete(t1);
+    } catch (TableNotFoundException e) {
+      assertEquals(NamespaceNotFoundException.class.getName(), e.getCause().getClass().getName());
     }
+    c.namespaceOperations().create(namespace);
     assertTrue(c.namespaceOperations().exists(namespace));
-    assertTrue(c.tableOperations().exists(tableName1));
-    assertTrue(c.tableOperations().exists(tableName2));
-
-    c.tableOperations().delete(tableName2);
-    assertTrue(!c.tableOperations().exists(tableName2));
+    assertFalse(c.tableOperations().exists(t1));
+    assertFalse(c.tableOperations().exists(t2));
+    c.tableOperations().create(t1);
     assertTrue(c.namespaceOperations().exists(namespace));
-
-    c.tableOperations().delete(tableName1);
-    assertTrue(!c.tableOperations().exists(tableName1));
+    assertTrue(c.tableOperations().exists(t1));
+    assertFalse(c.tableOperations().exists(t2));
+    c.tableOperations().create(t2);
+    assertTrue(c.namespaceOperations().exists(namespace));
+    assertTrue(c.tableOperations().exists(t1));
+    assertTrue(c.tableOperations().exists(t2));
+    c.tableOperations().delete(t1);
+    assertTrue(c.namespaceOperations().exists(namespace));
+    assertFalse(c.tableOperations().exists(t1));
+    assertTrue(c.tableOperations().exists(t2));
+    c.tableOperations().delete(t2);
+    assertTrue(c.namespaceOperations().exists(namespace));
+    assertFalse(c.tableOperations().exists(t1));
+    assertFalse(c.tableOperations().exists(t2));
     c.namespaceOperations().delete(namespace);
-    assertTrue(!c.namespaceOperations().exists(namespace));
+    assertFalse(c.namespaceOperations().exists(namespace));
+    assertFalse(c.tableOperations().exists(t1));
+    assertFalse(c.tableOperations().exists(t2));
   }
 
-  /**
-   * This test creates a namespace, modifies it's properties, and checks to make sure that those properties are applied to its tables. To do something on a
-   * namespace-wide level, use NamespaceOperations.
-   * 
-   * Checks to make sure namespace-level properties are overridden by table-level properties.
-   * 
-   * Checks to see if the default namespace's properties work as well.
-   */
-
-  @Test
-  public void testNamespaceProperties() throws Exception {
-    String namespace = "propchange";
-    String tableName1 = namespace + ".table1";
-    String tableName2 = namespace + ".table2";
-
-    String propKey = Property.TABLE_SCAN_MAXMEM.getKey();
-    String propVal = "42K";
-
-    Connector c = getConnector();
-
+  @Test(expected = NamespaceNotEmptyException.class)
+  public void deleteNonEmptyNamespace() throws Exception {
+    String tableName1 = namespace + ".1";
+    assertFalse(c.namespaceOperations().exists(namespace));
+    assertFalse(c.tableOperations().exists(tableName1));
     c.namespaceOperations().create(namespace);
     c.tableOperations().create(tableName1);
-    c.namespaceOperations().setProperty(namespace, propKey, propVal);
+    assertTrue(c.namespaceOperations().exists(namespace));
+    assertTrue(c.tableOperations().exists(tableName1));
+    c.namespaceOperations().delete(namespace); // should fail
+  }
 
-    // check the namespace has the property
-    assertTrue(checkNamespaceHasProp(c, namespace, propKey, propVal));
+  @Test
+  public void verifyPropertyInheritance() throws Exception {
+    String t0 = "0";
+    String t1 = namespace + ".1";
+    String t2 = namespace + ".2";
 
-    // check that the table gets it from the namespace
-    assertTrue(checkTableHasProp(c, tableName1, propKey, propVal));
+    String k = Property.TABLE_SCAN_MAXMEM.getKey();
+    String v = "42K";
 
-    // test a second table to be sure the first wasn't magical
-    // (also, changed the order, the namespace has the property already)
-    c.tableOperations().create(tableName2);
-    assertTrue(checkTableHasProp(c, tableName2, propKey, propVal));
+    assertFalse(c.namespaceOperations().exists(namespace));
+    assertFalse(c.tableOperations().exists(t1));
+    assertFalse(c.tableOperations().exists(t2));
+    c.namespaceOperations().create(namespace);
+    c.tableOperations().create(t1);
+    c.tableOperations().create(t0);
+    assertTrue(c.namespaceOperations().exists(namespace));
+    assertTrue(c.tableOperations().exists(t1));
+    assertTrue(c.tableOperations().exists(t0));
+
+    // verify no property
+    assertFalse(checkNamespaceHasProp(namespace, k, v));
+    assertFalse(checkTableHasProp(t1, k, v));
+    assertFalse(checkNamespaceHasProp(Namespaces.DEFAULT_NAMESPACE, k, v));
+    assertFalse(checkTableHasProp(t0, k, v));
+
+    // set property and verify
+    c.namespaceOperations().setProperty(namespace, k, v);
+    assertTrue(checkNamespaceHasProp(namespace, k, v));
+    assertTrue(checkTableHasProp(t1, k, v));
+    assertFalse(checkNamespaceHasProp(Namespaces.DEFAULT_NAMESPACE, k, v));
+    assertFalse(checkTableHasProp(t0, k, v));
+
+    // add a new table to namespace and verify
+    assertFalse(c.tableOperations().exists(t2));
+    c.tableOperations().create(t2);
+    assertTrue(c.tableOperations().exists(t2));
+    assertTrue(checkNamespaceHasProp(namespace, k, v));
+    assertTrue(checkTableHasProp(t1, k, v));
+    assertTrue(checkTableHasProp(t2, k, v));
+    assertFalse(checkNamespaceHasProp(Namespaces.DEFAULT_NAMESPACE, k, v));
+    assertFalse(checkTableHasProp(t0, k, v));
+
+    // remove property and verify
+    c.namespaceOperations().removeProperty(namespace, k);
+    assertFalse(checkNamespaceHasProp(namespace, k, v));
+    assertFalse(checkTableHasProp(t1, k, v));
+    assertFalse(checkTableHasProp(t2, k, v));
+    assertFalse(checkNamespaceHasProp(Namespaces.DEFAULT_NAMESPACE, k, v));
+    assertFalse(checkTableHasProp(t0, k, v));
+
+    // set property on default namespace and verify
+    c.namespaceOperations().setProperty(Namespaces.DEFAULT_NAMESPACE, k, v);
+    assertFalse(checkNamespaceHasProp(namespace, k, v));
+    assertFalse(checkTableHasProp(t1, k, v));
+    assertFalse(checkTableHasProp(t2, k, v));
+    assertTrue(checkNamespaceHasProp(Namespaces.DEFAULT_NAMESPACE, k, v));
+    assertTrue(checkTableHasProp(t0, k, v));
 
     // test that table properties override namespace properties
-    String propKey2 = Property.TABLE_FILE_MAX.getKey();
-    String propVal2 = "42";
-    String tablePropVal = "13";
+    String k2 = Property.TABLE_FILE_MAX.getKey();
+    String v2 = "42";
+    String table_v2 = "13";
 
-    c.tableOperations().setProperty(tableName2, propKey2, tablePropVal);
-    c.namespaceOperations().setProperty("propchange", propKey2, propVal2);
-
-    assertTrue(checkTableHasProp(c, tableName2, propKey2, tablePropVal));
-
-    // now check that you can change the default namespace's properties
-    propVal = "13K";
-    String tableName = "some_table";
-    c.tableOperations().create(tableName);
-    c.namespaceOperations().setProperty(Constants.DEFAULT_NAMESPACE, propKey, propVal);
+    // set new property on some
+    c.namespaceOperations().setProperty(namespace, k2, v2);
+    c.tableOperations().setProperty(t2, k2, table_v2);
+    assertTrue(checkNamespaceHasProp(namespace, k2, v2));
+    assertTrue(checkTableHasProp(t1, k2, v2));
+    assertTrue(checkTableHasProp(t2, k2, table_v2));
 
-    assertTrue(checkTableHasProp(c, tableName, propKey, propVal));
-
-    // test the properties server-side by configuring an iterator.
-    // should not show anything with column-family = 'a'
-    String tableName3 = namespace + ".table3";
-    c.tableOperations().create(tableName3);
+    c.tableOperations().delete(t1);
+    c.tableOperations().delete(t2);
+    c.tableOperations().delete(t0);
+    c.namespaceOperations().delete(namespace);
+  }
 
-    IteratorSetting setting = new IteratorSetting(250, "thing", SimpleFilter.class.getName());
-    c.namespaceOperations().attachIterator(namespace, setting);
+  @Test
+  public void verifyIteratorInheritance() throws Exception {
+    String t1 = namespace + ".1";
+    c.namespaceOperations().create(namespace);
+    c.tableOperations().create(t1);
+    String iterName = namespace + "_iter";
 
-    BatchWriter bw = c.createBatchWriter(tableName3, new BatchWriterConfig());
+    BatchWriter bw = c.createBatchWriter(t1, new BatchWriterConfig());
     Mutation m = new Mutation("r");
     m.put("a", "b", new Value("abcde".getBytes()));
     bw.addMutation(m);
     bw.flush();
     bw.close();
 
-    Scanner s = c.createScanner(tableName3, Authorizations.EMPTY);
-    assertTrue(!s.iterator().hasNext());
-  }
-
-  /**
-   * This test renames and clones two separate table into different namespaces. different namespace.
-   * 
-   */
-  @Test
-  public void testCloneTableToNewNamespace() throws Exception {
-    Connector c = getConnector();
-
-    String[] uniqueNames = getTableNames(2);
-    String namespace1 = uniqueNames[0];
-    String namespace2 = uniqueNames[1];
-    String tableName1 = namespace1 + ".table1";
-    String tableName2 = namespace2 + ".table2";
+    IteratorSetting setting = new IteratorSetting(250, iterName, SimpleFilter.class.getName());
 
-    c.namespaceOperations().create(namespace1);
-    c.tableOperations().create(tableName1);
-    assertTrue(c.tableOperations().exists(tableName1));
+    // verify can see inserted entry
+    Scanner s = c.createScanner(t1, Authorizations.EMPTY);
+    assertTrue(s.iterator().hasNext());
+    assertFalse(c.namespaceOperations().listIterators(namespace).containsKey(iterName));
+    assertFalse(c.tableOperations().listIterators(t1).containsKey(iterName));
 
-    c.namespaceOperations().create(namespace2);
-    c.tableOperations().clone(tableName1, tableName2, false, null, null);
-
-    assertTrue(c.tableOperations().exists(tableName1));
-    assertTrue(c.tableOperations().exists(tableName2));
-  }
-
-  /**
-   * This test renames a namespace and ensures that its tables are still correct
-   */
-  @Test
-  public void testNamespaceRename() throws Exception {
-    String namespace1 = "n1";
-    String namespace2 = "n2";
-    String table = "t";
-
-    Connector c = getConnector();
-    Instance instance = c.getInstance();
-
-    c.namespaceOperations().create(namespace1);
-    c.tableOperations().create(namespace1 + "." + table);
-
-    c.namespaceOperations().rename(namespace1, namespace2);
-
-    assertTrue(!c.namespaceOperations().exists(namespace1));
-    assertTrue(c.namespaceOperations().exists(namespace2));
-    assertTrue(c.tableOperations().exists(namespace2 + "." + table));
-    String tid = Tables.getTableId(instance, namespace2 + "." + table);
-    String tnid = Tables.getNamespace(instance, tid);
-    String tnamespace = Namespaces.getNamespaceName(instance, tnid);
-    assertTrue(namespace2.equals(tnamespace));
+    // verify entry is filtered out (also, verify conflict checking API)
+    c.namespaceOperations().checkIteratorConflicts(namespace, setting, EnumSet.allOf(IteratorScope.class));
+    c.namespaceOperations().attachIterator(namespace, setting);
+    try {
+      c.namespaceOperations().checkIteratorConflicts(namespace, setting, EnumSet.allOf(IteratorScope.class));
+      fail();
+    } catch (AccumuloException e) {
+      assertEquals(IllegalArgumentException.class.getName(), e.getCause().getClass().getName());
+    }
+    IteratorSetting setting2 = c.namespaceOperations().getIteratorSetting(namespace, setting.getName(), IteratorScope.scan);
+    assertEquals(setting, setting2);
+    s = c.createScanner(t1, Authorizations.EMPTY);
+    assertFalse(s.iterator().hasNext());
+    assertTrue(c.namespaceOperations().listIterators(namespace).containsKey(iterName));
+    assertTrue(c.tableOperations().listIterators(t1).containsKey(iterName));
+
+    // verify can see inserted entry again
+    c.namespaceOperations().removeIterator(namespace, setting.getName(), EnumSet.allOf(IteratorScope.class));
+    s = c.createScanner(t1, Authorizations.EMPTY);
+    assertTrue(s.iterator().hasNext());
+    assertFalse(c.namespaceOperations().listIterators(namespace).containsKey(iterName));
+    assertFalse(c.tableOperations().listIterators(t1).containsKey(iterName));
   }
 
-  /**
-   * This test clones a table to a different namespace and ensures it's properties are correct
-   */
   @Test
-  public void testCloneTableProperties() throws Exception {
-    String[] uniqueNames = getTableNames(2);
-    String n1 = uniqueNames[0];
-    String n2 = uniqueNames[1];
-    String t1 = n1 + ".table";
-    String t2 = n2 + ".table";
-
-    String propKey = Property.TABLE_FILE_MAX.getKey();
-    String propVal1 = "55";
-    String propVal2 = "66";
-
-    Connector c = getConnector();
+  public void cloneTable() throws Exception {
+    String namespace2 = namespace + "_clone";
+    String t1 = namespace + ".1";
+    String t2 = namespace + ".2";
+    String t3 = namespace2 + ".2";
+    String k1 = Property.TABLE_FILE_MAX.getKey();
+    String k2 = Property.TABLE_FILE_REPLICATION.getKey();
+    String k1v1 = "55";
+    String k1v2 = "66";
+    String k2v1 = "5";
+    String k2v2 = "6";
 
-    c.namespaceOperations().create(n1);
+    c.namespaceOperations().create(namespace);
     c.tableOperations().create(t1);
+    assertTrue(c.tableOperations().exists(t1));
+    assertFalse(c.namespaceOperations().exists(namespace2));
+    assertFalse(c.tableOperations().exists(t2));
+    assertFalse(c.tableOperations().exists(t3));
 
-    c.tableOperations().removeProperty(t1, Property.TABLE_FILE_MAX.getKey());
-    c.namespaceOperations().setProperty(n1, propKey, propVal1);
-
-    assertTrue(checkTableHasProp(c, t1, propKey, propVal1));
+    try {
+      // try to clone before namespace exists
+      c.tableOperations().clone(t1, t3, false, null, null); // should fail
+      fail();
+    } catch (AccumuloException e) {
+      assertEquals(NamespaceNotFoundException.class.getName(), e.getCause().getClass().getName());
+    }
 
-    c.namespaceOperations().create(n2);
-    c.namespaceOperations().setProperty(n2, propKey, propVal2);
-    c.tableOperations().clone(t1, t2, true, null, null);
-    c.tableOperations().removeProperty(t2, propKey);
+    // try to clone before when target tables exist
+    c.namespaceOperations().create(namespace2);
+    c.tableOperations().create(t2);
+    c.tableOperations().create(t3);
+    for (String t : Arrays.asList(t2, t3)) {
+      try {
+        c.tableOperations().clone(t1, t, false, null, null); // should fail
+        fail();
+      } catch (TableExistsException e) {
+        c.tableOperations().delete(t);
+      }
+    }
 
-    assertTrue(checkTableHasProp(c, t2, propKey, propVal2));
+    assertTrue(c.tableOperations().exists(t1));
+    assertTrue(c.namespaceOperations().exists(namespace2));
+    assertFalse(c.tableOperations().exists(t2));
+    assertFalse(c.tableOperations().exists(t3));
+
+    // set property with different values in two namespaces and a separate property with different values on the table and both namespaces
+    assertFalse(checkNamespaceHasProp(namespace, k1, k1v1));
+    assertFalse(checkNamespaceHasProp(namespace2, k1, k1v2));
+    assertFalse(checkTableHasProp(t1, k1, k1v1));
+    assertFalse(checkTableHasProp(t1, k1, k1v2));
+    assertFalse(checkNamespaceHasProp(namespace, k2, k2v1));
+    assertFalse(checkNamespaceHasProp(namespace2, k2, k2v1));
+    assertFalse(checkTableHasProp(t1, k2, k2v1));
+    assertFalse(checkTableHasProp(t1, k2, k2v2));
+    c.namespaceOperations().setProperty(namespace, k1, k1v1);
+    c.namespaceOperations().setProperty(namespace2, k1, k1v2);
+    c.namespaceOperations().setProperty(namespace, k2, k2v1);
+    c.namespaceOperations().setProperty(namespace2, k2, k2v1);
+    c.tableOperations().setProperty(t1, k2, k2v2);
+    assertTrue(checkNamespaceHasProp(namespace, k1, k1v1));
+    assertTrue(checkNamespaceHasProp(namespace2, k1, k1v2));
+    assertTrue(checkTableHasProp(t1, k1, k1v1));
+    assertFalse(checkTableHasProp(t1, k1, k1v2));
+    assertTrue(checkNamespaceHasProp(namespace, k2, k2v1));
+    assertTrue(checkNamespaceHasProp(namespace2, k2, k2v1));
+    assertFalse(checkTableHasProp(t1, k2, k2v1));
+    assertTrue(checkTableHasProp(t1, k2, k2v2));
+
+    // clone twice, once in same namespace, once in another
+    for (String t : Arrays.asList(t2, t3))
+      c.tableOperations().clone(t1, t, false, null, null);
 
-    c.tableOperations().delete(t1);
-    c.tableOperations().delete(t2);
-    c.namespaceOperations().delete(n1);
-    c.namespaceOperations().delete(n2);
+    assertTrue(c.namespaceOperations().exists(namespace2));
+    assertTrue(c.tableOperations().exists(t1));
+    assertTrue(c.tableOperations().exists(t2));
+    assertTrue(c.tableOperations().exists(t3));
+
+    // verify the properties got transferred
+    assertTrue(checkTableHasProp(t1, k1, k1v1));
+    assertTrue(checkTableHasProp(t2, k1, k1v1));
+    assertTrue(checkTableHasProp(t3, k1, k1v2));
+    assertTrue(checkTableHasProp(t1, k2, k2v2));
+    assertTrue(checkTableHasProp(t2, k2, k2v2));
+    assertTrue(checkTableHasProp(t3, k2, k2v2));
   }
 
-  /**
-   * This tests adding iterators to a namespace, listing them, and removing them
-   */
   @Test
-  public void testNamespaceIterators() throws Exception {
-    Connector c = getConnector();
-
-    String namespace = "iterator";
-    String tableName = namespace + ".table";
-    String iter = "thing";
+  public void renameNamespaceWithTable() throws Exception {
+    String namespace2 = namespace + "_renamed";
+    String t1 = namespace + ".t";
+    String t2 = namespace2 + ".t";
 
     c.namespaceOperations().create(namespace);
-    c.tableOperations().create(tableName);
+    c.tableOperations().create(t1);
+    assertTrue(c.namespaceOperations().exists(namespace));
+    assertTrue(c.tableOperations().exists(t1));
+    assertFalse(c.namespaceOperations().exists(namespace2));
+    assertFalse(c.tableOperations().exists(t2));
 
-    IteratorSetting setting = new IteratorSetting(250, iter, SimpleFilter.class.getName());
-    HashSet<IteratorScope> scope = new HashSet<IteratorScope>();
-    scope.add(IteratorScope.scan);
-    c.namespaceOperations().attachIterator(namespace, setting, EnumSet.copyOf(scope));
+    String namespaceId = c.namespaceOperations().namespaceIdMap().get(namespace);
+    String tableId = c.tableOperations().tableIdMap().get(t1);
 
-    BatchWriter bw = c.createBatchWriter(tableName, new BatchWriterConfig());
-    Mutation m = new Mutation("r");
-    m.put("a", "b", new Value("abcde".getBytes(Constants.UTF8)));
-    bw.addMutation(m);
-    bw.flush();
+    c.namespaceOperations().rename(namespace, namespace2);
+    assertFalse(c.namespaceOperations().exists(namespace));
+    assertFalse(c.tableOperations().exists(t1));
+    assertTrue(c.namespaceOperations().exists(namespace2));
+    assertTrue(c.tableOperations().exists(t2));
 
-    Scanner s = c.createScanner(tableName, Authorizations.EMPTY);
-    assertTrue(!s.iterator().hasNext());
+    // verify id's didn't change
+    String namespaceId2 = c.namespaceOperations().namespaceIdMap().get(namespace2);
+    String tableId2 = c.tableOperations().tableIdMap().get(t2);
 
-    assertTrue(c.namespaceOperations().listIterators(namespace).containsKey(iter));
+    assertEquals(namespaceId, namespaceId2);
+    assertEquals(tableId, tableId2);
   }
 
-  /**
-   * This tests adding iterators to a namespace, listing them, and removing them as well as adding and removing constraints
-   */
   @Test
-  public void testNamespaceConstraints() throws Exception {
-    Connector c = getConnector();
-
-    String namespace = "constraint";
-    String tableName = namespace + ".table";
-    String iter = "thing";
-
+  public void verifyConstraintInheritance() throws Exception {
+    String t1 = namespace + ".1";
     c.namespaceOperations().create(namespace);
-    c.tableOperations().create(tableName, false);
-
-    c.namespaceOperations().removeIterator(namespace, iter, EnumSet.of(IteratorScope.scan));
-
-    c.namespaceOperations().addConstraint(namespace, NumericValueConstraint.class.getName());
-    // doesn't take effect immediately, needs time to propagate
+    c.tableOperations().create(t1, false);
+    String constraintClassName = NumericValueConstraint.class.getName();
+
+    assertFalse(c.namespaceOperations().listConstraints(namespace).containsKey(constraintClassName));
+    assertFalse(c.tableOperations().listConstraints(t1).containsKey(constraintClassName));
+
+    c.namespaceOperations().addConstraint(namespace, constraintClassName);
+    assertTrue(c.namespaceOperations().listConstraints(namespace).containsKey(constraintClassName));
+    assertTrue(c.tableOperations().listConstraints(t1).containsKey(constraintClassName));
+    int num = c.namespaceOperations().listConstraints(namespace).get(constraintClassName);
+    assertEquals(num, (int) c.tableOperations().listConstraints(t1).get(constraintClassName));
+    // doesn't take effect immediately, needs time to propagate to tserver's ZooKeeper cache
     UtilWaitThread.sleep(250);
 
-    Mutation m = new Mutation("rowy");
-    m.put("a", "b", new Value("abcde".getBytes(Constants.UTF8)));
+    Mutation m1 = new Mutation("r1");
+    Mutation m2 = new Mutation("r2");
+    Mutation m3 = new Mutation("r3");
+    m1.put("a", "b", new Value("abcde".getBytes(Constants.UTF8)));
+    m2.put("e", "f", new Value("123".getBytes(Constants.UTF8)));
+    m3.put("c", "d", new Value("zyxwv".getBytes(Constants.UTF8)));
+    BatchWriter bw = c.createBatchWriter(t1, new BatchWriterConfig());
+    bw.addMutations(Arrays.asList(m1, m2, m3));
     try {
-      BatchWriter bw = c.createBatchWriter(tableName, new BatchWriterConfig());
-      bw.addMutation(m);
-      bw.flush();
       bw.close();
       fail();
     } catch (MutationsRejectedException e) {
-      // supposed to be thrown
+      assertEquals(1, e.getConstraintViolationSummaries().size());
+      assertEquals(2, e.getConstraintViolationSummaries().get(0).getNumberOfViolatingMutations());
     }
-    int num = c.namespaceOperations().listConstraints(namespace).get(NumericValueConstraint.class.getName());
     c.namespaceOperations().removeConstraint(namespace, num);
+    assertFalse(c.namespaceOperations().listConstraints(namespace).containsKey(constraintClassName));
+    assertFalse(c.tableOperations().listConstraints(t1).containsKey(constraintClassName));
+    // doesn't take effect immediately, needs time to propagate to tserver's ZooKeeper cache
+    UtilWaitThread.sleep(250);
+
+    bw = c.createBatchWriter(t1, new BatchWriterConfig());
+    bw.addMutations(Arrays.asList(m1, m2, m3));
+    bw.close();
   }
 
-  /**
-   * Tests disallowed rename across namespaces
-   */
   @Test
-  public void testRenameTable() throws Exception {
-    Connector c = getConnector();
-
-    String[] uniqueNames = getTableNames(4);
-    String namespace1 = uniqueNames[0];
-    String namespace2 = uniqueNames[1];
-    String tableName1 = namespace1 + "." + uniqueNames[2];
-    String tableName2 = namespace2 + "." + uniqueNames[3];
-    String tableName3 = namespace1 + "." + uniqueNames[3];
+  public void renameTable() throws Exception {
+    String namespace2 = namespace + "_renamed";
+    String t1 = namespace + ".1";
+    String t2 = namespace2 + ".2";
+    String t3 = namespace + ".3";
+    String t4 = namespace + ".4";
 
-    c.namespaceOperations().create(namespace1);
+    c.namespaceOperations().create(namespace);
     c.namespaceOperations().create(namespace2);
-    c.tableOperations().create(tableName1);
+
+    assertTrue(c.namespaceOperations().exists(namespace));
+    assertTrue(c.namespaceOperations().exists(namespace2));
+    assertFalse(c.tableOperations().exists(t1));
+    assertFalse(c.tableOperations().exists(t2));
+    assertFalse(c.tableOperations().exists(t3));
+    assertFalse(c.tableOperations().exists(t4));
+
+    c.tableOperations().create(t1);
 
     try {
-      c.tableOperations().rename(tableName1, tableName2);
+      c.tableOperations().rename(t1, t2);
       fail();
     } catch (AccumuloException e) {
       // this is expected, because we don't allow renames across namespaces
+      assertEquals(ThriftTableOperationException.class.getName(), e.getCause().getClass().getName());
+      assertEquals(TableOperation.RENAME, ((ThriftTableOperationException) e.getCause()).getOp());
+      assertEquals(TableOperationExceptionType.INVALID_NAME, ((ThriftTableOperationException) e.getCause()).getType());
+      assertTrue(c.tableOperations().exists(t1));
+      assertFalse(c.tableOperations().exists(t2));
+      assertFalse(c.tableOperations().exists(t3));
+      assertFalse(c.tableOperations().exists(t4));
     }
 
-    c.tableOperations().rename(tableName1, tableName3);
+    // fully qualified rename
+    c.tableOperations().rename(t1, t3);
+    assertFalse(c.tableOperations().exists(t1));
+    assertFalse(c.tableOperations().exists(t2));
+    assertTrue(c.tableOperations().exists(t3));
+    assertFalse(c.tableOperations().exists(t4));
+
+    // unqualified rename
+    c.tableOperations().rename(t3, Tables.qualify(t4).getSecond());
+    assertFalse(c.tableOperations().exists(t1));
+    assertFalse(c.tableOperations().exists(t2));
+    assertFalse(c.tableOperations().exists(t3));
+    assertTrue(c.tableOperations().exists(t4));
   }
 
   /**
@@ -378,187 +544,556 @@ public class NamespacesIT extends SimpleMacIT {
    */
   @Test
   public void testPermissions() throws Exception {
-    Connector c = getConnector();
-
-    String[] uniqueNames = getTableNames(8);
-    String user1 = uniqueNames[0];
-    String user2 = uniqueNames[1];
-    PasswordToken pass = new PasswordToken(uniqueNames[2]);
-    String n1 = uniqueNames[3];
-    String n2 = uniqueNames[4];
-    String t1 = uniqueNames[5];
-    String t2 = uniqueNames[6];
-    String t3 = uniqueNames[7];
+    String u1 = "u1";
+    String u2 = "u2";
+    PasswordToken pass = new PasswordToken("pass");
+
+    String n1 = namespace;
+    String t1 = n1 + ".1";
+    String t2 = n1 + ".2";
+    String t3 = n1 + ".3";
+
+    String n2 = namespace + "_2";
 
     c.namespaceOperations().create(n1);
-    c.tableOperations().create(n1 + "." + t1);
+    c.tableOperations().create(t1);
 
-    c.securityOperations().createLocalUser(user1, pass);
+    c.securityOperations().createLocalUser(u1, pass);
 
-    Connector user1Con = getConnector().getInstance().getConnector(user1, pass);
+    Connector user1Con = c.getInstance().getConnector(u1, pass);
 
     try {
-      user1Con.tableOperations().create(n1 + "." + t2);
+      user1Con.tableOperations().create(t2);
       fail();
     } catch (AccumuloSecurityException e) {
-      // supposed to happen
+      expectPermissionDenied(e);
     }
 
-    c.securityOperations().grantNamespacePermission(user1, n1, NamespacePermission.CREATE_TABLE);
-    user1Con.tableOperations().create(n1 + "." + t2);
-    assertTrue(c.tableOperations().list().contains(n1 + "." + t2));
-    c.securityOperations().revokeNamespacePermission(user1, n1, NamespacePermission.CREATE_TABLE);
+    c.securityOperations().grantNamespacePermission(u1, n1, NamespacePermission.CREATE_TABLE);
+    user1Con.tableOperations().create(t2);
+    assertTrue(c.tableOperations().list().contains(t2));
+    c.securityOperations().revokeNamespacePermission(u1, n1, NamespacePermission.CREATE_TABLE);
 
     try {
-      user1Con.tableOperations().delete(n1 + "." + t1);
+      user1Con.tableOperations().delete(t1);
       fail();
     } catch (AccumuloSecurityException e) {
-      // should happen
+      expectPermissionDenied(e);
     }
 
-    c.securityOperations().grantNamespacePermission(user1, n1, NamespacePermission.DROP_TABLE);
-    user1Con.tableOperations().delete(n1 + "." + t1);
-    assertTrue(!c.tableOperations().list().contains(n1 + "." + t1));
-    c.securityOperations().revokeNamespacePermission(user1, n1, NamespacePermission.DROP_TABLE);
+    c.securityOperations().grantNamespacePermission(u1, n1, NamespacePermission.DROP_TABLE);
+    user1Con.tableOperations().delete(t1);
+    assertTrue(!c.tableOperations().list().contains(t1));
+    c.securityOperations().revokeNamespacePermission(u1, n1, NamespacePermission.DROP_TABLE);
 
-    c.tableOperations().create(n1 + "." + t3);
-    BatchWriter bw = c.createBatchWriter(n1 + "." + t3, null);
+    c.tableOperations().create(t3);
+    BatchWriter bw = c.createBatchWriter(t3, null);
     Mutation m = new Mutation("row");
     m.put("cf", "cq", "value");
     bw.addMutation(m);
     bw.close();
 
-    Iterator<Entry<Key,Value>> i = user1Con.createScanner(n1 + "." + t3, new Authorizations()).iterator();
+    Iterator<Entry<Key,Value>> i = user1Con.createScanner(t3, new Authorizations()).iterator();
     try {
       i.next();
       fail();
     } catch (RuntimeException e) {
-      // yup
+      assertEquals(AccumuloSecurityException.class.getName(), e.getCause().getClass().getName());
+      expectPermissionDenied((AccumuloSecurityException) e.getCause());
     }
 
-    m = new Mutation(user1);
+    m = new Mutation(u1);
     m.put("cf", "cq", "turtles");
-    bw = user1Con.createBatchWriter(n1 + "." + t3, null);
+    bw = user1Con.createBatchWriter(t3, null);
     try {
       bw.addMutation(m);
       bw.close();
       fail();
     } catch (MutationsRejectedException e) {
-      // good
+      assertEquals(1, e.getAuthorizationFailuresMap().size());
+      assertEquals(1, e.getAuthorizationFailuresMap().entrySet().iterator().next().getValue().size());
+      switch (e.getAuthorizationFailuresMap().entrySet().iterator().next().getValue().iterator().next()) {
+        case PERMISSION_DENIED:
+          break;
+        default:
+          fail();
+      }
     }
 
-    c.securityOperations().grantNamespacePermission(user1, n1, NamespacePermission.READ);
-    i = user1Con.createScanner(n1 + "." + t3, new Authorizations()).iterator();
+    c.securityOperations().grantNamespacePermission(u1, n1, NamespacePermission.READ);
+    i = user1Con.createScanner(t3, new Authorizations()).iterator();
     assertTrue(i.hasNext());
-    c.securityOperations().revokeNamespacePermission(user1, n1, NamespacePermission.READ);
+    c.securityOperations().revokeNamespacePermission(u1, n1, NamespacePermission.READ);
 
-    c.securityOperations().grantNamespacePermission(user1, n1, NamespacePermission.WRITE);
-    m = new Mutation(user1);
+    c.securityOperations().grantNamespacePermission(u1, n1, NamespacePermission.WRITE);
+    m = new Mutation(u1);
     m.put("cf", "cq", "turtles");
-    bw = user1Con.createBatchWriter(n1 + "." + t3, null);
+    bw = user1Con.createBatchWriter(t3, null);
     bw.addMutation(m);
     bw.close();
-    c.securityOperations().revokeNamespacePermission(user1, n1, NamespacePermission.WRITE);
+    c.securityOperations().revokeNamespacePermission(u1, n1, NamespacePermission.WRITE);
 
     try {
-      user1Con.tableOperations().setProperty(n1 + "." + t3, Property.TABLE_FILE_MAX.getKey(), "42");
+      user1Con.tableOperations().setProperty(t3, Property.TABLE_FILE_MAX.getKey(), "42");
       fail();
-    } catch (AccumuloSecurityException e) {}
+    } catch (AccumuloSecurityException e) {
+      expectPermissionDenied(e);
+    }
 
-    c.securityOperations().grantNamespacePermission(user1, n1, NamespacePermission.ALTER_TABLE);
-    user1Con.tableOperations().setProperty(n1 + "." + t3, Property.TABLE_FILE_MAX.getKey(), "42");
-    user1Con.tableOperations().removeProperty(n1 + "." + t3, Property.TABLE_FILE_MAX.getKey());
-    c.securityOperations().revokeNamespacePermission(user1, n1, NamespacePermission.ALTER_TABLE);
+    c.securityOperations().grantNamespacePermission(u1, n1, NamespacePermission.ALTER_TABLE);
+    user1Con.tableOperations().setProperty(t3, Property.TABLE_FILE_MAX.getKey(), "42");
+    user1Con.tableOperations().removeProperty(t3, Property.TABLE_FILE_MAX.getKey());
+    c.securityOperations().revokeNamespacePermission(u1, n1, NamespacePermission.ALTER_TABLE);
 
     try {
       user1Con.namespaceOperations().setProperty(n1, Property.TABLE_FILE_MAX.getKey(), "55");
       fail();
-    } catch (AccumuloSecurityException e) {}
+    } catch (AccumuloSecurityException e) {
+      expectPermissionDenied(e);
+    }
 
-    c.securityOperations().grantNamespacePermission(user1, n1, NamespacePermission.ALTER_NAMESPACE);
+    c.securityOperations().grantNamespacePermission(u1, n1, NamespacePermission.ALTER_NAMESPACE);
     user1Con.namespaceOperations().setProperty(n1, Property.TABLE_FILE_MAX.getKey(), "42");
     user1Con.namespaceOperations().removeProperty(n1, Property.TABLE_FILE_MAX.getKey());
-    c.securityOperations().revokeNamespacePermission(user1, n1, NamespacePermission.ALTER_NAMESPACE);
+    c.securityOperations().revokeNamespacePermission(u1, n1, NamespacePermission.ALTER_NAMESPACE);
 
-    c.securityOperations().createLocalUser(user2, pass);
+    c.securityOperations().createLocalUser(u2, pass);
     try {
-      user1Con.securityOperations().grantNamespacePermission(user2, n1, NamespacePermission.ALTER_NAMESPACE);
+      user1Con.securityOperations().grantNamespacePermission(u2, n1, NamespacePermission.ALTER_NAMESPACE);
       fail();
-    } catch (AccumuloSecurityException e) {}
+    } catch (AccumuloSecurityException e) {
+      expectPermissionDenied(e);
+    }
 
-    c.securityOperations().grantNamespacePermission(user1, n1, NamespacePermission.GRANT);
-    user1Con.securityOperations().grantNamespacePermission(user2, n1, NamespacePermission.ALTER_NAMESPACE);
-    user1Con.securityOperations().revokeNamespacePermission(user2, n1, NamespacePermission.ALTER_NAMESPACE);
-    c.securityOperations().revokeNamespacePermission(user1, n1, NamespacePermission.GRANT);
+    c.securityOperations().grantNamespacePermission(u1, n1, NamespacePermission.GRANT);
+    user1Con.securityOperations().grantNamespacePermission(u2, n1, NamespacePermission.ALTER_NAMESPACE);
+    user1Con.securityOperations().revokeNamespacePermission(u2, n1, NamespacePermission.ALTER_NAMESPACE);
+    c.securityOperations().revokeNamespacePermission(u1, n1, NamespacePermission.GRANT);
 
     try {
       user1Con.namespaceOperations().create(n2);
       fail();
-    } catch (AccumuloSecurityException e) {}
+    } catch (AccumuloSecurityException e) {
+      expectPermissionDenied(e);
+    }
 
-    c.securityOperations().grantSystemPermission(user1, SystemPermission.CREATE_NAMESPACE);
+    c.securityOperations().grantSystemPermission(u1, SystemPermission.CREATE_NAMESPACE);
     user1Con.namespaceOperations().create(n2);
-    c.securityOperations().revokeSystemPermission(user1, SystemPermission.CREATE_NAMESPACE);
+    c.securityOperations().revokeSystemPermission(u1, SystemPermission.CREATE_NAMESPACE);
 
     try {
       user1Con.namespaceOperations().delete(n2);
       fail();
-    } catch (AccumuloSecurityException e) {}
+    } catch (AccumuloSecurityException e) {
+      expectPermissionDenied(e);
+    }
 
-    c.securityOperations().grantSystemPermission(user1, SystemPermission.DROP_NAMESPACE);
+    c.securityOperations().grantSystemPermission(u1, SystemPermission.DROP_NAMESPACE);
     user1Con.namespaceOperations().delete(n2);
-    c.securityOperations().revokeSystemPermission(user1, SystemPermission.DROP_NAMESPACE);
+    c.securityOperations().revokeSystemPermission(u1, SystemPermission.DROP_NAMESPACE);
 
     try {
       user1Con.namespaceOperations().setProperty(n1, Property.TABLE_FILE_MAX.getKey(), "33");
       fail();
-    } catch (AccumuloSecurityException e) {}
+    } catch (AccumuloSecurityException e) {
+      expectPermissionDenied(e);
+    }
 
-    c.securityOperations().grantSystemPermission(user1, SystemPermission.ALTER_NAMESPACE);
+    c.securityOperations().grantSystemPermission(u1, SystemPermission.ALTER_NAMESPACE);
     user1Con.namespaceOperations().setProperty(n1, Property.TABLE_FILE_MAX.getKey(), "33");
     user1Con.namespaceOperations().removeProperty(n1, Property.TABLE_FILE_MAX.getKey());
-    c.securityOperations().revokeSystemPermission(user1, SystemPermission.ALTER_NAMESPACE);
+    c.securityOperations().revokeSystemPermission(u1, SystemPermission.ALTER_NAMESPACE);
   }
 
-  /**
-   * This test makes sure that system-level iterators and constraints are ignored by the system namespace so that the metadata and root tables aren't affected
-   */
   @Test
-  public void excludeSystemIterConst() throws Exception {
-    Connector c = getConnector();
+  public void verifySystemPropertyInheritance() throws Exception {
+    String t1 = "1";
+    String t2 = namespace + "." + t1;
+    c.tableOperations().create(t1);
+    c.namespaceOperations().create(namespace);
+    c.tableOperations().create(t2);
 
-    c.instanceOperations().setProperty("table.iterator.scan.sum", "20," + SimpleFilter.class.getName());
-    assertTrue(c.instanceOperations().getSystemConfiguration().containsValue("20," + SimpleFilter.class.getName()));
+    // verify iterator inheritance
+    _verifySystemPropertyInheritance(t1, t2, Property.TABLE_ITERATOR_PREFIX.getKey() + "scan.sum", "20," + SimpleFilter.class.getName(), false);
 
-    assertTrue(checkNamespaceHasProp(c, Constants.DEFAULT_NAMESPACE, "table.iterator.scan.sum", "20," + SimpleFilter.class.getName()));
-    assertTrue(!checkNamespaceHasProp(c, Constants.ACCUMULO_NAMESPACE, "table.iterator.scan.sum", "20," + SimpleFilter.class.getName()));
-    c.instanceOperations().removeProperty("table.iterator.scan.sum");
+    // verify constraint inheritance
+    _verifySystemPropertyInheritance(t1, t2, Property.TABLE_CONSTRAINT_PREFIX.getKey() + "42", NumericValueConstraint.class.getName(), false);
 
-    c.instanceOperations().setProperty("table.constraint.42", NumericValueConstraint.class.getName());
-    assertTrue(c.instanceOperations().getSystemConfiguration().containsValue(NumericValueConstraint.class.getName()));
+    // verify other inheritance
+    _verifySystemPropertyInheritance(t1, t2, Property.TABLE_LOCALITY_GROUP_PREFIX.getKey() + "dummy", "dummy", true);
+  }
+
+  private void _verifySystemPropertyInheritance(String defaultNamespaceTable, String namespaceTable, String k, String v, boolean systemNamespaceShouldInherit)
+      throws Exception {
+    // nobody should have any of these properties yet
+    assertFalse(c.instanceOperations().getSystemConfiguration().containsValue(v));
+    assertFalse(checkNamespaceHasProp(Namespaces.ACCUMULO_NAMESPACE, k, v));
+    assertFalse(checkTableHasProp(RootTable.NAME, k, v));
+    assertFalse(checkTableHasProp(MetadataTable.NAME, k, v));
+    assertFalse(checkNamespaceHasProp(Namespaces.DEFAULT_NAMESPACE, k, v));
+    assertFalse(checkTableHasProp(defaultNamespaceTable, k, v));
+    assertFalse(checkNamespaceHasProp(namespace, k, v));
+    assertFalse(checkTableHasProp(namespaceTable, k, v));
+
+    // set the filter, verify that accumulo namespace is the only one unaffected
+    c.instanceOperations().setProperty(k, v);
+    // doesn't take effect immediately, needs time to propagate to tserver's ZooKeeper cache
+    UtilWaitThread.sleep(250);
+    assertTrue(c.instanceOperations().getSystemConfiguration().containsValue(v));
+    assertEquals(systemNamespaceShouldInherit, checkNamespaceHasProp(Namespaces.ACCUMULO_NAMESPACE, k, v));
+    assertEquals(systemNamespaceShouldInherit, checkTableHasProp(RootTable.NAME, k, v));
+    assertEquals(systemNamespaceShouldInherit, checkTableHasProp(MetadataTable.NAME, k, v));
+    assertTrue(checkNamespaceHasProp(Namespaces.DEFAULT_NAMESPACE, k, v));
+    assertTrue(checkTableHasProp(defaultNamespaceTable, k, v));
+    assertTrue(checkNamespaceHasProp(namespace, k, v));
+    assertTrue(checkTableHasProp(namespaceTable, k, v));
+
+    // verify it is no longer inherited
+    c.instanceOperations().removeProperty(k);
+    // doesn't take effect immediately, needs time to propagate to tserver's ZooKeeper cache
+    UtilWaitThread.sleep(250);
+    assertFalse(c.instanceOperations().getSystemConfiguration().containsValue(v));
+    assertFalse(checkNamespaceHasProp(Namespaces.ACCUMULO_NAMESPACE, k, v));
+    assertFalse(checkTableHasProp(RootTable.NAME, k, v));
+    assertFalse(checkTableHasProp(MetadataTable.NAME, k, v));
+    assertFalse(checkNamespaceHasProp(Namespaces.DEFAULT_NAMESPACE, k, v));
+    assertFalse(checkTableHasProp(defaultNamespaceTable, k, v));
+    assertFalse(checkNamespaceHasProp(namespace, k, v));
+    assertFalse(checkTableHasProp(namespaceTable, k, v));
+  }
+
+  @Test
+  public void listNamespaces() throws Exception {
+    SortedSet<String> namespaces = c.namespaceOperations().list();
+    Map<String,String> map = c.namespaceOperations().namespaceIdMap();
+    assertEquals(2, namespaces.size());
+    assertEquals(2, map.size());
+    assertTrue(namespaces.contains(Namespaces.ACCUMULO_NAMESPACE));
+    assertTrue(namespaces.contains(Namespaces.DEFAULT_NAMESPACE));
+    assertFalse(namespaces.contains(namespace));
+    assertEquals(Namespaces.ACCUMULO_NAMESPACE_ID, map.get(Namespaces.ACCUMULO_NAMESPACE));
+    assertEquals(Namespaces.DEFAULT_NAMESPACE_ID, map.get(Namespaces.DEFAULT_NAMESPACE));
+    assertNull(map.get(namespace));
+
+    c.namespaceOperations().create(namespace);
+    namespaces = c.namespaceOperations().list();
+    map = c.namespaceOperations().namespaceIdMap();
+    assertEquals(3, namespaces.size());
+    assertEquals(3, map.size());
+    assertTrue(namespaces.contains(Namespaces.ACCUMULO_NAMESPACE));
+    assertTrue(namespaces.contains(Namespaces.DEFAULT_NAMESPACE));
+    assertTrue(namespaces.contains(namespace));
+    assertEquals(Namespaces.ACCUMULO_NAMESPACE_ID, map.get(Namespaces.ACCUMULO_NAMESPACE));
+    assertEquals(Namespaces.DEFAULT_NAMESPACE_ID, map.get(Namespaces.DEFAULT_NAMESPACE));
+    assertNotNull(map.get(namespace));
+
+    c.namespaceOperations().delete(namespace);
+    namespaces = c.namespaceOperations().list();
+    map = c.namespaceOperations().namespaceIdMap();
+    assertEquals(2, namespaces.size());
+    assertEquals(2, map.size());
+    assertTrue(namespaces.contains(Namespaces.ACCUMULO_NAMESPACE));
+    assertTrue(namespaces.contains(Namespaces.DEFAULT_NAMESPACE));
+    assertFalse(namespaces.contains(namespace));
+    assertEquals(Namespaces.ACCUMULO_NAMESPACE_ID, map.get(Namespaces.ACCUMULO_NAMESPACE));
+    assertEquals(Namespaces.DEFAULT_NAMESPACE_ID, map.get(Namespaces.DEFAULT_NAMESPACE));
+    assertNull(map.get(namespace));
+  }
 
-    assertTrue(checkNamespaceHasProp(c, Constants.DEFAULT_NAMESPACE, "table.constraint.42", NumericValueConstraint.class.getName()));
-    assertTrue(!checkNamespaceHasProp(c, Constants.ACCUMULO_NAMESPACE, "table.constraint.42", NumericValueConstraint.class.getName()));
-    c.instanceOperations().removeProperty("table.constraint.42");
+  @Test
+  public void loadClass() throws Exception {
+    assertTrue(c.namespaceOperations().testClassLoad(Namespaces.DEFAULT_NAMESPACE, VersioningIterator.class.getName(), SortedKeyValueIterator.class.getName()));
+    assertFalse(c.namespaceOperations().testClassLoad(Namespaces.DEFAULT_NAMESPACE, "dummy", SortedKeyValueIterator.class.getName()));
+    try {
+      c.namespaceOperations().testClassLoad(namespace, "dummy", "dummy");
+      fail();
+    } catch (NamespaceNotFoundException e) {}
   }
 
-  private boolean checkTableHasProp(Connector c, String t, String propKey, String propVal) throws AccumuloException, TableNotFoundException {
-    for (Entry<String,String> e : c.tableOperations().getProperties(t)) {
-      if (e.getKey().equals(propKey) && e.getValue().equals(propVal)) {
-        return true;
+  @Test
+  public void verifyTableOperationsExceptions() throws Exception {
+    String tableName = namespace + ".1";
+    IteratorSetting setting = new IteratorSetting(200, VersioningIterator.class);
+    Text a = new Text("a");
+    Text z = new Text("z");
+    TableOperations ops = c.tableOperations();
+
+    // this one doesn't throw an exception, so don't fail; just check that it works
+    assertFalse(ops.exists(tableName));
+
+    // table operations that should throw an AccumuloException caused by NamespaceNotFoundException
+    int numRun = 0;
+    ACCUMULOEXCEPTIONS_NAMESPACENOTFOUND: for (int i = 0;; ++i)
+      try {
+        switch (i) {
+          case 0:
+            ops.create(tableName);
+            fail();
+          case 1:
+            ops.create("a");
+            ops.clone("a", tableName, true, Collections.<String,String> emptyMap(), Collections.<String> emptySet());
+            fail();
+          case 2:
+            ops.importTable(tableName, System.getProperty("user.dir") + "/target");
+            fail();
+          default:
+            // break out of infinite loop
+            assertEquals(3, i); // check test integrity
+            assertEquals(3, numRun); // check test integrity
+            break ACCUMULOEXCEPTIONS_NAMESPACENOTFOUND;
+        }
+      } catch (Exception e) {
+        numRun++;
+        if (!(e instanceof AccumuloException) || !(e.getCause() instanceof NamespaceNotFoundException))
+          throw new Exception("Case " + i + " resulted in " + e.getClass().getName(), e);
+      }
+
+    // table operations that should throw an AccumuloException caused by a TableNotFoundException caused by a NamespaceNotFoundException
+    // these are here because we didn't declare TableNotFoundException in the API :(
+    numRun = 0;
+    ACCUMULOEXCEPTIONS_TABLENOTFOUND: for (int i = 0;; ++i)
+      try {
+        switch (i) {
+          case 0:
+            ops.removeConstraint(tableName, 0);
+            fail();
+          case 1:
+            ops.removeProperty(tableName, "a");
+            fail();
+          case 2:
+            ops.setProperty(tableName, "a", "b");
+            fail();
+          default:
+            // break out of infinite loop
+            assertEquals(3, i); // check test integrity
+            assertEquals(3, numRun); // check test integrity
+            break ACCUMULOEXCEPTIONS_TABLENOTFOUND;
+        }
+      } catch (Exception e) {
+        numRun++;
+        if (!(e instanceof AccumuloException) || !(e.getCause() instanceof TableNotFoundException)
+            || !(e.getCause().getCause() instanceof NamespaceNotFoundException))
+          throw new Exception("Case " + i + " resulted in " + e.getClass().getName(), e);
+      }
+
+    // table operations that should throw a TableNotFoundException caused by NamespaceNotFoundException
+    numRun = 0;
+    TABLENOTFOUNDEXCEPTIONS: for (int i = 0;; ++i)
+      try {
+        switch (i) {
+          case 0:
+            ops.addConstraint(tableName, NumericValueConstraint.class.getName());
+            fail();
+          case 1:
+            ops.addSplits(tableName, new TreeSet<Text>());
+            fail();
+          case 2:
+            ops.attachIterator(tableName, setting);
+            fail();
+          case 3:
+            ops.cancelCompaction(tableName);
+            fail();
+          case 4:
+            ops.checkIteratorConflicts(tableName, setting, EnumSet.allOf(IteratorScope.class));
+            fail();
+          case 5:
+            ops.clearLocatorCache(tableName);
+            fail();
+          case 6:
+            ops.clone(tableName, "2", true, Collections.<String,String> emptyMap(), Collections.<String> emptySet());
+            fail();
+          case 7:
+            ops.compact(tableName, a, z, true, true);
+            fail();
+          case 8:
+            ops.delete(tableName);
+            fail();
+          case 9:
+            ops.deleteRows(tableName, a, z);
+            fail();
+          case 10:
+            ops.splitRangeByTablets(tableName, new Range(), 10);
+            fail();
+          case 11:
+            ops.exportTable(tableName, namespace + "_dir");
+            fail();
+          case 12:
+            ops.flush(tableName, a, z, true);
+            fail();
+          case 13:
+            ops.getDiskUsage(Collections.singleton(tableName));
+            fail();
+          case 14:
+            ops.getIteratorSetting(tableName, "a", IteratorScope.scan);
+            fail();
+          case 15:
+            ops.getLocalityGroups(tableName);
+            fail();
+          case 16:
+            ops.getMaxRow(tableName, Authorizations.EMPTY, a, true, z, true);
+            fail();
+          case 17:
+            ops.getProperties(tableName);
+            fail();
+          case 18:
+            ops.importDirectory(tableName, "", "", false);
+            fail();
+          case 19:
+            ops.testClassLoad(tableName, VersioningIterator.class.getName(), SortedKeyValueIterator.class.getName());
+            fail();
+          case 20:
+            ops.listConstraints(tableName);
+            fail();
+          case 21:
+            ops.listIterators(tableName);
+            fail();
+          case 22:
+            ops.listSplits(tableName);
+            fail();
+          case 23:
+            ops.merge(tableName, a, z);
+            fail();
+          case 24:
+            ops.offline(tableName, true);
+            fail();
+          case 25:
+            ops.online(tableName, true);
+            fail();
+          case 26:
+            ops.removeIterator(tableName, "a", EnumSet.of(IteratorScope.scan));
+            fail();
+          case 27:
+            ops.rename(tableName, tableName + "2");
+            fail();
+          case 28:
+            ops.setLocalityGroups(tableName, Collections.<String,Set<Text>> emptyMap());
+            fail();
+          default:
+            // break out of infinite loop
+            assertEquals(29, i); // check test integrity
+            assertEquals(29, numRun); // check test integrity
+            break TABLENOTFOUNDEXCEPTIONS;
+        }
+      } catch (Exception e) {
+        numRun++;
+        if (!(e instanceof TableNotFoundException) || !(e.getCause() instanceof NamespaceNotFoundException))
+          throw new Exception("Case " + i + " resulted in " + e.getClass().getName(), e);
       }
-    }
-    return false;
   }
 
-  private boolean checkNamespaceHasProp(Connector c, String n, String propKey, String propVal) throws AccumuloException, NamespaceNotFoundException,
-      AccumuloSecurityException {
-    for (Entry<String,String> e : c.namespaceOperations().getProperties(n)) {
-      if (e.getKey().equals(propKey) && e.getValue().equals(propVal)) {
-        return true;
+  @Test
+  public void verifyNamespaceOperationsExceptions() throws Exception {
+    IteratorSetting setting = new IteratorSetting(200, VersioningIterator.class);
+    NamespaceOperations ops = c.namespaceOperations();
+
+    // this one doesn't throw an exception, so don't fail; just check that it works
+    assertFalse(ops.exists(namespace));
+
+    // namespace operations that should throw a NamespaceNotFoundException
+    int numRun = 0;
+    NAMESPACENOTFOUND: for (int i = 0;; ++i)
+      try {
+        switch (i) {
+          case 0:
+            ops.addConstraint(namespace, NumericValueConstraint.class.getName());
+            fail();
+          case 1:
+            ops.attachIterator(namespace, setting);
+            fail();
+          case 2:
+            ops.checkIteratorConflicts(namespace, setting, EnumSet.of(IteratorScope.scan));
+            fail();
+          case 3:
+            ops.delete(namespace);
+            fail();
+          case 4:
+            ops.getIteratorSetting(namespace, "thing", IteratorScope.scan);
+            fail();
+          case 5:
+            ops.getProperties(namespace);
+            fail();
+          case 6:
+            ops.listConstraints(namespace);
+            fail();
+          case 7:
+            ops.listIterators(namespace);
+            fail();
+          case 8:
+            ops.removeConstraint(namespace, 1);
+            fail();
+          case 9:
+            ops.removeIterator(namespace, "thing", EnumSet.allOf(IteratorScope.class));
+            fail();
+          case 10:
+            ops.removeProperty(namespace, "a");
+            fail();
+          case 11:
+            ops.rename(namespace, namespace + "2");
+            fail();
+          case 12:
+            ops.setProperty(namespace, "k", "v");
+            fail();
+          case 13:
+            ops.testClassLoad(namespace, VersioningIterator.class.getName(), SortedKeyValueIterator.class.getName());
+            fail();
+          default:
+            // break out of infinite loop
+            assertEquals(14, i); // check test integrity
+            assertEquals(14, numRun); // check test integrity
+            break NAMESPACENOTFOUND;
+        }
+      } catch (Exception e) {
+        numRun++;
+        if (!(e instanceof NamespaceNotFoundException))
+          throw new Exception("Case " + i + " resulted in " + e.getClass().getName(), e);
+      }
+
+    // namespace operations that should throw a NamespaceExistsException
+    numRun = 0;
+    NAMESPACEEXISTS: for (int i = 0;; ++i)
+      try {
+        switch (i) {
+          case 0:
+            ops.create(namespace + "0");
+            ops.create(namespace + "0"); // should fail here
+            fail();
+          case 1:
+            ops.create(namespace + "1");
+            ops.create(namespace + "2");
+            ops.rename(namespace + "1", namespace + "2"); // should fail here
+            fail();
+          default:
+            // break out of infinite loop
+            assertEquals(2, i); // check test integrity
+            assertEquals(2, numRun); // check test integrity
+            break NAMESPACEEXISTS;
+        }
+      } catch (Exception e) {
+        numRun++;
+        if (!(e instanceof NamespaceExistsException))
+          throw new Exception("Case " + i + " resulted in " + e.getClass().getName(), e);
       }
+  }
+
+  private boolean checkTableHasProp(String t, String propKey, String propVal) {
+    return checkHasProperty(t, propKey, propVal, true);
+  }
+
+  private boolean checkNamespaceHasProp(String n, String propKey, String propVal) {
+    return checkHasProperty(n, propKey, propVal, false);
+  }
+
+  private boolean checkHasProperty(String name, String propKey, String propVal, boolean nameIsTable) {
+    try {
+      Iterable<Entry<String,String>> iterable = nameIsTable ? c.tableOperations().getProperties(name) : c.namespaceOperations().getProperties(name);
+      for (Entry<String,String> e : iterable)
+        if (propKey.equals(e.getKey()))
+          return propVal.equals(e.getValue());
+      return false;
+    } catch (Exception e) {
+      fail();
+      return false;
     }
-    return false;
   }
 
   public static class SimpleFilter extends Filter {
@@ -569,4 +1104,15 @@ public class NamespacesIT extends SimpleMacIT {
       return true;
     }
   }
+
+  private void expectPermissionDenied(AccumuloSecurityException sec) {
+    assertEquals(sec.getSecurityErrorCode().getClass(), SecurityErrorCode.class);
+    switch (sec.getSecurityErrorCode()) {
+      case PERMISSION_DENIED:
+        break;
+      default:
+        fail();
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
index 21b56b7..7a126ee 100644
--- a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
@@ -34,6 +34,7 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
@@ -835,7 +836,8 @@ public class ShellServerIT extends SimpleMacIT {
 
   @Test(timeout = 30 * 1000)
   public void namespaces() throws Exception {
-    exec("namespaces", true, Constants.DEFAULT_NAMESPACE, true);
+    exec("namespaces", true, "(default)", true); // default namespace for display purposes only
+    exec("namespaces", true, Namespaces.ACCUMULO_NAMESPACE, true);
     exec("createnamespace thing1", true);
     String namespaces = exec("namespaces");
     assertTrue(namespaces.contains("thing1"));
@@ -899,7 +901,7 @@ public class ShellServerIT extends SimpleMacIT {
     // should fail
     exec("insert r cf cq abc", false);
     exec("constraint -l", true, "NumericValueConstraint", true);
-    exec("constraint -ns thing4 -d 1");
+    exec("constraint -ns thing4 -d 2");
     exec("sleep 1");
     exec("insert r cf cq abc", true);
   }


[7/7] git commit: ACCUMULO-1965 Fix exception handling for namespaces

Posted by ct...@apache.org.
ACCUMULO-1965 Fix exception handling for namespaces

  Make the exception handling propagate correctly to the client.
  Vastly expanded on existing namespace tests to include checks for
  throwing the correct exceptions. Consolidated a lot of the fate
  operations code and refactored the master (slightly; moved inner
  classes to separate files) to be easier to modify the relevant RPC
  handling code.

  Fixed many API bugs related to throwing correct exceptions, found from
  the new tests added to NamespacesIT.


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/f35e3f47
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/f35e3f47
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/f35e3f47

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: f35e3f47265868c560b3d49edebf1a8c24196512
Parents: c41900e
Author: Christopher Tubbs <ct...@apache.org>
Authored: Tue Dec 24 00:06:12 2013 -0500
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Fri Dec 27 19:55:34 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/core/Constants.java     |    4 -
 .../core/client/TableNotFoundException.java     |   21 +-
 .../core/client/admin/NamespaceOperations.java  |    2 +-
 .../client/admin/NamespaceOperationsImpl.java   |  193 +-
 .../client/admin/TableOperationsHelper.java     |   35 +-
 .../core/client/admin/TableOperationsImpl.java  |  256 +-
 .../core/client/impl/ConnectorImpl.java         |   55 +-
 .../accumulo/core/client/impl/MasterClient.java |   76 +-
 .../accumulo/core/client/impl/Namespaces.java   |   33 +-
 .../accumulo/core/client/impl/Tables.java       |   32 +-
 .../thrift/TableOperationExceptionType.java     |   11 +-
 .../accumulo/core/client/mock/MockAccumulo.java |    6 +-
 .../core/client/mock/MockTableOperations.java   |   12 +-
 .../core/master/thrift/FateOperation.java       |  103 +
 .../core/master/thrift/FateService.java         | 5017 +++++++++
 .../core/master/thrift/MasterClientService.java | 9946 +-----------------
 .../core/master/thrift/TableOperation.java      |   94 -
 .../accumulo/core/metadata/MetadataTable.java   |    4 +-
 .../accumulo/core/metadata/RootTable.java       |    4 +-
 .../util/shell/commands/NamespacesCommand.java  |    4 +-
 core/src/main/thrift/client.thrift              |  203 +-
 core/src/main/thrift/master.thrift              |  222 +-
 .../core/client/mock/MockNamespacesTest.java    |    5 +-
 .../client/mock/MockTableOperationsTest.java    |   49 +-
 .../server/client/ClientServiceHandler.java     |   37 +-
 .../server/conf/NamespaceConfiguration.java     |    5 +-
 .../apache/accumulo/server/init/Initialize.java |    9 +-
 .../accumulo/server/master/LiveTServerSet.java  |  114 +-
 .../security/AuditedSecurityOperation.java      |    8 +-
 .../server/security/SecurityOperation.java      |    6 +-
 .../server/security/handler/ZKPermHandler.java  |    6 +-
 .../accumulo/master/FateServiceHandler.java     |  412 +
 .../java/org/apache/accumulo/master/Master.java |  852 +-
 .../master/MasterClientServiceHandler.java      |  463 +
 .../accumulo/master/tableOps/CloneTable.java    |   10 +-
 .../master/tableOps/RenameNamespace.java        |    3 +-
 .../accumulo/master/tableOps/RenameTable.java   |    3 +-
 .../apache/accumulo/master/tableOps/Utils.java  |    4 +-
 .../test/randomwalk/concurrent/Config.java      |    4 +-
 .../org/apache/accumulo/test/NamespacesIT.java  | 1202 ++-
 .../org/apache/accumulo/test/ShellServerIT.java |    6 +-
 41 files changed, 7683 insertions(+), 11848 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/Constants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/Constants.java b/core/src/main/java/org/apache/accumulo/core/Constants.java
index 75452e3..c151abb 100644
--- a/core/src/main/java/org/apache/accumulo/core/Constants.java
+++ b/core/src/main/java/org/apache/accumulo/core/Constants.java
@@ -113,8 +113,4 @@ public class Constants {
 
   // Variables that will be substituted with environment vars in PropertyType.PATH values
   public static final String[] PATH_PROPERTY_ENV_VARS = new String[] {"ACCUMULO_HOME", "ACCUMULO_CONF_DIR"};
-  public static final String DEFAULT_NAMESPACE = "";
-  public static final String ACCUMULO_NAMESPACE = "accumulo";
-  public static final String DEFAULT_NAMESPACE_ID = "+default";
-  public static final String ACCUMULO_NAMESPACE_ID = "+accumulo";
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/TableNotFoundException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/TableNotFoundException.java b/core/src/main/java/org/apache/accumulo/core/client/TableNotFoundException.java
index 88ab449..6d27336 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/TableNotFoundException.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/TableNotFoundException.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.core.client;
 
+import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
 
 /**
@@ -27,9 +28,9 @@ public class TableNotFoundException extends Exception {
    * 
    */
   private static final long serialVersionUID = 1L;
-  
+
   private String tableName;
-  
+
   /**
    * @param tableId
    *          the internal id of the table that was sought
@@ -43,7 +44,7 @@ public class TableNotFoundException extends Exception {
         + " does not exist" + (description != null && !description.isEmpty() ? " (" + description + ")" : ""));
     this.tableName = tableName;
   }
-  
+
   /**
    * @param tableId
    *          the internal id of the table that was sought
@@ -58,7 +59,7 @@ public class TableNotFoundException extends Exception {
     this(tableId, tableName, description);
     super.initCause(cause);
   }
-  
+
   /**
    * @param e
    *          constructs an exception from a thrift exception
@@ -66,7 +67,17 @@ public class TableNotFoundException extends Exception {
   public TableNotFoundException(ThriftTableOperationException e) {
     this(e.getTableId(), e.getTableName(), e.getDescription(), e);
   }
-  
+
+  /**
+   * @param tableName
+   *          the original specified table
+   * @param e
+   *          indicates that a table wasn't found because the namespace specified in the table name wasn't found
+   */
+  public TableNotFoundException(String tableName, NamespaceNotFoundException e) {
+    this(null, tableName, "Namespace " + Tables.qualify(tableName).getFirst() + " does not exist.", e);
+  }
+
   /**
    * @return the name of the table sought
    */

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java
index 535287d..b81310e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java
@@ -277,7 +277,7 @@ public interface NamespaceOperations {
 
   /**
    * Check whether a given iterator configuration conflicts with existing configuration; in particular, determine if the name or priority are already in use for
-   * the specified scopes. If not, an IllegalArgumentException is thrown, wrapped in an AccumuloException.
+   * the specified scopes. If so, an IllegalArgumentException is thrown, wrapped in an AccumuloException.
    * 
    * @param namespace
    *          the name of the namespace

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
index e5e18dc..569a3b6 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
@@ -27,7 +27,6 @@ import java.util.Map.Entry;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
-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;
@@ -35,13 +34,13 @@ import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.NamespaceExistsException;
 import org.apache.accumulo.core.client.NamespaceNotEmptyException;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
-import org.apache.accumulo.core.client.TableOfflineException;
+import org.apache.accumulo.core.client.TableExistsException;
+import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.ClientExec;
 import org.apache.accumulo.core.client.impl.ClientExecReturn;
 import org.apache.accumulo.core.client.impl.MasterClient;
 import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.ServerClient;
-import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.ClientService;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
@@ -49,29 +48,27 @@ import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException
 import org.apache.accumulo.core.constraints.Constraint;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.master.thrift.FateOperation;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
-import org.apache.accumulo.core.master.thrift.TableOperation;
 import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.util.ArgumentChecker;
-import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.OpTimer;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.trace.instrument.Tracer;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
-import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransportException;
 
 public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
   private Instance instance;
   private Credentials credentials;
+  private TableOperationsImpl tableOps;
 
   private static final Logger log = Logger.getLogger(TableOperations.class);
 
-  public NamespaceOperationsImpl(Instance instance, Credentials credentials) {
+  public NamespaceOperationsImpl(Instance instance, Credentials credentials, TableOperationsImpl tableOps) {
     ArgumentChecker.notNull(instance, credentials);
     this.instance = instance;
     this.credentials = credentials;
+    this.tableOps = tableOps;
   }
 
   @Override
@@ -97,120 +94,10 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
     ArgumentChecker.notNull(namespace);
 
     try {
-      doNamespaceOperation(TableOperation.CREATE, Arrays.asList(ByteBuffer.wrap(namespace.getBytes())), Collections.<String,String> emptyMap());
+      doNamespaceFateOperation(FateOperation.NAMESPACE_CREATE, Arrays.asList(ByteBuffer.wrap(namespace.getBytes())), Collections.<String,String> emptyMap());
     } catch (NamespaceNotFoundException e) {
-      throw new AssertionError("Shouldn't happen: " + e.getMessage());
-    }
-  }
-
-  private long beginNamespaceOperation() throws ThriftSecurityException, TException {
-    while (true) {
-      MasterClientService.Iface client = null;
-      try {
-        client = MasterClient.getConnectionWithRetry(instance);
-        return client.beginNamespaceOperation(Tracer.traceInfo(), credentials.toThrift(instance));
-      } catch (TTransportException tte) {
-        log.debug("Failed to call beginNamespaceOperation(), retrying ... ", tte);
-        UtilWaitThread.sleep(100);
-      } finally {
-        MasterClient.close(client);
-      }
-    }
-  }
-
-  private void executeNamespaceOperation(long opid, TableOperation op, List<ByteBuffer> args, Map<String,String> opts, boolean autoCleanUp)
-      throws ThriftSecurityException, TException, ThriftTableOperationException {
-    while (true) {
-      MasterClientService.Iface client = null;
-      try {
-        client = MasterClient.getConnectionWithRetry(instance);
-        client.executeNamespaceOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid, op, args, opts, autoCleanUp);
-        break;
-      } catch (TTransportException tte) {
-        log.debug("Failed to call executeTableOperation(), retrying ... ", tte);
-        UtilWaitThread.sleep(100);
-      } finally {
-        MasterClient.close(client);
-      }
-    }
-  }
-
-  private String waitForNamespaceOperation(long opid) throws ThriftSecurityException, TException, ThriftTableOperationException {
-    while (true) {
-      MasterClientService.Iface client = null;
-      try {
-        client = MasterClient.getConnectionWithRetry(instance);
-        return client.waitForNamespaceOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid);
-      } catch (TTransportException tte) {
-        log.debug("Failed to call waitForTableOperation(), retrying ... ", tte);
-        UtilWaitThread.sleep(100);
-      } finally {
-        MasterClient.close(client);
-      }
-    }
-  }
-
-  private void finishNamespaceOperation(long opid) throws ThriftSecurityException, TException {
-    while (true) {
-      MasterClientService.Iface client = null;
-      try {
-        client = MasterClient.getConnectionWithRetry(instance);
-        client.finishNamespaceOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid);
-        break;
-      } catch (TTransportException tte) {
-        log.debug("Failed to call finishTableOperation(), retrying ... ", tte);
-        UtilWaitThread.sleep(100);
-      } finally {
-        MasterClient.close(client);
-      }
-    }
-  }
-
-  private String doNamespaceOperation(TableOperation op, List<ByteBuffer> args, Map<String,String> opts) throws AccumuloSecurityException,
-      NamespaceExistsException, NamespaceNotFoundException, AccumuloException {
-    return doNamespaceOperation(op, args, opts, true);
-  }
-
-  private String doNamespaceOperation(TableOperation op, List<ByteBuffer> args, Map<String,String> opts, boolean wait) throws AccumuloSecurityException,
-      NamespaceExistsException, NamespaceNotFoundException, AccumuloException {
-    Long opid = null;
-
-    try {
-      opid = beginNamespaceOperation();
-      executeNamespaceOperation(opid, op, args, opts, !wait);
-      if (!wait) {
-        opid = null;
-        return null;
-      }
-      String ret = waitForNamespaceOperation(opid);
-      Tables.clearCache(instance);
-      return ret;
-    } catch (ThriftSecurityException e) {
-      String tableName = ByteBufferUtil.toString(args.get(0));
-      String tableInfo = Tables.getPrintableTableInfoFromName(instance, tableName);
-      throw new AccumuloSecurityException(e.user, e.code, tableInfo, e);
-    } catch (ThriftTableOperationException e) {
-      switch (e.getType()) {
-        case EXISTS:
-          throw new NamespaceExistsException(e);
-        case NOTFOUND:
-          throw new NamespaceNotFoundException(e);
-        case OFFLINE:
-          throw new TableOfflineException(instance, null);
-        case OTHER:
-        default:
-          throw new AccumuloException(e.description, e);
-      }
-    } catch (Exception e) {
-      throw new AccumuloException(e.getMessage(), e);
-    } finally {
-      // always finish table op, even when exception
-      if (opid != null)
-        try {
-          finishNamespaceOperation(opid);
-        } catch (Exception e) {
-          log.warn(e.getMessage(), e);
-        }
+      // should not happen
+      throw new AssertionError(e);
     }
   }
 
@@ -219,7 +106,7 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
     ArgumentChecker.notNull(namespace);
     String namespaceId = Namespaces.getNamespaceId(instance, namespace);
 
-    if (namespaceId.equals(Constants.ACCUMULO_NAMESPACE_ID) || namespaceId.equals(Constants.DEFAULT_NAMESPACE_ID)) {
+    if (namespaceId.equals(Namespaces.ACCUMULO_NAMESPACE_ID) || namespaceId.equals(Namespaces.DEFAULT_NAMESPACE_ID)) {
       log.debug(credentials.getPrincipal() + " attempted to delete the " + namespaceId + " namespace");
       throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.UNSUPPORTED_OPERATION);
     }
@@ -232,9 +119,10 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
     Map<String,String> opts = new HashMap<String,String>();
 
     try {
-      doNamespaceOperation(TableOperation.DELETE, args, opts);
+      doNamespaceFateOperation(FateOperation.NAMESPACE_DELETE, args, opts);
     } catch (NamespaceExistsException e) {
-      throw new AssertionError("Shouldn't happen: " + e.getMessage());
+      // should not happen
+      throw new AssertionError(e);
     }
 
   }
@@ -245,14 +133,15 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
 
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(oldNamespaceName.getBytes()), ByteBuffer.wrap(newNamespaceName.getBytes()));
     Map<String,String> opts = new HashMap<String,String>();
-    doNamespaceOperation(TableOperation.RENAME, args, opts);
+    doNamespaceFateOperation(FateOperation.NAMESPACE_RENAME, args, opts);
   }
 
   @Override
-  public void setProperty(final String namespace, final String property, final String value) throws AccumuloException, AccumuloSecurityException {
+  public void setProperty(final String namespace, final String property, final String value) throws AccumuloException, AccumuloSecurityException,
+      NamespaceNotFoundException {
     ArgumentChecker.notNull(namespace, property, value);
 
-    MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
+    MasterClient.executeNamespace(instance, new ClientExec<MasterClientService.Client>() {
       @Override
       public void execute(MasterClientService.Client client) throws Exception {
         client.setNamespaceProperty(Tracer.traceInfo(), credentials.toThrift(instance), namespace, property, value);
@@ -261,10 +150,10 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
   }
 
   @Override
-  public void removeProperty(final String namespace, final String property) throws AccumuloException, AccumuloSecurityException {
+  public void removeProperty(final String namespace, final String property) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
     ArgumentChecker.notNull(namespace, property);
 
-    MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
+    MasterClient.executeNamespace(instance, new ClientExec<MasterClientService.Client>() {
       @Override
       public void execute(MasterClientService.Client client) throws Exception {
         client.removeNamespaceProperty(Tracer.traceInfo(), credentials.toThrift(instance), namespace, property);
@@ -284,7 +173,7 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
       }).entrySet();
     } catch (ThriftTableOperationException e) {
       switch (e.getType()) {
-        case NOTFOUND:
+        case NAMESPACE_NOTFOUND:
           throw new NamespaceNotFoundException(e);
         case OTHER:
         default:
@@ -304,19 +193,6 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
   }
 
   @Override
-  public void attachIterator(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
-      NamespaceNotFoundException {
-    testClassLoad(namespace, setting.getIteratorClass(), SortedKeyValueIterator.class.getName());
-    super.attachIterator(namespace, setting, scopes);
-  }
-
-  @Override
-  public int addConstraint(String namespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
-    testClassLoad(namespace, constraintClassName, Constraint.class.getName());
-    return super.addConstraint(namespace, constraintClassName);
-  }
-
-  @Override
   public boolean testClassLoad(final String namespace, final String className, final String asTypeName) throws NamespaceNotFoundException, AccumuloException,
       AccumuloSecurityException {
     ArgumentChecker.notNull(namespace, className, asTypeName);
@@ -330,9 +206,8 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
       });
     } catch (ThriftTableOperationException e) {
       switch (e.getType()) {
-        case NOTFOUND:
+        case NAMESPACE_NOTFOUND:
           throw new NamespaceNotFoundException(e);
-        case OTHER:
         default:
           throw new AccumuloException(e.description, e);
       }
@@ -344,4 +219,30 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
       throw new AccumuloException(e);
     }
   }
+
+  @Override
+  public void attachIterator(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+      NamespaceNotFoundException {
+    testClassLoad(namespace, setting.getIteratorClass(), SortedKeyValueIterator.class.getName());
+    super.attachIterator(namespace, setting, scopes);
+  }
+
+  @Override
+  public int addConstraint(String namespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
+    testClassLoad(namespace, constraintClassName, Constraint.class.getName());
+    return super.addConstraint(namespace, constraintClassName);
+  }
+
+  private String doNamespaceFateOperation(FateOperation op, List<ByteBuffer> args, Map<String,String> opts) throws AccumuloSecurityException,
+      AccumuloException, NamespaceExistsException, NamespaceNotFoundException {
+    try {
+      return tableOps.doFateOperation(op, args, opts);
+    } catch (TableExistsException e) {
+      // should not happen
+      throw new AssertionError(e);
+    } catch (TableNotFoundException e) {
+      // should not happen
+      throw new AssertionError(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java
index 9e86466..843f572 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java
@@ -29,17 +29,19 @@ import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.util.ArgumentChecker;
 
 public abstract class TableOperationsHelper implements TableOperations {
-  
+
   @Override
   public void attachIterator(String tableName, IteratorSetting setting) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
     attachIterator(tableName, setting, EnumSet.allOf(IteratorScope.class));
   }
-  
+
   @Override
   public void attachIterator(String tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
       TableNotFoundException {
+    ArgumentChecker.notNull(tableName, setting, scopes);
     checkIteratorConflicts(tableName, setting, scopes);
     for (IteratorScope scope : scopes) {
       String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), setting.getName());
@@ -49,12 +51,10 @@ public abstract class TableOperationsHelper implements TableOperations {
       this.setProperty(tableName, root, setting.getPriority() + "," + setting.getIteratorClass());
     }
   }
-  
+
   @Override
   public void removeIterator(String tableName, String name, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
       TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(null, tableName, null);
     Map<String,String> copy = new TreeMap<String,String>();
     for (Entry<String,String> property : this.getProperties(tableName)) {
       copy.put(property.getKey(), property.getValue());
@@ -67,16 +67,15 @@ public abstract class TableOperationsHelper implements TableOperations {
       }
     }
   }
-  
+
   @Override
   public IteratorSetting getIteratorSetting(String tableName, String name, IteratorScope scope) throws AccumuloSecurityException, AccumuloException,
       TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(null, tableName, null);
+    ArgumentChecker.notNull(tableName, name, scope);
     int priority = -1;
     String classname = null;
     Map<String,String> settings = new HashMap<String,String>();
-    
+
     String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), name);
     String opt = root + ".opt.";
     for (Entry<String,String> property : this.getProperties(tableName)) {
@@ -96,11 +95,9 @@ public abstract class TableOperationsHelper implements TableOperations {
     }
     return new IteratorSetting(priority, name, classname, settings);
   }
-  
+
   @Override
   public Map<String,EnumSet<IteratorScope>> listIterators(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(null, tableName, null);
     Map<String,EnumSet<IteratorScope>> result = new TreeMap<String,EnumSet<IteratorScope>>();
     for (Entry<String,String> property : this.getProperties(tableName)) {
       String name = property.getKey();
@@ -116,11 +113,10 @@ public abstract class TableOperationsHelper implements TableOperations {
     }
     return result;
   }
-  
+
   @Override
   public void checkIteratorConflicts(String tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException, TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(null, tableName, null);
+    ArgumentChecker.notNull(tableName, setting, scopes);
     for (IteratorScope scope : scopes) {
       String scopeStr = String.format("%s%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase());
       String nameStr = String.format("%s.%s", scopeStr, setting.getName());
@@ -129,7 +125,8 @@ public abstract class TableOperationsHelper implements TableOperations {
       for (Entry<String,String> property : this.getProperties(tableName)) {
         if (property.getKey().startsWith(scopeStr)) {
           if (property.getKey().equals(nameStr))
-            throw new AccumuloException(new IllegalArgumentException("iterator name conflict for " + setting.getName() + ": " + property.getKey() + "=" + property.getValue()));
+            throw new AccumuloException(new IllegalArgumentException("iterator name conflict for " + setting.getName() + ": " + property.getKey() + "="
+                + property.getValue()));
           if (property.getKey().startsWith(optStr))
             optionConflicts.put(property.getKey(), property.getValue());
           if (property.getKey().contains(".opt."))
@@ -149,7 +146,7 @@ public abstract class TableOperationsHelper implements TableOperations {
         throw new AccumuloException(new IllegalArgumentException("iterator options conflict for " + setting.getName() + ": " + optionConflicts));
     }
   }
-  
+
   @Override
   public int addConstraint(String tableName, String constraintClassName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     TreeSet<Integer> constraintNumbers = new TreeSet<Integer>();
@@ -175,12 +172,12 @@ public abstract class TableOperationsHelper implements TableOperations {
     this.setProperty(tableName, Property.TABLE_CONSTRAINT_PREFIX.toString() + i, constraintClassName);
     return i;
   }
-  
+
   @Override
   public void removeConstraint(String tableName, int number) throws AccumuloException, AccumuloSecurityException {
     this.removeProperty(tableName, Property.TABLE_CONSTRAINT_PREFIX.toString() + number);
   }
-  
+
   @Override
   public Map<String,Integer> listConstraints(String tableName) throws AccumuloException, TableNotFoundException {
     Map<String,Integer> constraints = new TreeMap<String,Integer>();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
index 76cda7d..9956538 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
@@ -50,6 +50,7 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IsolatedScanner;
 import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.NamespaceExistsException;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.RowIterator;
 import org.apache.accumulo.core.client.Scanner;
@@ -61,7 +62,6 @@ import org.apache.accumulo.core.client.impl.AccumuloServerException;
 import org.apache.accumulo.core.client.impl.ClientExec;
 import org.apache.accumulo.core.client.impl.ClientExecReturn;
 import org.apache.accumulo.core.client.impl.MasterClient;
-import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.ServerClient;
 import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
 import org.apache.accumulo.core.client.impl.Tables;
@@ -72,7 +72,6 @@ import org.apache.accumulo.core.client.impl.thrift.ClientService.Client;
 import org.apache.accumulo.core.client.impl.thrift.TDiskUsage;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
-import org.apache.accumulo.core.client.security.SecurityErrorCode;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.Property;
@@ -87,8 +86,8 @@ import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.master.state.tables.TableState;
+import org.apache.accumulo.core.master.thrift.FateOperation;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
-import org.apache.accumulo.core.master.thrift.TableOperation;
 import org.apache.accumulo.core.metadata.MetadataServicer;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
@@ -120,10 +119,6 @@ import org.apache.thrift.TApplicationException;
 import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransportException;
 
-/**
- * Provides a class for administering tables
- * 
- */
 public class TableOperationsImpl extends TableOperationsHelper {
   private Instance instance;
   private Credentials credentials;
@@ -225,21 +220,21 @@ public class TableOperationsImpl extends TableOperationsHelper {
       opts = Collections.emptyMap();
 
     try {
-      doTableOperation(TableOperation.CREATE, args, opts);
-    } catch (TableNotFoundException e1) {
+      doTableFateOperation(tableName, AccumuloException.class, FateOperation.TABLE_CREATE, args, opts);
+    } catch (TableNotFoundException e) {
       // should not happen
-      throw new RuntimeException(e1);
+      throw new AssertionError(e);
     }
   }
 
-  private long beginTableOperation() throws ThriftSecurityException, TException {
+  private long beginFateOperation() throws ThriftSecurityException, TException {
     while (true) {
       MasterClientService.Iface client = null;
       try {
         client = MasterClient.getConnectionWithRetry(instance);
-        return client.beginTableOperation(Tracer.traceInfo(), credentials.toThrift(instance));
+        return client.beginFateOperation(Tracer.traceInfo(), credentials.toThrift(instance));
       } catch (TTransportException tte) {
-        log.debug("Failed to call beginTableOperation(), retrying ... ", tte);
+        log.debug("Failed to call beginFateOperation(), retrying ... ", tte);
         UtilWaitThread.sleep(100);
       } finally {
         MasterClient.close(client);
@@ -247,16 +242,17 @@ public class TableOperationsImpl extends TableOperationsHelper {
     }
   }
 
-  private void executeTableOperation(long opid, TableOperation op, List<ByteBuffer> args, Map<String,String> opts, boolean autoCleanUp)
+  // This method is for retrying in the case of network failures; anything else it passes to the caller to deal with
+  private void executeFateOperation(long opid, FateOperation op, List<ByteBuffer> args, Map<String,String> opts, boolean autoCleanUp)
       throws ThriftSecurityException, TException, ThriftTableOperationException {
     while (true) {
       MasterClientService.Iface client = null;
       try {
         client = MasterClient.getConnectionWithRetry(instance);
-        client.executeTableOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid, op, args, opts, autoCleanUp);
+        client.executeFateOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid, op, args, opts, autoCleanUp);
         break;
       } catch (TTransportException tte) {
-        log.debug("Failed to call executeTableOperation(), retrying ... ", tte);
+        log.debug("Failed to call executeFateOperation(), retrying ... ", tte);
         UtilWaitThread.sleep(100);
       } finally {
         MasterClient.close(client);
@@ -264,14 +260,14 @@ public class TableOperationsImpl extends TableOperationsHelper {
     }
   }
 
-  private String waitForTableOperation(long opid) throws ThriftSecurityException, TException, ThriftTableOperationException {
+  private String waitForFateOperation(long opid) throws ThriftSecurityException, TException, ThriftTableOperationException {
     while (true) {
       MasterClientService.Iface client = null;
       try {
         client = MasterClient.getConnectionWithRetry(instance);
-        return client.waitForTableOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid);
+        return client.waitForFateOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid);
       } catch (TTransportException tte) {
-        log.debug("Failed to call waitForTableOperation(), retrying ... ", tte);
+        log.debug("Failed to call waitForFateOperation(), retrying ... ", tte);
         UtilWaitThread.sleep(100);
       } finally {
         MasterClient.close(client);
@@ -279,15 +275,15 @@ public class TableOperationsImpl extends TableOperationsHelper {
     }
   }
 
-  private void finishTableOperation(long opid) throws ThriftSecurityException, TException {
+  private void finishFateOperation(long opid) throws ThriftSecurityException, TException {
     while (true) {
       MasterClientService.Iface client = null;
       try {
         client = MasterClient.getConnectionWithRetry(instance);
-        client.finishTableOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid);
+        client.finishFateOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid);
         break;
       } catch (TTransportException tte) {
-        log.debug("Failed to call finishTableOperation(), retrying ... ", tte);
+        log.debug("Failed to call finishFateOperation(), retrying ... ", tte);
         UtilWaitThread.sleep(100);
       } finally {
         MasterClient.close(client);
@@ -295,38 +291,48 @@ public class TableOperationsImpl extends TableOperationsHelper {
     }
   }
 
-  private String doTableOperation(TableOperation op, List<ByteBuffer> args, Map<String,String> opts) throws AccumuloSecurityException, TableExistsException,
-      TableNotFoundException, AccumuloException {
-    return doTableOperation(op, args, opts, true);
+  String doFateOperation(FateOperation op, List<ByteBuffer> args, Map<String,String> opts) throws AccumuloSecurityException, TableExistsException,
+      TableNotFoundException, AccumuloException, NamespaceExistsException, NamespaceNotFoundException {
+    return doFateOperation(op, args, opts, true);
   }
 
-  private String doTableOperation(TableOperation op, List<ByteBuffer> args, Map<String,String> opts, boolean wait) throws AccumuloSecurityException,
-      TableExistsException, TableNotFoundException, AccumuloException {
+  String doFateOperation(FateOperation op, List<ByteBuffer> args, Map<String,String> opts, boolean wait) throws AccumuloSecurityException,
+      TableExistsException, TableNotFoundException, AccumuloException, NamespaceExistsException, NamespaceNotFoundException {
     Long opid = null;
 
     try {
-      opid = beginTableOperation();
-      executeTableOperation(opid, op, args, opts, !wait);
+      opid = beginFateOperation();
+      executeFateOperation(opid, op, args, opts, !wait);
       if (!wait) {
         opid = null;
         return null;
       }
-      String ret = waitForTableOperation(opid);
+      String ret = waitForFateOperation(opid);
       Tables.clearCache(instance);
       return ret;
     } catch (ThriftSecurityException e) {
       String tableName = ByteBufferUtil.toString(args.get(0));
-      String tableInfo = Tables.getPrintableTableInfoFromName(instance, tableName);
-      throw new AccumuloSecurityException(e.user, e.code, tableInfo, e);
+      switch (e.getCode()) {
+        case TABLE_DOESNT_EXIST:
+          throw new TableNotFoundException(null, tableName, "Target table does not exist");
+        case NAMESPACE_DOESNT_EXIST:
+          throw new NamespaceNotFoundException(null, tableName, "Target namespace does not exist");
+        default:
+          String tableInfo = Tables.getPrintableTableInfoFromName(instance, tableName);
+          throw new AccumuloSecurityException(e.user, e.code, tableInfo, e);
+      }
     } catch (ThriftTableOperationException e) {
       switch (e.getType()) {
         case EXISTS:
           throw new TableExistsException(e);
         case NOTFOUND:
           throw new TableNotFoundException(e);
+        case NAMESPACE_EXISTS:
+          throw new NamespaceExistsException(e);
+        case NAMESPACE_NOTFOUND:
+          throw new NamespaceNotFoundException(e);
         case OFFLINE:
           throw new TableOfflineException(instance, null);
-        case OTHER:
         default:
           throw new AccumuloException(e.description, e);
       }
@@ -336,7 +342,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       // always finish table op, even when exception
       if (opid != null)
         try {
-          finishTableOperation(opid);
+          finishFateOperation(opid);
         } catch (Exception e) {
           log.warn(e.getMessage(), e);
         }
@@ -525,10 +531,10 @@ public class TableOperationsImpl extends TableOperationsHelper {
         : TextUtil.getByteBuffer(end));
     Map<String,String> opts = new HashMap<String,String>();
     try {
-      doTableOperation(TableOperation.MERGE, args, opts);
+      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_MERGE, args, opts);
     } catch (TableExistsException e) {
       // should not happen
-      throw new RuntimeException(e);
+      throw new AssertionError(e);
     }
   }
 
@@ -541,10 +547,10 @@ public class TableOperationsImpl extends TableOperationsHelper {
         : TextUtil.getByteBuffer(end));
     Map<String,String> opts = new HashMap<String,String>();
     try {
-      doTableOperation(TableOperation.DELETE_RANGE, args, opts);
+      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_DELETE_RANGE, args, opts);
     } catch (TableExistsException e) {
       // should not happen
-      throw new RuntimeException(e);
+      throw new AssertionError(e);
     }
   }
 
@@ -666,10 +672,10 @@ public class TableOperationsImpl extends TableOperationsHelper {
     Map<String,String> opts = new HashMap<String,String>();
 
     try {
-      doTableOperation(TableOperation.DELETE, args, opts);
+      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_DELETE, args, opts);
     } catch (TableExistsException e) {
       // should not happen
-      throw new RuntimeException(e);
+      throw new AssertionError(e);
     }
 
   }
@@ -680,12 +686,6 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
     ArgumentChecker.notNull(srcTableName, newTableName);
 
-    String namespace = Tables.qualify(newTableName).getFirst();
-    if (!namespaceExists(namespace)) {
-      String info = "Namespace not found while cloning table";
-      throw new IllegalArgumentException(new NamespaceNotFoundException(null, namespace, info));
-    }
-
     String srcTableId = Tables.getTableId(instance, srcTableName);
 
     if (flush)
@@ -697,14 +697,6 @@ public class TableOperationsImpl extends TableOperationsHelper {
     if (propertiesToSet == null)
       propertiesToSet = Collections.emptyMap();
 
-    HashSet<String> excludeProps = getUniqueNamespaceProperties(namespace, srcTableName, propertiesToSet);
-    for (String p : propertiesToExclude) {
-      excludeProps.add(p);
-    }
-
-    if (!Collections.disjoint(excludeProps, propertiesToSet.keySet()))
-      throw new IllegalArgumentException("propertiesToSet and propertiesToExclude not disjoint");
-
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(srcTableId.getBytes()), ByteBuffer.wrap(newTableName.getBytes()));
     Map<String,String> opts = new HashMap<String,String>();
     for (Entry<String,String> entry : propertiesToSet.entrySet()) {
@@ -717,35 +709,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       opts.put(CLONE_EXCLUDE_PREFIX + prop, "");
     }
 
-    doTableOperation(TableOperation.CLONE, args, opts);
-  }
-
-  // get the properties that are only in the namespace so that we can exclude them when copying table properties.
-  // also, don't exclude properties that are going to be explicitly set.
-  private HashSet<String> getUniqueNamespaceProperties(String namespace, String table, Map<String,String> propsToSet) throws TableNotFoundException,
-      AccumuloException {
-    HashSet<String> props = new HashSet<String>();
-    try {
-      Iterable<Entry<String,String>> n = new NamespaceOperationsImpl(instance, credentials).getProperties(namespace);
-      Iterable<Entry<String,String>> t = getProperties(table);
-      Map<String,String> tmap = new HashMap<String,String>();
-      for (Entry<String,String> e : t) {
-        tmap.put(e.getKey(), e.getValue());
-      }
-      for (Entry<String,String> e : n) {
-        String val = tmap.get(e.getKey());
-        if (e.getValue().equals(val)) {
-          props.add(e.getKey());
-        }
-      }
-    } catch (NamespaceNotFoundException e) {
-      throw new IllegalStateException(new NamespaceNotFoundException(null, namespace, null));
-    }
-
-    for (Entry<String,String> e : propsToSet.entrySet()) {
-      props.remove(e.getKey());
-    }
-    return props;
+    doTableFateOperation(newTableName, AccumuloException.class, FateOperation.TABLE_CLONE, args, opts);
   }
 
   /**
@@ -770,7 +734,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(oldTableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(newTableName.getBytes(Constants.UTF8)));
     Map<String,String> opts = new HashMap<String,String>();
-    doTableOperation(TableOperation.RENAME, args, opts);
+    doTableFateOperation(oldTableName, TableNotFoundException.class, FateOperation.TABLE_RENAME, args, opts);
   }
 
   /**
@@ -827,10 +791,15 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
     Map<String,String> opts = new HashMap<String,String>();
     try {
-      doTableOperation(TableOperation.COMPACT, args, opts, wait);
+      doFateOperation(FateOperation.TABLE_COMPACT, args, opts, wait);
     } catch (TableExistsException e) {
       // should not happen
-      throw new RuntimeException(e);
+      throw new AssertionError(e);
+    } catch (NamespaceExistsException e) {
+      // should not happen
+      throw new AssertionError(e);
+    } catch (NamespaceNotFoundException e) {
+      throw new TableNotFoundException(null, tableName, "Namespace not found", e);
     }
   }
 
@@ -842,10 +811,10 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
     Map<String,String> opts = new HashMap<String,String>();
     try {
-      doTableOperation(TableOperation.COMPACT_CANCEL, args, opts, true);
+      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_CANCEL_COMPACT, args, opts);
     } catch (TableExistsException e) {
       // should not happen
-      throw new RuntimeException(e);
+      throw new AssertionError(e);
     }
 
   }
@@ -887,16 +856,17 @@ public class TableOperationsImpl extends TableOperationsHelper {
         }
       }
     } catch (ThriftSecurityException e) {
-      if (e.getCode().equals(SecurityErrorCode.TABLE_DOESNT_EXIST)) {
-        throw new TableNotFoundException(tableId, null, e.getMessage(), e);
+      switch (e.getCode()) {
+        case TABLE_DOESNT_EXIST:
+          throw new TableNotFoundException(tableId, null, e.getMessage(), e);
+        default:
+          log.debug("flush security exception on table id " + tableId);
+          throw new AccumuloSecurityException(e.user, e.code, e);
       }
-      log.debug("flush security exception on table id " + tableId);
-      throw new AccumuloSecurityException(e.user, e.code, e);
     } catch (ThriftTableOperationException e) {
       switch (e.getType()) {
         case NOTFOUND:
           throw new TableNotFoundException(e);
-        case OTHER:
         default:
           throw new AccumuloException(e.description, e);
       }
@@ -922,12 +892,16 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void setProperty(final String tableName, final String property, final String value) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(tableName, property, value);
-    MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
-      @Override
-      public void execute(MasterClientService.Client client) throws Exception {
-        client.setTableProperty(Tracer.traceInfo(), credentials.toThrift(instance), tableName, property, value);
-      }
-    });
+    try {
+      MasterClient.executeTable(instance, new ClientExec<MasterClientService.Client>() {
+        @Override
+        public void execute(MasterClientService.Client client) throws Exception {
+          client.setTableProperty(Tracer.traceInfo(), credentials.toThrift(instance), tableName, property, value);
+        }
+      });
+    } catch (TableNotFoundException e) {
+      throw new AccumuloException(e);
+    }
   }
 
   /**
@@ -945,12 +919,16 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void removeProperty(final String tableName, final String property) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(tableName, property);
-    MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
-      @Override
-      public void execute(MasterClientService.Client client) throws Exception {
-        client.removeTableProperty(Tracer.traceInfo(), credentials.toThrift(instance), tableName, property);
-      }
-    });
+    try {
+      MasterClient.executeTable(instance, new ClientExec<MasterClientService.Client>() {
+        @Override
+        public void execute(MasterClientService.Client client) throws Exception {
+          client.removeTableProperty(Tracer.traceInfo(), credentials.toThrift(instance), tableName, property);
+        }
+      });
+    } catch (TableNotFoundException e) {
+      throw new AccumuloException(e);
+    }
   }
 
   /**
@@ -976,7 +954,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
       switch (e.getType()) {
         case NOTFOUND:
           throw new TableNotFoundException(e);
-        case OTHER:
+        case NAMESPACE_NOTFOUND:
+          throw new TableNotFoundException(tableName, new NamespaceNotFoundException(e));
         default:
           throw new AccumuloException(e.description, e);
       }
@@ -1021,7 +1000,13 @@ public class TableOperationsImpl extends TableOperationsHelper {
       setProperty(tableName, Property.TABLE_LOCALITY_GROUP_PREFIX + entry.getKey(), value);
     }
 
-    setProperty(tableName, Property.TABLE_LOCALITY_GROUPS.getKey(), StringUtil.join(groups.keySet(), ","));
+    try {
+      setProperty(tableName, Property.TABLE_LOCALITY_GROUPS.getKey(), StringUtil.join(groups.keySet(), ","));
+    } catch (AccumuloException e) {
+      if (e.getCause() instanceof TableNotFoundException)
+        throw (TableNotFoundException) e.getCause();
+      throw e;
+    }
 
     // remove anything extraneous
     String prefix = Property.TABLE_LOCALITY_GROUP_PREFIX.getKey();
@@ -1181,6 +1166,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
   public void importDirectory(String tableName, String dir, String failureDir, boolean setTime) throws IOException, AccumuloSecurityException,
       TableNotFoundException, AccumuloException {
     ArgumentChecker.notNull(tableName, dir, failureDir);
+    // check for table existance
+    Tables.getTableId(instance, tableName);
 
     Path dirPath = checkPath(dir, "Bulk", "");
     Path failPath = checkPath(failureDir, "Bulk", "failure");
@@ -1190,13 +1177,11 @@ public class TableOperationsImpl extends TableOperationsHelper {
     Map<String,String> opts = new HashMap<String,String>();
 
     try {
-      doTableOperation(TableOperation.BULK_IMPORT, args, opts);
+      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_BULK_IMPORT, args, opts);
     } catch (TableExistsException e) {
       // should not happen
-      throw new RuntimeException(e);
+      throw new AssertionError(e);
     }
-    // return new BulkImportHelper(instance, credentials, tableName).importDirectory(new Path(dir), new Path(failureDir), numThreads, numAssignThreads,
-    // disableGC);
   }
 
   private void waitForTableStateTransition(String tableId, TableState expectedState) throws AccumuloException, TableNotFoundException,
@@ -1340,10 +1325,10 @@ public class TableOperationsImpl extends TableOperationsHelper {
     Map<String,String> opts = new HashMap<String,String>();
 
     try {
-      doTableOperation(TableOperation.OFFLINE, args, opts);
+      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_OFFLINE, args, opts);
     } catch (TableExistsException e) {
       // should not happen
-      throw new RuntimeException(e);
+      throw new AssertionError(e);
     }
 
     if (wait)
@@ -1373,10 +1358,10 @@ public class TableOperationsImpl extends TableOperationsHelper {
     Map<String,String> opts = new HashMap<String,String>();
 
     try {
-      doTableOperation(TableOperation.ONLINE, args, opts);
+      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_ONLINE, args, opts);
     } catch (TableExistsException e) {
       // should not happen
-      throw new RuntimeException(e);
+      throw new AssertionError(e);
     }
 
     if (wait)
@@ -1431,7 +1416,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
         switch (e.getType()) {
           case NOTFOUND:
             throw new TableNotFoundException(e);
-          case OTHER:
+          case NAMESPACE_NOTFOUND:
+            throw new TableNotFoundException(e.getTableName(), new NamespaceNotFoundException(e));
           default:
             throw new AccumuloException(e.description, e);
         }
@@ -1513,10 +1499,10 @@ public class TableOperationsImpl extends TableOperationsHelper {
     Map<String,String> opts = Collections.emptyMap();
 
     try {
-      doTableOperation(TableOperation.IMPORT, args, opts);
-    } catch (TableNotFoundException e1) {
+      doTableFateOperation(tableName, AccumuloException.class, FateOperation.TABLE_IMPORT, args, opts);
+    } catch (TableNotFoundException e) {
       // should not happen
-      throw new RuntimeException(e1);
+      throw new AssertionError(e);
     }
 
   }
@@ -1530,10 +1516,10 @@ public class TableOperationsImpl extends TableOperationsHelper {
     Map<String,String> opts = Collections.emptyMap();
 
     try {
-      doTableOperation(TableOperation.EXPORT, args, opts);
-    } catch (TableExistsException e1) {
+      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_EXPORT, args, opts);
+    } catch (TableExistsException e) {
       // should not happen
-      throw new RuntimeException(e1);
+      throw new AssertionError(e);
     }
   }
 
@@ -1553,7 +1539,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
       switch (e.getType()) {
         case NOTFOUND:
           throw new TableNotFoundException(e);
-        case OTHER:
+        case NAMESPACE_NOTFOUND:
+          throw new TableNotFoundException(tableName, new NamespaceNotFoundException(e));
         default:
           throw new AccumuloException(e.description, e);
       }
@@ -1579,7 +1566,26 @@ public class TableOperationsImpl extends TableOperationsHelper {
     return super.addConstraint(tableName, constraintClassName);
   }
 
-  private boolean namespaceExists(String namespace) {
-    return Namespaces.getNameToIdMap(instance).containsKey(namespace);
+  private void doTableFateOperation(String tableName, Class<? extends Exception> namespaceNotFoundExceptionClass, FateOperation op, List<ByteBuffer> args,
+      Map<String,String> opts) throws AccumuloSecurityException, AccumuloException, TableExistsException, TableNotFoundException {
+    try {
+      doFateOperation(op, args, opts);
+    } catch (NamespaceExistsException e) {
+      // should not happen
+      throw new AssertionError(e);
+    } catch (NamespaceNotFoundException e) {
+      if (namespaceNotFoundExceptionClass == null) {
+        // should not happen
+        throw new AssertionError(e);
+      } else if (AccumuloException.class.isAssignableFrom(namespaceNotFoundExceptionClass)) {
+        throw new AccumuloException("Cannot create table in non-existent namespace", e);
+      } else if (TableNotFoundException.class.isAssignableFrom(namespaceNotFoundExceptionClass)) {
+        throw new TableNotFoundException(null, tableName, "Namespace not found", e);
+      } else {
+        // should not happen
+        throw new AssertionError(e);
+      }
+    }
   }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
index fcecb3a..7d9d3ab 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
@@ -34,10 +34,10 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.client.admin.InstanceOperations;
 import org.apache.accumulo.core.client.admin.InstanceOperationsImpl;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.admin.SecurityOperationsImpl;
 import org.apache.accumulo.core.client.admin.NamespaceOperations;
 import org.apache.accumulo.core.client.admin.NamespaceOperationsImpl;
+import org.apache.accumulo.core.client.admin.SecurityOperations;
+import org.apache.accumulo.core.client.admin.SecurityOperationsImpl;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.admin.TableOperationsImpl;
 import org.apache.accumulo.core.client.impl.thrift.ClientService;
@@ -55,16 +55,16 @@ public class ConnectorImpl extends Connector {
   private TableOperations tableops = null;
   private NamespaceOperations namespaceops = null;
   private InstanceOperations instanceops = null;
-  
+
   public ConnectorImpl(final Instance instance, Credentials cred) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(instance, cred);
     if (cred.getToken().isDestroyed())
       throw new AccumuloSecurityException(cred.getPrincipal(), SecurityErrorCode.TOKEN_EXPIRED);
-    
+
     this.instance = instance;
-    
+
     this.credentials = cred;
-    
+
     // Skip fail fast for system services; string literal for class name, to avoid
     if (!"org.apache.accumulo.server.security.SystemCredentials$SystemToken".equals(cred.getToken().getClass().getName())) {
       ServerClient.execute(instance, new ClientExec<ClientService.Client>() {
@@ -75,26 +75,29 @@ public class ConnectorImpl extends Connector {
         }
       });
     }
+
+    this.tableops = new TableOperationsImpl(instance, credentials);
+    this.namespaceops = new NamespaceOperationsImpl(instance, credentials, (TableOperationsImpl) tableops);
   }
-  
+
   private String getTableId(String tableName) throws TableNotFoundException {
     String tableId = Tables.getTableId(instance, tableName);
     if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
       throw new TableOfflineException(instance, tableId);
     return tableId;
   }
-  
+
   @Override
   public Instance getInstance() {
     return instance;
   }
-  
+
   @Override
   public BatchScanner createBatchScanner(String tableName, Authorizations authorizations, int numQueryThreads) throws TableNotFoundException {
     ArgumentChecker.notNull(tableName, authorizations);
     return new TabletServerBatchReader(instance, credentials, getTableId(tableName), authorizations, numQueryThreads);
   }
-  
+
   @Deprecated
   @Override
   public BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations, int numQueryThreads, long maxMemory, long maxLatency,
@@ -103,14 +106,14 @@ public class ConnectorImpl extends Connector {
     return new TabletServerBatchDeleter(instance, credentials, getTableId(tableName), authorizations, numQueryThreads, new BatchWriterConfig()
         .setMaxMemory(maxMemory).setMaxLatency(maxLatency, TimeUnit.MILLISECONDS).setMaxWriteThreads(maxWriteThreads));
   }
-  
+
   @Override
   public BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations, int numQueryThreads, BatchWriterConfig config)
       throws TableNotFoundException {
     ArgumentChecker.notNull(tableName, authorizations);
     return new TabletServerBatchDeleter(instance, credentials, getTableId(tableName), authorizations, numQueryThreads, config);
   }
-  
+
   @Deprecated
   @Override
   public BatchWriter createBatchWriter(String tableName, long maxMemory, long maxLatency, int maxWriteThreads) throws TableNotFoundException {
@@ -118,68 +121,64 @@ public class ConnectorImpl extends Connector {
     return new BatchWriterImpl(instance, credentials, getTableId(tableName), new BatchWriterConfig().setMaxMemory(maxMemory)
         .setMaxLatency(maxLatency, TimeUnit.MILLISECONDS).setMaxWriteThreads(maxWriteThreads));
   }
-  
+
   @Override
   public BatchWriter createBatchWriter(String tableName, BatchWriterConfig config) throws TableNotFoundException {
     ArgumentChecker.notNull(tableName);
     return new BatchWriterImpl(instance, credentials, getTableId(tableName), config);
   }
-  
+
   @Deprecated
   @Override
   public MultiTableBatchWriter createMultiTableBatchWriter(long maxMemory, long maxLatency, int maxWriteThreads) {
     return new MultiTableBatchWriterImpl(instance, credentials, new BatchWriterConfig().setMaxMemory(maxMemory)
         .setMaxLatency(maxLatency, TimeUnit.MILLISECONDS).setMaxWriteThreads(maxWriteThreads));
   }
-  
+
   @Override
   public MultiTableBatchWriter createMultiTableBatchWriter(BatchWriterConfig config) {
     return new MultiTableBatchWriterImpl(instance, credentials, config);
   }
-  
+
   @Override
   public ConditionalWriter createConditionalWriter(String tableName, ConditionalWriterConfig config) throws TableNotFoundException {
     return new ConditionalWriterImpl(instance, credentials, getTableId(tableName), config);
   }
-  
+
   @Override
   public Scanner createScanner(String tableName, Authorizations authorizations) throws TableNotFoundException {
     ArgumentChecker.notNull(tableName, authorizations);
     return new ScannerImpl(instance, credentials, getTableId(tableName), authorizations);
   }
-  
+
   @Override
   public String whoami() {
     return credentials.getPrincipal();
   }
-  
+
   @Override
   public synchronized TableOperations tableOperations() {
-    if (tableops == null)
-      tableops = new TableOperationsImpl(instance, credentials);
     return tableops;
   }
-  
+
   @Override
   public synchronized NamespaceOperations namespaceOperations() {
-    if (namespaceops == null)
-      namespaceops = new NamespaceOperationsImpl(instance, credentials);
     return namespaceops;
   }
-  
+
   @Override
   public synchronized SecurityOperations securityOperations() {
     if (secops == null)
       secops = new SecurityOperationsImpl(instance, credentials);
-    
+
     return secops;
   }
-  
+
   @Override
   public synchronized InstanceOperations instanceOperations() {
     if (instanceops == null)
       instanceops = new InstanceOperationsImpl(instance, credentials);
-    
+
     return instanceops;
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java b/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java
index 6bef3a7..7356663 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java
@@ -22,7 +22,10 @@ import java.util.List;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
+import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
 import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.accumulo.core.util.ThriftUtil;
@@ -33,35 +36,36 @@ import org.apache.thrift.transport.TTransportException;
 
 public class MasterClient {
   private static final Logger log = Logger.getLogger(MasterClient.class);
-  
+
   public static MasterClientService.Client getConnectionWithRetry(Instance instance) {
     ArgumentChecker.notNull(instance);
-    
+
     while (true) {
-      
+
       MasterClientService.Client result = getConnection(instance);
       if (result != null)
         return result;
       UtilWaitThread.sleep(250);
     }
-    
+
   }
-  
+
   public static MasterClientService.Client getConnection(Instance instance) {
     List<String> locations = instance.getMasterLocations();
-    
+
     if (locations.size() == 0) {
       log.debug("No masters...");
       return null;
     }
-    
+
     String master = locations.get(0);
     if (master.endsWith(":0"))
       return null;
-    
+
     try {
       // Master requests can take a long time: don't ever time out
-      MasterClientService.Client client = ThriftUtil.getClientNoTimeout(new MasterClientService.Client.Factory(), master, ServerConfigurationUtil.getConfiguration(instance));
+      MasterClientService.Client client = ThriftUtil.getClientNoTimeout(new MasterClientService.Client.Factory(), master,
+          ServerConfigurationUtil.getConfiguration(instance));
       return client;
     } catch (TTransportException tte) {
       if (tte.getCause().getClass().equals(UnknownHostException.class)) {
@@ -72,7 +76,7 @@ public class MasterClient {
       return null;
     }
   }
-  
+
   public static void close(MasterClientService.Iface iface) {
     TServiceClient client = (TServiceClient) iface;
     if (client != null && client.getInputProtocol() != null && client.getInputProtocol().getTransport() != null) {
@@ -81,8 +85,9 @@ public class MasterClient {
       log.debug("Attempt to close null connection to the master", new Exception());
     }
   }
-  
-  public static <T> T execute(Instance instance, ClientExecReturn<T,MasterClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
+
+  public static <T> T execute(Instance instance, ClientExecReturn<T,MasterClientService.Client> exec) throws AccumuloException, AccumuloSecurityException,
+      TableNotFoundException {
     MasterClientService.Client client = null;
     while (true) {
       try {
@@ -95,6 +100,15 @@ public class MasterClient {
         throw new AccumuloSecurityException(e.user, e.code, e);
       } catch (AccumuloException e) {
         throw e;
+      } catch (ThriftTableOperationException e) {
+        switch (e.getType()) {
+          case NAMESPACE_NOTFOUND:
+            throw new TableNotFoundException(e.getTableName(), new NamespaceNotFoundException(e));
+          case NOTFOUND:
+            throw new TableNotFoundException(e);
+          default:
+            throw new AccumuloException(e);
+        }
       } catch (Exception e) {
         throw new AccumuloException(e);
       } finally {
@@ -103,8 +117,9 @@ public class MasterClient {
       }
     }
   }
-  
-  public static void execute(Instance instance, ClientExec<MasterClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
+
+  public static void executeGeneric(Instance instance, ClientExec<MasterClientService.Client> exec) throws AccumuloException, AccumuloSecurityException,
+      TableNotFoundException {
     MasterClientService.Client client = null;
     while (true) {
       try {
@@ -118,6 +133,15 @@ public class MasterClient {
         throw new AccumuloSecurityException(e.user, e.code, e);
       } catch (AccumuloException e) {
         throw e;
+      } catch (ThriftTableOperationException e) {
+        switch (e.getType()) {
+          case NAMESPACE_NOTFOUND:
+            throw new TableNotFoundException(e.getTableName(), new NamespaceNotFoundException(e));
+          case NOTFOUND:
+            throw new TableNotFoundException(e);
+          default:
+            throw new AccumuloException(e);
+        }
       } catch (Exception e) {
         throw new AccumuloException(e);
       } finally {
@@ -126,5 +150,27 @@ public class MasterClient {
       }
     }
   }
-  
+
+  public static void executeTable(Instance instance, ClientExec<MasterClientService.Client> exec) throws AccumuloException, AccumuloSecurityException,
+      TableNotFoundException {
+    executeGeneric(instance, exec);
+  }
+
+  public static void executeNamespace(Instance instance, ClientExec<MasterClientService.Client> exec) throws AccumuloException, AccumuloSecurityException,
+      NamespaceNotFoundException {
+    try {
+      executeGeneric(instance, exec);
+    } catch (TableNotFoundException e) {
+      if (e.getCause() instanceof NamespaceNotFoundException)
+        throw (NamespaceNotFoundException) e.getCause();
+    }
+  }
+
+  public static void execute(Instance instance, ClientExec<MasterClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
+    try {
+      executeGeneric(instance, exec);
+    } catch (TableNotFoundException e) {
+      throw new AssertionError(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
index 64a6441..1ca5dee 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
@@ -19,6 +19,7 @@ package org.apache.accumulo.core.client.impl;
 import java.security.SecurityPermission;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map.Entry;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
@@ -31,6 +32,11 @@ import org.apache.accumulo.fate.zookeeper.ZooCache;
 public class Namespaces {
   private static SecurityPermission TABLES_PERMISSION = new SecurityPermission("tablesPermission");
 
+  public static final String DEFAULT_NAMESPACE_ID = "+default";
+  public static final String DEFAULT_NAMESPACE = "";
+  public static final String ACCUMULO_NAMESPACE_ID = "+accumulo";
+  public static final String ACCUMULO_NAMESPACE = "accumulo";
+
   private static ZooCache getZooCache(Instance instance) {
     SecurityManager sm = System.getSecurityManager();
     if (sm != null) {
@@ -87,27 +93,20 @@ public class Namespaces {
   }
 
   public static List<String> getTableIds(Instance instance, String namespaceId) throws NamespaceNotFoundException {
-    List<String> l = new LinkedList<String>();
-    for (String id : Tables.getIdToNameMap(instance).keySet()) {
-      if (Tables.getNamespace(instance, id).equals(namespaceId)) {
-        l.add(id);
-      }
-    }
-    return l;
+    String namespace = getNamespaceName(instance, namespaceId);
+    List<String> names = new LinkedList<String>();
+    for (Entry<String,String> nameToId : Tables.getNameToIdMap(instance).entrySet())
+      if (namespace.equals(Tables.qualify(nameToId.getKey()).getFirst()))
+        names.add(nameToId.getValue());
+    return names;
   }
 
   public static List<String> getTableNames(Instance instance, String namespaceId) throws NamespaceNotFoundException {
-    ZooCache zc = getZooCache(instance);
-    List<String> ids = getTableIds(instance, namespaceId);
+    String namespace = getNamespaceName(instance, namespaceId);
     List<String> names = new LinkedList<String>();
-    String namespace = getNamespaceName(instance, namespaceId) + ".";
-    if (namespaceId.equals(Constants.DEFAULT_NAMESPACE_ID) || namespaceId.equals(Constants.ACCUMULO_NAMESPACE_ID)) {
-      // default and system namespaces aren't displayed for backwards compatibility
-      namespace = "";
-    }
-    for (String id : ids) {
-      names.add(namespace + new String(zc.get(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + id + Constants.ZTABLE_NAME), Constants.UTF8));
-    }
+    for (String name : Tables.getNameToIdMap(instance).keySet())
+      if (namespace.equals(Tables.qualify(name).getFirst()))
+        names.add(name);
     return names;
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
index f3f46d5..279453b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
@@ -59,13 +59,13 @@ public class Tables {
     for (String tableId : tableIds) {
       byte[] tableName = zc.get(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_NAME);
       byte[] nId = zc.get(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_NAMESPACE);
-      String namespaceName = Constants.DEFAULT_NAMESPACE;
+      String namespaceName = Namespaces.DEFAULT_NAMESPACE;
       // create fully qualified table name
       if (nId == null) {
         namespaceName = null;
       } else if (nId != null) {
         String namespaceId = new String(nId, Constants.UTF8);
-        if (!namespaceId.equals(Constants.DEFAULT_NAMESPACE_ID)) {
+        if (!namespaceId.equals(Namespaces.DEFAULT_NAMESPACE_ID)) {
           try {
             namespaceName = namespaceIdToNameMap.get(namespaceId);
             if (namespaceName == null) {
@@ -91,9 +91,27 @@ public class Tables {
   }
 
   public static String getTableId(Instance instance, String tableName) throws TableNotFoundException {
+    try {
+      return _getTableId(instance, tableName);
+    } catch (NamespaceNotFoundException e) {
+      throw new TableNotFoundException(tableName, e);
+    }
+  }
+
+  public static String _getTableId(Instance instance, String tableName) throws NamespaceNotFoundException, TableNotFoundException {
     String tableId = getNameToIdMap(instance).get(tableName);
-    if (tableId == null)
-      throw new TableNotFoundException(tableId, tableName, null);
+    if (tableId == null) {
+      // maybe the table exist, but the cache was not updated yet... so try to clear the cache and check again
+      clearCache(instance);
+      tableId = getNameToIdMap(instance).get(tableName);
+      if (tableId == null) {
+        String namespace = qualify(tableName).getFirst();
+        if (Namespaces.getNameToIdMap(instance).containsKey(namespace))
+          throw new TableNotFoundException(null, tableName, null);
+        else
+          throw new NamespaceNotFoundException(null, namespace, null);
+      }
+    }
     return tableId;
   }
 
@@ -169,19 +187,19 @@ public class Tables {
   }
 
   public static String qualified(String tableName) {
-    return qualified(tableName, Constants.DEFAULT_NAMESPACE);
+    return qualified(tableName, Namespaces.DEFAULT_NAMESPACE);
   }
 
   public static String qualified(String tableName, String defaultNamespace) {
     Pair<String,String> qualifiedTableName = Tables.qualify(tableName, defaultNamespace);
-    if (Constants.DEFAULT_NAMESPACE.equals(qualifiedTableName.getFirst()))
+    if (Namespaces.DEFAULT_NAMESPACE.equals(qualifiedTableName.getFirst()))
       return qualifiedTableName.getSecond();
     else
       return qualifiedTableName.toString("", ".", "");
   }
 
   public static Pair<String,String> qualify(String tableName) {
-    return qualify(tableName, Constants.DEFAULT_NAMESPACE);
+    return qualify(tableName, Namespaces.DEFAULT_NAMESPACE);
   }
 
   public static Pair<String,String> qualify(String tableName, String defaultNamespace) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TableOperationExceptionType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TableOperationExceptionType.java b/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TableOperationExceptionType.java
index 313ca08..6222f8e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TableOperationExceptionType.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TableOperationExceptionType.java
@@ -34,7 +34,10 @@ import org.apache.thrift.TEnum;
   BULK_BAD_INPUT_DIRECTORY(3),
   BULK_BAD_ERROR_DIRECTORY(4),
   BAD_RANGE(5),
-  OTHER(6);
+  OTHER(6),
+  NAMESPACE_EXISTS(7),
+  NAMESPACE_NOTFOUND(8),
+  INVALID_NAME(9);
 
   private final int value;
 
@@ -69,6 +72,12 @@ import org.apache.thrift.TEnum;
         return BAD_RANGE;
       case 6:
         return OTHER;
+      case 7:
+        return NAMESPACE_EXISTS;
+      case 8:
+        return NAMESPACE_NOTFOUND;
+      case 9:
+        return INVALID_NAME;
       default:
         return null;
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
index 1d1853e..2c26ecc 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
@@ -22,9 +22,9 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.SortedSet;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.admin.TimeType;
+import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Mutation;
@@ -52,8 +52,8 @@ public class MockAccumulo {
     MockUser root = new MockUser("root", new PasswordToken(new byte[0]), Authorizations.EMPTY);
     root.permissions.add(SystemPermission.SYSTEM);
     users.put(root.name, root);
-    namespaces.put(Constants.DEFAULT_NAMESPACE, new MockNamespace());
-    namespaces.put(Constants.ACCUMULO_NAMESPACE, new MockNamespace());
+    namespaces.put(Namespaces.DEFAULT_NAMESPACE, new MockNamespace());
+    namespaces.put(Namespaces.ACCUMULO_NAMESPACE, new MockNamespace());
     createTable("root", RootTable.NAME, true, TimeType.LOGICAL);
     createTable("root", MetadataTable.NAME, true, TimeType.LOGICAL);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
index 6bf9904..8c73c0f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
@@ -34,6 +34,7 @@ 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.IteratorSetting;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.DiskUsage;
@@ -183,17 +184,14 @@ public class MockTableOperations extends TableOperationsHelper {
   @Override
   public Iterable<Entry<String,String>> getProperties(String tableName) throws TableNotFoundException {
     String namespace = Tables.qualify(tableName).getFirst();
-
-    if (!namespaceExists(namespace)) {
-      throw new IllegalArgumentException("Namespace (" + namespace + ") does not exist");
+    if (!exists(tableName)) {
+      if (!namespaceExists(namespace))
+        throw new TableNotFoundException(tableName, new NamespaceNotFoundException(null, namespace, null));
+      throw new TableNotFoundException(null, tableName, null);
     }
 
     Set<Entry<String,String>> props = new HashSet<Entry<String,String>>(acu.namespaces.get(namespace).settings.entrySet());
 
-    if (!exists(tableName)) {
-      throw new TableNotFoundException(tableName, tableName, "");
-    }
-
     Set<Entry<String,String>> tableProps = acu.tables.get(tableName).settings.entrySet();
     for (Entry<String,String> e : tableProps) {
       if (props.contains(e)) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/master/thrift/FateOperation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/master/thrift/FateOperation.java b/core/src/main/java/org/apache/accumulo/core/master/thrift/FateOperation.java
new file mode 100644
index 0000000..60cb63e
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/master/thrift/FateOperation.java
@@ -0,0 +1,103 @@
+/*
+ * 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.master.thrift;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+@SuppressWarnings("all") public enum FateOperation implements org.apache.thrift.TEnum {
+  TABLE_CREATE(0),
+  TABLE_CLONE(1),
+  TABLE_DELETE(2),
+  TABLE_RENAME(3),
+  TABLE_ONLINE(4),
+  TABLE_OFFLINE(5),
+  TABLE_MERGE(6),
+  TABLE_DELETE_RANGE(7),
+  TABLE_BULK_IMPORT(8),
+  TABLE_COMPACT(9),
+  TABLE_IMPORT(10),
+  TABLE_EXPORT(11),
+  TABLE_CANCEL_COMPACT(12),
+  NAMESPACE_CREATE(13),
+  NAMESPACE_DELETE(14),
+  NAMESPACE_RENAME(15);
+
+  private final int value;
+
+  private FateOperation(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static FateOperation findByValue(int value) { 
+    switch (value) {
+      case 0:
+        return TABLE_CREATE;
+      case 1:
+        return TABLE_CLONE;
+      case 2:
+        return TABLE_DELETE;
+      case 3:
+        return TABLE_RENAME;
+      case 4:
+        return TABLE_ONLINE;
+      case 5:
+        return TABLE_OFFLINE;
+      case 6:
+        return TABLE_MERGE;
+      case 7:
+        return TABLE_DELETE_RANGE;
+      case 8:
+        return TABLE_BULK_IMPORT;
+      case 9:
+        return TABLE_COMPACT;
+      case 10:
+        return TABLE_IMPORT;
+      case 11:
+        return TABLE_EXPORT;
+      case 12:
+        return TABLE_CANCEL_COMPACT;
+      case 13:
+        return NAMESPACE_CREATE;
+      case 14:
+        return NAMESPACE_DELETE;
+      case 15:
+        return NAMESPACE_RENAME;
+      default:
+        return null;
+    }
+  }
+}


[3/7] ACCUMULO-1965 Fix exception handling for namespaces

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java b/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
new file mode 100644
index 0000000..e7f1dde
--- /dev/null
+++ b/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
@@ -0,0 +1,412 @@
+/*
+ * 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.master;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
+import org.apache.accumulo.core.client.admin.TableOperationsImpl;
+import org.apache.accumulo.core.client.admin.TimeType;
+import org.apache.accumulo.core.client.impl.Namespaces;
+import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.client.impl.thrift.TableOperation;
+import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
+import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+import org.apache.accumulo.core.iterators.IteratorUtil;
+import org.apache.accumulo.core.master.thrift.FateOperation;
+import org.apache.accumulo.core.master.thrift.FateService;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.core.util.ByteBufferUtil;
+import org.apache.accumulo.fate.TStore.TStatus;
+import org.apache.accumulo.master.tableOps.BulkImport;
+import org.apache.accumulo.master.tableOps.CancelCompactions;
+import org.apache.accumulo.master.tableOps.ChangeTableState;
+import org.apache.accumulo.master.tableOps.CloneTable;
+import org.apache.accumulo.master.tableOps.CompactRange;
+import org.apache.accumulo.master.tableOps.CreateNamespace;
+import org.apache.accumulo.master.tableOps.CreateTable;
+import org.apache.accumulo.master.tableOps.DeleteNamespace;
+import org.apache.accumulo.master.tableOps.DeleteTable;
+import org.apache.accumulo.master.tableOps.ExportTable;
+import org.apache.accumulo.master.tableOps.ImportTable;
+import org.apache.accumulo.master.tableOps.RenameNamespace;
+import org.apache.accumulo.master.tableOps.RenameTable;
+import org.apache.accumulo.master.tableOps.TableRangeOp;
+import org.apache.accumulo.master.tableOps.TraceRepo;
+import org.apache.accumulo.server.client.ClientServiceHandler;
+import org.apache.accumulo.server.master.state.MergeInfo;
+import org.apache.accumulo.server.util.TablePropUtil;
+import org.apache.accumulo.trace.thrift.TInfo;
+import org.apache.hadoop.io.Text;
+import org.apache.log4j.Logger;
+
+/**
+ * 
+ */
+class FateServiceHandler implements FateService.Iface {
+
+  protected final Master master;
+  protected static final Logger log = Master.log;
+
+  public FateServiceHandler(Master master) {
+    this.master = master;
+  }
+
+  @Override
+  public long beginFateOperation(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException {
+    authenticate(credentials);
+    return master.fate.startTransaction();
+  }
+
+  @Override
+  public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOperation op, List<ByteBuffer> arguments, Map<String,String> options,
+      boolean autoCleanup) throws ThriftSecurityException, ThriftTableOperationException {
+    authenticate(c);
+
+    switch (op) {
+      case NAMESPACE_CREATE: {
+        String namespace = ByteBufferUtil.toString(arguments.get(0));
+        if (!master.security.canCreateNamespace(c, namespace))
+          throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
+        checkNotSystemNamespace(namespace, TableOperation.CREATE);
+        checkNamespaceName(namespace, TableOperation.CREATE);
+        master.fate.seedTransaction(opid, new TraceRepo<Master>(new CreateNamespace(c.getPrincipal(), namespace, options)), autoCleanup);
+        break;
+      }
+      case NAMESPACE_RENAME: {
+
+        String oldName = ByteBufferUtil.toString(arguments.get(0));
+        String newName = ByteBufferUtil.toString(arguments.get(1));
+        String namespaceId = checkNamespaceId(oldName, TableOperation.RENAME);
+
+        checkNotSystemNamespace(oldName, TableOperation.RENAME);
+        checkNotSystemNamespace(newName, TableOperation.RENAME);
+        checkNamespaceName(newName, TableOperation.RENAME);
+        if (!master.security.canRenameNamespace(c, namespaceId, oldName, newName))
+          throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
+        master.fate.seedTransaction(opid, new TraceRepo<Master>(new RenameNamespace(namespaceId, oldName, newName)), autoCleanup);
+        break;
+      }
+      case NAMESPACE_DELETE: {
+        String namespace = ByteBufferUtil.toString(arguments.get(0));
+        checkNotSystemNamespace(namespace, TableOperation.DELETE);
+        String namespaceId = checkNamespaceId(namespace, TableOperation.DELETE);
+        if (!master.security.canDeleteNamespace(c, namespaceId))
+          throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
+        master.fate.seedTransaction(opid, new TraceRepo<Master>(new DeleteNamespace(namespaceId)), autoCleanup);
+        break;
+      }
+      case TABLE_CREATE: {
+        String tableName = ByteBufferUtil.toString(arguments.get(0));
+        if (!master.security.canCreateTable(c, tableName))
+          throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+        checkNotMetadataTable(tableName, TableOperation.DELETE);
+        checkTableName(tableName, TableOperation.CREATE);
+
+        TimeType timeType = TimeType.valueOf(ByteBufferUtil.toString(arguments.get(1)));
+
+        try {
+          master.fate.seedTransaction(opid, new TraceRepo<Master>(new CreateTable(c.getPrincipal(), tableName, timeType, options)), autoCleanup);
+        } catch (NamespaceNotFoundException e) {
+          throw new ThriftTableOperationException(null, tableName, TableOperation.CREATE, TableOperationExceptionType.NAMESPACE_NOTFOUND, "");
+        }
+        break;
+      }
+      case TABLE_RENAME: {
+        String oldTableName = ByteBufferUtil.toString(arguments.get(0));
+        String newTableName = ByteBufferUtil.toString(arguments.get(1));
+
+        String tableId = ClientServiceHandler.checkTableId(master.getInstance(), oldTableName, TableOperation.RENAME);
+        checkNotMetadataTable(oldTableName, TableOperation.RENAME);
+        checkNotMetadataTable(newTableName, TableOperation.RENAME);
+        checkTableName(newTableName, TableOperation.RENAME);
+        if (!master.security.canRenameTable(c, tableId, oldTableName, newTableName))
+          throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
+        try {
+          master.fate.seedTransaction(opid, new TraceRepo<Master>(new RenameTable(tableId, oldTableName, newTableName)), autoCleanup);
+        } catch (NamespaceNotFoundException e) {
+          throw new ThriftTableOperationException(null, oldTableName, TableOperation.RENAME, TableOperationExceptionType.NAMESPACE_NOTFOUND, "");
+        }
+
+        break;
+      }
+      case TABLE_CLONE: {
+        String srcTableId = ByteBufferUtil.toString(arguments.get(0));
+        String tableName = ByteBufferUtil.toString(arguments.get(1));
+        checkNotMetadataTable(tableName, TableOperation.CLONE);
+        checkTableName(tableName, TableOperation.CLONE);
+        if (!master.security.canCloneTable(c, srcTableId, tableName))
+          throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
+        Map<String,String> propertiesToSet = new HashMap<String,String>();
+        Set<String> propertiesToExclude = new HashSet<String>();
+
+        for (Entry<String,String> entry : options.entrySet()) {
+          if (entry.getKey().startsWith(TableOperationsImpl.CLONE_EXCLUDE_PREFIX)) {
+            propertiesToExclude.add(entry.getKey().substring(TableOperationsImpl.CLONE_EXCLUDE_PREFIX.length()));
+            continue;
+          }
+
+          if (!TablePropUtil.isPropertyValid(entry.getKey(), entry.getValue())) {
+            throw new ThriftTableOperationException(null, tableName, TableOperation.CLONE, TableOperationExceptionType.OTHER, "Property or value not valid "
+                + entry.getKey() + "=" + entry.getValue());
+          }
+
+          propertiesToSet.put(entry.getKey(), entry.getValue());
+        }
+
+        master.fate.seedTransaction(opid, new TraceRepo<Master>(new CloneTable(c.getPrincipal(), srcTableId, tableName, propertiesToSet, propertiesToExclude)),
+            autoCleanup);
+
+        break;
+      }
+      case TABLE_DELETE: {
+        String tableName = ByteBufferUtil.toString(arguments.get(0));
+        final String tableId = ClientServiceHandler.checkTableId(master.getInstance(), tableName, TableOperation.DELETE);
+        checkNotMetadataTable(tableName, TableOperation.DELETE);
+        if (!master.security.canDeleteTable(c, tableId))
+          throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+        master.fate.seedTransaction(opid, new TraceRepo<Master>(new DeleteTable(tableId)), autoCleanup);
+        break;
+      }
+      case TABLE_ONLINE: {
+        final String tableId = ByteBufferUtil.toString(arguments.get(0));
+        checkNotRootID(tableId, TableOperation.ONLINE);
+
+        if (!master.security.canOnlineOfflineTable(c, tableId, op))
+          throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
+        master.fate.seedTransaction(opid, new TraceRepo<Master>(new ChangeTableState(tableId, TableOperation.ONLINE)), autoCleanup);
+        break;
+      }
+      case TABLE_OFFLINE: {
+        final String tableId = ByteBufferUtil.toString(arguments.get(0));
+        checkNotRootID(tableId, TableOperation.OFFLINE);
+
+        if (!master.security.canOnlineOfflineTable(c, tableId, op))
+          throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
+        master.fate.seedTransaction(opid, new TraceRepo<Master>(new ChangeTableState(tableId, TableOperation.OFFLINE)), autoCleanup);
+        break;
+      }
+      case TABLE_MERGE: {
+        String tableName = ByteBufferUtil.toString(arguments.get(0));
+        Text startRow = ByteBufferUtil.toText(arguments.get(1));
+        Text endRow = ByteBufferUtil.toText(arguments.get(2));
+        final String tableId = ClientServiceHandler.checkTableId(master.getInstance(), tableName, TableOperation.MERGE);
+        Master.log.debug("Creating merge op: " + tableId + " " + startRow + " " + endRow);
+
+        if (!master.security.canMerge(c, tableId))
+          throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
+        master.fate.seedTransaction(opid, new TraceRepo<Master>(new TableRangeOp(MergeInfo.Operation.MERGE, tableId, startRow, endRow)), autoCleanup);
+        break;
+      }
+      case TABLE_DELETE_RANGE: {
+        String tableName = ByteBufferUtil.toString(arguments.get(0));
+        Text startRow = ByteBufferUtil.toText(arguments.get(1));
+        Text endRow = ByteBufferUtil.toText(arguments.get(2));
+
+        final String tableId = ClientServiceHandler.checkTableId(master.getInstance(), tableName, TableOperation.DELETE_RANGE);
+        checkNotMetadataTable(tableName, TableOperation.DELETE_RANGE);
+
+        if (!master.security.canDeleteRange(c, tableId, tableName, startRow, endRow))
+          throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
+        master.fate.seedTransaction(opid, new TraceRepo<Master>(new TableRangeOp(MergeInfo.Operation.DELETE, tableId, startRow, endRow)), autoCleanup);
+        break;
+      }
+      case TABLE_BULK_IMPORT: {
+        String tableName = ByteBufferUtil.toString(arguments.get(0));
+        String dir = ByteBufferUtil.toString(arguments.get(1));
+        String failDir = ByteBufferUtil.toString(arguments.get(2));
+        boolean setTime = Boolean.parseBoolean(ByteBufferUtil.toString(arguments.get(3)));
+
+        final String tableId = ClientServiceHandler.checkTableId(master.getInstance(), tableName, TableOperation.BULK_IMPORT);
+        checkNotMetadataTable(tableName, TableOperation.BULK_IMPORT);
+
+        if (!master.security.canBulkImport(c, tableId, tableName, dir, failDir))
+          throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
+        master.fate.seedTransaction(opid, new TraceRepo<Master>(new BulkImport(tableId, dir, failDir, setTime)), autoCleanup);
+        break;
+      }
+      case TABLE_COMPACT: {
+        String tableId = ByteBufferUtil.toString(arguments.get(0));
+        byte[] startRow = ByteBufferUtil.toBytes(arguments.get(1));
+        byte[] endRow = ByteBufferUtil.toBytes(arguments.get(2));
+        List<IteratorSetting> iterators = IteratorUtil.decodeIteratorSettings(ByteBufferUtil.toBytes(arguments.get(3)));
+
+        if (!master.security.canCompact(c, tableId))
+          throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
+        master.fate.seedTransaction(opid, new TraceRepo<Master>(new CompactRange(tableId, startRow, endRow, iterators)), autoCleanup);
+        break;
+      }
+      case TABLE_CANCEL_COMPACT: {
+        String tableId = ByteBufferUtil.toString(arguments.get(0));
+
+        if (!master.security.canCompact(c, tableId))
+          throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
+        master.fate.seedTransaction(opid, new TraceRepo<Master>(new CancelCompactions(tableId)), autoCleanup);
+        break;
+      }
+      case TABLE_IMPORT: {
+        String tableName = ByteBufferUtil.toString(arguments.get(0));
+        String exportDir = ByteBufferUtil.toString(arguments.get(1));
+
+        if (!master.security.canImport(c, tableName, exportDir))
+          throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
+        checkNotMetadataTable(tableName, TableOperation.IMPORT);
+        checkTableName(tableName, TableOperation.CREATE);
+
+        try {
+          master.fate.seedTransaction(opid, new TraceRepo<Master>(new ImportTable(c.getPrincipal(), tableName, exportDir)), autoCleanup);
+        } catch (NamespaceNotFoundException e) {
+          throw new ThriftTableOperationException(null, tableName, TableOperation.IMPORT, TableOperationExceptionType.NAMESPACE_NOTFOUND, "");
+        }
+        break;
+      }
+      case TABLE_EXPORT: {
+        String tableName = ByteBufferUtil.toString(arguments.get(0));
+        String exportDir = ByteBufferUtil.toString(arguments.get(1));
+
+        String tableId = ClientServiceHandler.checkTableId(master.getInstance(), tableName, TableOperation.EXPORT);
+
+        if (!master.security.canExport(c, tableId, tableName, exportDir))
+          throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
+        checkNotMetadataTable(tableName, TableOperation.EXPORT);
+
+        master.fate.seedTransaction(opid, new TraceRepo<Master>(new ExportTable(tableName, tableId, exportDir)), autoCleanup);
+        break;
+      }
+
+      default:
+        throw new UnsupportedOperationException();
+    }
+
+  }
+
+  @Override
+  public String waitForFateOperation(TInfo tinfo, TCredentials credentials, long opid) throws ThriftSecurityException, ThriftTableOperationException {
+    authenticate(credentials);
+
+    TStatus status = master.fate.waitForCompletion(opid);
+    if (status == TStatus.FAILED) {
+      Exception e = master.fate.getException(opid);
+      if (e instanceof ThriftTableOperationException)
+        throw (ThriftTableOperationException) e;
+      else if (e instanceof ThriftSecurityException)
+        throw (ThriftSecurityException) e;
+      else if (e instanceof RuntimeException)
+        throw (RuntimeException) e;
+      else
+        throw new RuntimeException(e);
+    }
+
+    String ret = master.fate.getReturn(opid);
+    if (ret == null)
+      ret = ""; // thrift does not like returning null
+    return ret;
+  }
+
+  @Override
+  public void finishFateOperation(TInfo tinfo, TCredentials credentials, long opid) throws ThriftSecurityException {
+    authenticate(credentials);
+    master.fate.delete(opid);
+  }
+
+  protected void authenticate(TCredentials c) throws ThriftSecurityException {
+    if (!master.security.authenticateUser(c, c))
+      throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.BAD_CREDENTIALS);
+  }
+
+  private static void checkNotRootID(String tableId, TableOperation operation) throws ThriftTableOperationException {
+    if (RootTable.ID.equals(tableId)) {
+      String why = "Table name cannot be == " + RootTable.NAME;
+      log.warn(why);
+      throw new ThriftTableOperationException(tableId, null, operation, TableOperationExceptionType.OTHER, why);
+    }
+  }
+
+  private static void checkNotMetadataTable(String tableName, TableOperation operation) throws ThriftTableOperationException {
+    if (MetadataTable.NAME.equals(tableName) || RootTable.NAME.equals(tableName)) {
+      String why = "Table names cannot be == " + RootTable.NAME + " or " + MetadataTable.NAME;
+      log.warn(why);
+      throw new ThriftTableOperationException(null, tableName, operation, TableOperationExceptionType.OTHER, why);
+    }
+  }
+
+  private static void checkNotSystemNamespace(String namespace, TableOperation operation) throws ThriftTableOperationException {
+    if (Namespaces.ACCUMULO_NAMESPACE.equals(namespace)) {
+      String why = "Namespaces cannot be == " + Namespaces.ACCUMULO_NAMESPACE;
+      log.warn(why);
+      throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.OTHER, why);
+    }
+  }
+
+  private void checkTableName(String tableName, TableOperation operation) throws ThriftTableOperationException {
+    if (!tableName.matches(Constants.VALID_TABLE_NAME_REGEX)) {
+      String why = "Table names must only contain word characters (letters, digits, and underscores): " + tableName;
+      log.warn(why);
+      throw new ThriftTableOperationException(null, tableName, operation, TableOperationExceptionType.OTHER, why);
+    }
+    if (Tables.getNameToIdMap(master.getInstance()).containsKey(tableName)) {
+      String why = "Table name already exists: " + tableName;
+      throw new ThriftTableOperationException(null, tableName, operation, TableOperationExceptionType.EXISTS, why);
+    }
+  }
+
+  private void checkNamespaceName(String namespace, TableOperation operation) throws ThriftTableOperationException {
+    if (!namespace.matches(Constants.VALID_NAMESPACE_REGEX)) {
+      String why = "Namespaces must only contain word characters (letters, digits, and underscores): " + namespace;
+      log.warn(why);
+      throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.INVALID_NAME, why);
+    }
+    if (Namespaces.getNameToIdMap(master.getInstance()).containsKey(namespace)) {
+      String why = "Namespace already exists: " + namespace;
+      throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.NAMESPACE_EXISTS, why);
+    }
+  }
+
+  protected String checkNamespaceId(String namespace, TableOperation operation) throws ThriftTableOperationException {
+    final String namespaceId = Namespaces.getNameToIdMap(master.getInstance()).get(namespace);
+    if (namespaceId == null)
+      throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.NAMESPACE_NOTFOUND, null);
+    return namespaceId;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/server/master/src/main/java/org/apache/accumulo/master/Master.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/Master.java b/server/master/src/main/java/org/apache/accumulo/master/Master.java
index 65d6030..60ec46c 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/Master.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/Master.java
@@ -17,11 +17,9 @@
 package org.apache.accumulo.master;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
@@ -38,81 +36,44 @@ 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.IsolatedScanner;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.NamespaceNotFoundException;
-import org.apache.accumulo.core.client.RowIterator;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.TableOperationsImpl;
-import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.ThriftTransportPool;
-import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.impl.thrift.TableOperation;
 import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
-import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.data.thrift.TKeyExtent;
-import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.master.state.tables.TableState;
-import org.apache.accumulo.core.master.thrift.MasterClientService;
 import org.apache.accumulo.core.master.thrift.MasterClientService.Iface;
 import org.apache.accumulo.core.master.thrift.MasterClientService.Processor;
 import org.apache.accumulo.core.master.thrift.MasterGoalState;
-import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
 import org.apache.accumulo.core.master.thrift.MasterState;
-import org.apache.accumulo.core.master.thrift.TableInfo;
-import org.apache.accumulo.core.master.thrift.TabletLoadState;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.master.thrift.TabletSplit;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.security.SecurityUtil;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.security.thrift.TCredentials;
-import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.Daemon;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.AgeOffStore;
 import org.apache.accumulo.fate.Fate;
-import org.apache.accumulo.fate.TStore.TStatus;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter.Mutator;
 import org.apache.accumulo.fate.zookeeper.ZooLock.LockLossReason;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.accumulo.master.recovery.RecoveryManager;
 import org.apache.accumulo.master.state.TableCounts;
-import org.apache.accumulo.master.tableOps.BulkImport;
-import org.apache.accumulo.master.tableOps.CancelCompactions;
-import org.apache.accumulo.master.tableOps.ChangeTableState;
-import org.apache.accumulo.master.tableOps.CloneTable;
-import org.apache.accumulo.master.tableOps.CompactRange;
-import org.apache.accumulo.master.tableOps.CreateNamespace;
-import org.apache.accumulo.master.tableOps.CreateTable;
-import org.apache.accumulo.master.tableOps.DeleteNamespace;
-import org.apache.accumulo.master.tableOps.DeleteTable;
-import org.apache.accumulo.master.tableOps.ExportTable;
-import org.apache.accumulo.master.tableOps.ImportTable;
-import org.apache.accumulo.master.tableOps.RenameNamespace;
-import org.apache.accumulo.master.tableOps.RenameTable;
-import org.apache.accumulo.master.tableOps.TableRangeOp;
-import org.apache.accumulo.master.tableOps.TraceRepo;
-import org.apache.accumulo.master.tserverOps.ShutdownTServer;
 import org.apache.accumulo.server.Accumulo;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.ServerOpts;
@@ -135,7 +96,6 @@ import org.apache.accumulo.server.master.state.RootTabletStateStore;
 import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.master.state.TabletLocationState;
 import org.apache.accumulo.server.master.state.TabletMigration;
-import org.apache.accumulo.server.master.state.TabletServerState;
 import org.apache.accumulo.server.master.state.TabletState;
 import org.apache.accumulo.server.master.state.ZooStore;
 import org.apache.accumulo.server.master.state.ZooTabletStateStore;
@@ -148,18 +108,12 @@ import org.apache.accumulo.server.tables.TableObserver;
 import org.apache.accumulo.server.util.DefaultMap;
 import org.apache.accumulo.server.util.Halt;
 import org.apache.accumulo.server.util.MetadataTableUtil;
-import org.apache.accumulo.server.util.NamespacePropUtil;
-import org.apache.accumulo.server.util.SystemPropUtil;
 import org.apache.accumulo.server.util.TServerUtils;
 import org.apache.accumulo.server.util.TServerUtils.ServerAddress;
-import org.apache.accumulo.server.util.TableInfoUtil;
-import org.apache.accumulo.server.util.TablePropUtil;
-import org.apache.accumulo.server.util.TabletIterator.TabletDeletedException;
 import org.apache.accumulo.server.util.time.SimpleTimer;
 import org.apache.accumulo.server.zookeeper.ZooLock;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.trace.instrument.thrift.TraceWrap;
-import org.apache.accumulo.trace.thrift.TInfo;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
@@ -169,7 +123,6 @@ import org.apache.thrift.TException;
 import org.apache.thrift.server.TServer;
 import org.apache.thrift.transport.TTransportException;
 import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.data.Stat;
@@ -183,7 +136,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
 
   final static Logger log = Logger.getLogger(Master.class);
 
-  final private static int ONE_SECOND = 1000;
+  final static int ONE_SECOND = 1000;
   final private static Text METADATA_TABLE_ID = new Text(MetadataTable.ID);
   final private static Text ROOT_TABLE_ID = new Text(RootTable.ID);
   final static long TIME_TO_WAIT_BETWEEN_SCANS = 60 * ONE_SECOND;
@@ -200,8 +153,8 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
   final private String hostname;
   final LiveTServerSet tserverSet;
   final private List<TabletGroupWatcher> watchers = new ArrayList<TabletGroupWatcher>();
-  final private SecurityOperation security;
-  final private Map<TServerInstance,AtomicInteger> badServers = Collections.synchronizedMap(new DefaultMap<TServerInstance,AtomicInteger>(new AtomicInteger()));
+  final SecurityOperation security;
+  final Map<TServerInstance,AtomicInteger> badServers = Collections.synchronizedMap(new DefaultMap<TServerInstance,AtomicInteger>(new AtomicInteger()));
   final Set<TServerInstance> serversToShutdown = Collections.synchronizedSet(new HashSet<TServerInstance>());
   final SortedMap<KeyExtent,TServerInstance> migrations = Collections.synchronizedSortedMap(new TreeMap<KeyExtent,TServerInstance>());
   final EventCoordinator nextEvent = new EventCoordinator();
@@ -214,13 +167,11 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
 
   private MasterState state = MasterState.INITIAL;
 
-  private Fate<Master> fate;
+  Fate<Master> fate;
 
   volatile SortedMap<TServerInstance,TabletServerStatus> tserverStatus = Collections.unmodifiableSortedMap(new TreeMap<TServerInstance,TabletServerStatus>());
 
-  private final Set<String> recoveriesInProgress = Collections.synchronizedSet(new HashSet<String>());
-
-  synchronized private MasterState getMasterState() {
+  synchronized MasterState getMasterState() {
     return state;
   }
 
@@ -241,7 +192,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       /* UNLOAD_ROOT_TABLET */        {_,     _,        _,        X,      X,         X,          X},
       /* STOP */                      {_,     _,        _,        _,      _,         X,          X}};
   //@formatter:on
-  synchronized private void setMasterState(MasterState newState) {
+  synchronized void setMasterState(MasterState newState) {
     if (state.equals(newState))
       return;
     if (!transitionOK[state.ordinal()][newState.ordinal()]) {
@@ -319,14 +270,15 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
         // create initial namespaces
         String namespaces = ZooUtil.getRoot(instance) + Constants.ZNAMESPACES;
         zoo.putPersistentData(namespaces, new byte[0], NodeExistsPolicy.SKIP);
-        if (!Namespaces.exists(instance, Constants.ACCUMULO_NAMESPACE_ID))
-          TableManager.prepareNewNamespaceState(instance.getInstanceID(), Constants.ACCUMULO_NAMESPACE_ID, Constants.ACCUMULO_NAMESPACE, NodeExistsPolicy.SKIP);
-        if (!Namespaces.exists(instance, Constants.DEFAULT_NAMESPACE_ID))
-          TableManager.prepareNewNamespaceState(instance.getInstanceID(), Constants.DEFAULT_NAMESPACE_ID, Constants.DEFAULT_NAMESPACE, NodeExistsPolicy.SKIP);
+        if (!Namespaces.exists(instance, Namespaces.ACCUMULO_NAMESPACE_ID))
+          TableManager.prepareNewNamespaceState(instance.getInstanceID(), Namespaces.ACCUMULO_NAMESPACE_ID, Namespaces.ACCUMULO_NAMESPACE,
+              NodeExistsPolicy.SKIP);
+        if (!Namespaces.exists(instance, Namespaces.DEFAULT_NAMESPACE_ID))
+          TableManager.prepareNewNamespaceState(instance.getInstanceID(), Namespaces.DEFAULT_NAMESPACE_ID, Namespaces.DEFAULT_NAMESPACE, NodeExistsPolicy.SKIP);
 
         // create root table
         if (!Tables.exists(instance, RootTable.ID)) {
-          TableManager.prepareNewTableState(instance.getInstanceID(), RootTable.ID, Constants.ACCUMULO_NAMESPACE_ID, RootTable.NAME, TableState.ONLINE,
+          TableManager.prepareNewTableState(instance.getInstanceID(), RootTable.ID, Namespaces.ACCUMULO_NAMESPACE_ID, RootTable.NAME, TableState.ONLINE,
               NodeExistsPolicy.SKIP);
           Initialize.initMetadataConfig(RootTable.ID);
           // ensure root user can flush root table
@@ -336,8 +288,8 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
         // put existing tables in the correct namespaces
         String tables = ZooUtil.getRoot(instance) + Constants.ZTABLES;
         for (Entry<String,String> table : Tables.getIdToNameMap(instance).entrySet()) {
-          String targetNamespace = (MetadataTable.ID.equals(table.getKey()) || RootTable.ID.equals(table.getKey())) ? Constants.ACCUMULO_NAMESPACE_ID
-              : Constants.DEFAULT_NAMESPACE_ID;
+          String targetNamespace = (MetadataTable.ID.equals(table.getKey()) || RootTable.ID.equals(table.getKey())) ? Namespaces.ACCUMULO_NAMESPACE_ID
+              : Namespaces.DEFAULT_NAMESPACE_ID;
           zoo.putPersistentData(tables + "/" + table.getKey() + Constants.ZTABLE_NAMESPACE, targetNamespace.getBytes(Constants.UTF8), NodeExistsPolicy.SKIP);
         }
 
@@ -353,9 +305,9 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
         String users = ZooUtil.getRoot(instance) + "/users";
         for (String user : zoo.getChildren(users)) {
           zoo.putPersistentData(users + "/" + user + "/Namespaces", new byte[0], NodeExistsPolicy.SKIP);
-          perm.grantNamespacePermission(user, Constants.ACCUMULO_NAMESPACE_ID, NamespacePermission.READ);
+          perm.grantNamespacePermission(user, Namespaces.ACCUMULO_NAMESPACE_ID, NamespacePermission.READ);
         }
-        perm.grantNamespacePermission("root", Constants.ACCUMULO_NAMESPACE_ID, NamespacePermission.ALTER_TABLE);
+        perm.grantNamespacePermission("root", Namespaces.ACCUMULO_NAMESPACE_ID, NamespacePermission.ALTER_TABLE);
 
       } catch (Exception ex) {
         log.fatal("Error performing upgrade", ex);
@@ -429,7 +381,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
   }
 
   // The number of unassigned tablets that should be assigned: displayed on the monitor page
-  private int displayUnassigned() {
+  int displayUnassigned() {
     int result = 0;
     switch (getMasterState()) {
       case NORMAL:
@@ -464,148 +416,17 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
     return result;
   }
 
-  private static void checkNotRootID(String tableId, TableOperation operation) throws ThriftTableOperationException {
-    if (RootTable.ID.equals(tableId)) {
-      String why = "Table name cannot be == " + RootTable.NAME;
-      log.warn(why);
-      throw new ThriftTableOperationException(tableId, null, operation, TableOperationExceptionType.OTHER, why);
-    }
-  }
-
-  private static void checkNotMetadataTable(String tableName, TableOperation operation) throws ThriftTableOperationException {
-    if (MetadataTable.NAME.equals(tableName) || RootTable.NAME.equals(tableName)) {
-      String why = "Table names cannot be == " + RootTable.NAME + " or " + MetadataTable.NAME;
-      log.warn(why);
-      throw new ThriftTableOperationException(null, tableName, operation, TableOperationExceptionType.OTHER, why);
-    }
-  }
-
-  private static void checkNotSystemNamespace(String namespace, TableOperation operation) throws ThriftTableOperationException {
-    if (Constants.ACCUMULO_NAMESPACE.equals(namespace)) {
-      String why = "Namespaces cannot be == " + Constants.ACCUMULO_NAMESPACE;
-      log.warn(why);
-      throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.OTHER, why);
-    }
-  }
-
-  private void checkTableName(String tableName, TableOperation operation) throws ThriftTableOperationException {
-    if (!tableName.matches(Constants.VALID_TABLE_NAME_REGEX)) {
-      String why = "Table names must only contain word characters (letters, digits, and underscores): " + tableName;
-      log.warn(why);
-      throw new ThriftTableOperationException(null, tableName, operation, TableOperationExceptionType.OTHER, why);
-    }
-    if (Tables.getNameToIdMap(instance).containsKey(tableName)) {
-      String why = "Table name already exists: " + tableName;
-      throw new ThriftTableOperationException(null, tableName, operation, TableOperationExceptionType.EXISTS, why);
-    }
-  }
-
-  private void checkNamespaceName(String namespace, TableOperation operation) throws ThriftTableOperationException {
-    if (!namespace.matches(Constants.VALID_NAMESPACE_REGEX)) {
-      String why = "Namespaces must only contain word characters (letters, digits, and underscores): " + namespace;
-      log.warn(why);
-      throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.OTHER, why);
-    }
-    if (Namespaces.getNameToIdMap(instance).containsKey(namespace)) {
-      String why = "Namespace already exists: " + namespace;
-      throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.EXISTS, why);
-    }
-  }
-
-  private String checkNamespaceId(String namespace, TableOperation operation) throws ThriftTableOperationException {
-    final String namespaceId = Namespaces.getNameToIdMap(instance).get(namespace);
-    if (namespaceId == null)
-      throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.NOTFOUND, null);
-    return namespaceId;
-  }
-
-  private void authenticate(TCredentials c) throws ThriftSecurityException {
-    if (!security.authenticateUser(c, c))
-      throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.BAD_CREDENTIALS);
-
-  }
-
-  private void updatePlugins(String property) {
-    if (property.equals(Property.MASTER_TABLET_BALANCER.getKey())) {
-      TabletBalancer balancer = ServerConfiguration.getSystemConfiguration(instance).instantiateClassProperty(Property.MASTER_TABLET_BALANCER,
-          TabletBalancer.class, new DefaultLoadBalancer());
-      balancer.init(serverConfig);
-      tabletBalancer = balancer;
-      log.info("tablet balancer changed to " + tabletBalancer.getClass().getName());
-    }
-  }
-
   public void mustBeOnline(final String tableId) throws ThriftTableOperationException {
     Tables.clearCache(instance);
     if (!Tables.getTableState(instance, tableId).equals(TableState.ONLINE))
       throw new ThriftTableOperationException(tableId, null, TableOperation.MERGE, TableOperationExceptionType.OFFLINE, "table is not online");
   }
 
-  private void alterTableProperty(TCredentials c, String tableName, String property, String value, TableOperation op) throws ThriftSecurityException,
-      ThriftTableOperationException {
-    final String tableId = checkTableId(tableName, op);
-    if (!security.canAlterTable(c, tableId))
-      throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-
-    try {
-      if (value == null || value.isEmpty()) {
-        TablePropUtil.removeTableProperty(tableId, property);
-      } else if (!TablePropUtil.setTableProperty(tableId, property, value)) {
-        throw new Exception("Invalid table property.");
-      }
-    } catch (KeeperException.NoNodeException e) {
-      // race condition... table no longer exists? This call will throw an exception if the table was deleted:
-      checkTableId(tableName, op);
-      log.info("Error altering table property", e);
-      throw new ThriftTableOperationException(tableId, tableName, op, TableOperationExceptionType.OTHER, "Problem altering table property");
-    } catch (Exception e) {
-      log.error("Problem altering table property", e);
-      throw new ThriftTableOperationException(tableId, tableName, op, TableOperationExceptionType.OTHER, "Problem altering table property");
-    }
-  }
-
-  protected String checkTableId(String tableName, TableOperation operation) throws ThriftTableOperationException {
-    final String tableId = Tables.getNameToIdMap(getConfiguration().getInstance()).get(tableName);
-    if (tableId == null)
-      throw new ThriftTableOperationException(null, tableName, operation, TableOperationExceptionType.NOTFOUND, null);
-    return tableId;
-  }
-
-  private void alterNamespaceProperty(TCredentials c, String namespace, String property, String value, TableOperation op) throws ThriftSecurityException,
-      ThriftTableOperationException {
-
-    String namespaceId = null;
-    namespaceId = checkNamespaceId(namespace, op);
-
-    if (!security.canAlterNamespace(c, namespaceId))
-      throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-
-    try {
-      if (value == null) {
-        NamespacePropUtil.removeNamespaceProperty(namespaceId, property);
-      } else {
-        NamespacePropUtil.setNamespaceProperty(namespaceId, property, value);
-      }
-    } catch (KeeperException.NoNodeException e) {
-      // race condition... namespace no longer exists? This call will throw an exception if the namespace was deleted:
-      checkNamespaceId(namespaceId, op);
-      log.info("Error altering namespace property", e);
-      throw new ThriftTableOperationException(namespaceId, namespace, op, TableOperationExceptionType.OTHER, "Problem altering namespaceproperty");
-    } catch (Exception e) {
-      log.error("Problem altering namespace property", e);
-      throw new ThriftTableOperationException(namespaceId, namespace, op, TableOperationExceptionType.OTHER, "Problem altering namespace property");
-    }
-  }
-
   public Connector getConnector() throws AccumuloException, AccumuloSecurityException {
     return instance.getConnector(SystemCredentials.get().getPrincipal(), SystemCredentials.get().getToken());
   }
 
-  private void waitAround(EventCoordinator.Listener listener) {
-    listener.waitForEvents(ONE_SECOND);
-  }
-
-  public Master(ServerConfiguration config, VolumeManager fs, String hostname) throws IOException {
+  private Master(ServerConfiguration config, VolumeManager fs, String hostname) throws IOException {
     this.serverConfig = config;
     this.instance = config.getInstance();
     this.fs = fs;
@@ -623,614 +444,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
   }
 
   public TServerConnection getConnection(TServerInstance server) {
-    try {
-      return tserverSet.getConnection(server);
-    } catch (TException ex) {
-      return null;
-    }
-  }
-
-  private class MasterClientServiceHandler implements MasterClientService.Iface {
-
-    @Override
-    public long initiateFlush(TInfo tinfo, TCredentials c, String tableId) throws ThriftSecurityException, ThriftTableOperationException, TException {
-      security.canFlush(c, tableId);
-
-      String zTablePath = Constants.ZROOT + "/" + getConfiguration().getInstance().getInstanceID() + Constants.ZTABLES + "/" + tableId
-          + Constants.ZTABLE_FLUSH_ID;
-
-      IZooReaderWriter zoo = ZooReaderWriter.getInstance();
-      byte fid[];
-      try {
-        fid = zoo.mutate(zTablePath, null, null, new Mutator() {
-          @Override
-          public byte[] mutate(byte[] currentValue) throws Exception {
-            long flushID = Long.parseLong(new String(currentValue));
-            flushID++;
-            return ("" + flushID).getBytes();
-          }
-        });
-      } catch (NoNodeException nne) {
-        throw new ThriftTableOperationException(tableId, null, TableOperation.FLUSH, TableOperationExceptionType.NOTFOUND, null);
-      } catch (Exception e) {
-        log.warn(e.getMessage(), e);
-        throw new ThriftTableOperationException(tableId, null, TableOperation.FLUSH, TableOperationExceptionType.OTHER, null);
-      }
-      return Long.parseLong(new String(fid));
-    }
-
-    @Override
-    public void waitForFlush(TInfo tinfo, TCredentials c, String tableId, ByteBuffer startRow, ByteBuffer endRow, long flushID, long maxLoops)
-        throws ThriftSecurityException, ThriftTableOperationException, TException {
-      security.canFlush(c, tableId);
-
-      if (endRow != null && startRow != null && ByteBufferUtil.toText(startRow).compareTo(ByteBufferUtil.toText(endRow)) >= 0)
-        throw new ThriftTableOperationException(tableId, null, TableOperation.FLUSH, TableOperationExceptionType.BAD_RANGE,
-            "start row must be less than end row");
-
-      Set<TServerInstance> serversToFlush = new HashSet<TServerInstance>(tserverSet.getCurrentServers());
-
-      for (long l = 0; l < maxLoops; l++) {
-
-        for (TServerInstance instance : serversToFlush) {
-          try {
-            final TServerConnection server = tserverSet.getConnection(instance);
-            if (server != null)
-              server.flush(masterLock, tableId, ByteBufferUtil.toBytes(startRow), ByteBufferUtil.toBytes(endRow));
-          } catch (TException ex) {
-            log.error(ex.toString());
-          }
-        }
-
-        if (l == maxLoops - 1)
-          break;
-
-        UtilWaitThread.sleep(50);
-
-        serversToFlush.clear();
-
-        try {
-          Connector conn = getConnector();
-          Scanner scanner;
-          if (tableId.equals(MetadataTable.ID)) {
-            scanner = new IsolatedScanner(conn.createScanner(RootTable.NAME, Authorizations.EMPTY));
-            scanner.setRange(MetadataSchema.TabletsSection.getRange());
-          } else {
-            scanner = new IsolatedScanner(conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY));
-            scanner.setRange(new KeyExtent(new Text(tableId), null, ByteBufferUtil.toText(startRow)).toMetadataRange());
-          }
-          TabletsSection.ServerColumnFamily.FLUSH_COLUMN.fetch(scanner);
-          TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.fetch(scanner);
-          scanner.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);
-          scanner.fetchColumnFamily(LogColumnFamily.NAME);
-
-          RowIterator ri = new RowIterator(scanner);
-
-          int tabletsToWaitFor = 0;
-          int tabletCount = 0;
-
-          Text ert = ByteBufferUtil.toText(endRow);
-
-          while (ri.hasNext()) {
-            Iterator<Entry<Key,Value>> row = ri.next();
-            long tabletFlushID = -1;
-            int logs = 0;
-            boolean online = false;
-
-            TServerInstance server = null;
-
-            Entry<Key,Value> entry = null;
-            while (row.hasNext()) {
-              entry = row.next();
-              Key key = entry.getKey();
-
-              if (TabletsSection.ServerColumnFamily.FLUSH_COLUMN.equals(key.getColumnFamily(), key.getColumnQualifier())) {
-                tabletFlushID = Long.parseLong(entry.getValue().toString());
-              }
-
-              if (LogColumnFamily.NAME.equals(key.getColumnFamily()))
-                logs++;
-
-              if (TabletsSection.CurrentLocationColumnFamily.NAME.equals(key.getColumnFamily())) {
-                online = true;
-                server = new TServerInstance(entry.getValue(), key.getColumnQualifier());
-              }
-
-            }
-
-            // when tablet is not online and has no logs, there is no reason to wait for it
-            if ((online || logs > 0) && tabletFlushID < flushID) {
-              tabletsToWaitFor++;
-              if (server != null)
-                serversToFlush.add(server);
-            }
-
-            tabletCount++;
-
-            Text tabletEndRow = new KeyExtent(entry.getKey().getRow(), (Text) null).getEndRow();
-            if (tabletEndRow == null || (ert != null && tabletEndRow.compareTo(ert) >= 0))
-              break;
-          }
-
-          if (tabletsToWaitFor == 0)
-            break;
-
-          // TODO detect case of table offline AND tablets w/ logs? - ACCUMULO-1296
-
-          if (tabletCount == 0 && !Tables.exists(instance, tableId))
-            throw new ThriftTableOperationException(tableId, null, TableOperation.FLUSH, TableOperationExceptionType.NOTFOUND, null);
-
-        } catch (AccumuloException e) {
-          log.debug("Failed to scan " + MetadataTable.NAME + " table to wait for flush " + tableId, e);
-        } catch (TabletDeletedException tde) {
-          log.debug("Failed to scan " + MetadataTable.NAME + " table to wait for flush " + tableId, tde);
-        } catch (AccumuloSecurityException e) {
-          log.warn(e.getMessage(), e);
-          throw new ThriftSecurityException();
-        } catch (TableNotFoundException e) {
-          log.error(e.getMessage(), e);
-          throw new ThriftTableOperationException();
-        }
-      }
-
-    }
-
-    @Override
-    public MasterMonitorInfo getMasterStats(TInfo info, TCredentials credentials) throws ThriftSecurityException, TException {
-      final MasterMonitorInfo result = new MasterMonitorInfo();
-
-      result.tServerInfo = new ArrayList<TabletServerStatus>();
-      result.tableMap = new DefaultMap<String,TableInfo>(new TableInfo());
-      for (Entry<TServerInstance,TabletServerStatus> serverEntry : tserverStatus.entrySet()) {
-        final TabletServerStatus status = serverEntry.getValue();
-        result.tServerInfo.add(status);
-        for (Entry<String,TableInfo> entry : status.tableMap.entrySet()) {
-          TableInfoUtil.add(result.tableMap.get(entry.getKey()), entry.getValue());
-        }
-      }
-      result.badTServers = new HashMap<String,Byte>();
-      synchronized (badServers) {
-        for (TServerInstance bad : badServers.keySet()) {
-          result.badTServers.put(bad.hostPort(), TabletServerState.UNRESPONSIVE.getId());
-        }
-      }
-      result.state = getMasterState();
-      result.goalState = getMasterGoalState();
-      result.unassignedTablets = Master.this.displayUnassigned();
-      result.serversShuttingDown = new HashSet<String>();
-      synchronized (serversToShutdown) {
-        for (TServerInstance server : serversToShutdown)
-          result.serversShuttingDown.add(server.hostPort());
-      }
-      DeadServerList obit = new DeadServerList(ZooUtil.getRoot(instance) + Constants.ZDEADTSERVERS);
-      result.deadTabletServers = obit.getList();
-      return result;
-    }
-
-    @Override
-    public void removeTableProperty(TInfo info, TCredentials credentials, String tableName, String property) throws ThriftSecurityException,
-        ThriftTableOperationException, TException {
-      alterTableProperty(credentials, tableName, property, null, TableOperation.REMOVE_PROPERTY);
-    }
-
-    @Override
-    public void setTableProperty(TInfo info, TCredentials credentials, String tableName, String property, String value) throws ThriftSecurityException,
-        ThriftTableOperationException, TException {
-      alterTableProperty(credentials, tableName, property, value, TableOperation.SET_PROPERTY);
-    }
-
-    @Override
-    public void shutdown(TInfo info, TCredentials c, boolean stopTabletServers) throws ThriftSecurityException, TException {
-      security.canPerformSystemActions(c);
-      Master.this.shutdown(stopTabletServers);
-    }
-
-    @Override
-    public void shutdownTabletServer(TInfo info, TCredentials c, String tabletServer, boolean force) throws ThriftSecurityException, TException {
-      security.canPerformSystemActions(c);
-
-      final TServerInstance doomed = tserverSet.find(tabletServer);
-      if (!force) {
-        final TServerConnection server = tserverSet.getConnection(doomed);
-        if (server == null) {
-          log.warn("No server found for name " + tabletServer);
-          return;
-        }
-      }
-
-      long tid = fate.startTransaction();
-      fate.seedTransaction(tid, new TraceRepo<Master>(new ShutdownTServer(doomed, force)), false);
-      fate.waitForCompletion(tid);
-      fate.delete(tid);
-    }
-
-    @Override
-    public void reportSplitExtent(TInfo info, TCredentials credentials, String serverName, TabletSplit split) throws TException {
-      KeyExtent oldTablet = new KeyExtent(split.oldTablet);
-      if (migrations.remove(oldTablet) != null) {
-        log.info("Canceled migration of " + split.oldTablet);
-      }
-      for (TServerInstance instance : tserverSet.getCurrentServers()) {
-        if (serverName.equals(instance.hostPort())) {
-          nextEvent.event("%s reported split %s, %s", serverName, new KeyExtent(split.newTablets.get(0)), new KeyExtent(split.newTablets.get(1)));
-          return;
-        }
-      }
-      log.warn("Got a split from a server we don't recognize: " + serverName);
-    }
-
-    @Override
-    public void reportTabletStatus(TInfo info, TCredentials credentials, String serverName, TabletLoadState status, TKeyExtent ttablet) throws TException {
-      KeyExtent tablet = new KeyExtent(ttablet);
-
-      switch (status) {
-        case LOAD_FAILURE:
-          log.error(serverName + " reports assignment failed for tablet " + tablet);
-          break;
-        case LOADED:
-          nextEvent.event("tablet %s was loaded on %s", tablet, serverName);
-          break;
-        case UNLOADED:
-          nextEvent.event("tablet %s was unloaded from %s", tablet, serverName);
-          break;
-        case UNLOAD_ERROR:
-          log.error(serverName + " reports unload failed for tablet " + tablet);
-          break;
-        case UNLOAD_FAILURE_NOT_SERVING:
-          if (log.isTraceEnabled()) {
-            log.trace(serverName + " reports unload failed: not serving tablet, could be a split: " + tablet);
-          }
-          break;
-        case CHOPPED:
-          nextEvent.event("tablet %s chopped", tablet);
-          break;
-      }
-    }
-
-    @Override
-    public void setMasterGoalState(TInfo info, TCredentials c, MasterGoalState state) throws ThriftSecurityException, TException {
-      security.canPerformSystemActions(c);
-
-      Master.this.setMasterGoalState(state);
-    }
-
-    @Override
-    public void removeSystemProperty(TInfo info, TCredentials c, String property) throws ThriftSecurityException, TException {
-      security.canPerformSystemActions(c);
-
-      try {
-        SystemPropUtil.removeSystemProperty(property);
-        updatePlugins(property);
-      } catch (Exception e) {
-        log.error("Problem removing config property in zookeeper", e);
-        throw new TException(e.getMessage());
-      }
-    }
-
-    @Override
-    public void setSystemProperty(TInfo info, TCredentials c, String property, String value) throws ThriftSecurityException, TException {
-      security.canPerformSystemActions(c);
-
-      try {
-        SystemPropUtil.setSystemProperty(property, value);
-        updatePlugins(property);
-      } catch (Exception e) {
-        log.error("Problem setting config property in zookeeper", e);
-        throw new TException(e.getMessage());
-      }
-    }
-
-    @Override
-    public long beginTableOperation(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException, TException {
-      authenticate(credentials);
-      return fate.startTransaction();
-    }
-
-    @Override
-    public void executeTableOperation(TInfo tinfo, TCredentials c, long opid, org.apache.accumulo.core.master.thrift.TableOperation op,
-        List<ByteBuffer> arguments, Map<String,String> options, boolean autoCleanup) throws ThriftSecurityException, ThriftTableOperationException, TException {
-      authenticate(c);
-
-      switch (op) {
-        case CREATE: {
-          String tableName = ByteBufferUtil.toString(arguments.get(0));
-          if (!security.canCreateTable(c, tableName))
-            throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-          checkNotMetadataTable(tableName, TableOperation.CREATE);
-          checkTableName(tableName, TableOperation.CREATE);
-
-          TimeType timeType = TimeType.valueOf(ByteBufferUtil.toString(arguments.get(1)));
-
-          try {
-            fate.seedTransaction(opid, new TraceRepo<Master>(new CreateTable(c.getPrincipal(), tableName, timeType, options)), autoCleanup);
-          } catch (NamespaceNotFoundException e) {
-            throw new TException(e.getMessage(), e);
-          }
-          break;
-        }
-        case RENAME: {
-          String oldTableName = ByteBufferUtil.toString(arguments.get(0));
-          String newTableName = ByteBufferUtil.toString(arguments.get(1));
-
-          String tableId = checkTableId(oldTableName, TableOperation.RENAME);
-          checkNotMetadataTable(oldTableName, TableOperation.RENAME);
-          checkNotMetadataTable(newTableName, TableOperation.RENAME);
-          checkTableName(newTableName, TableOperation.RENAME);
-          if (!security.canRenameTable(c, tableId, oldTableName, newTableName))
-            throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-
-          try {
-            fate.seedTransaction(opid, new TraceRepo<Master>(new RenameTable(tableId, oldTableName, newTableName)), autoCleanup);
-          } catch (NamespaceNotFoundException e) {
-            throw new TException(e.getMessage(), e);
-          }
-
-          break;
-        }
-        case CLONE: {
-          String srcTableId = ByteBufferUtil.toString(arguments.get(0));
-          String tableName = ByteBufferUtil.toString(arguments.get(1));
-          checkNotMetadataTable(tableName, TableOperation.CLONE);
-          checkTableName(tableName, TableOperation.CLONE);
-          if (!security.canCloneTable(c, srcTableId, tableName))
-            throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-
-          Map<String,String> propertiesToSet = new HashMap<String,String>();
-          Set<String> propertiesToExclude = new HashSet<String>();
-
-          for (Entry<String,String> entry : options.entrySet()) {
-            if (entry.getKey().startsWith(TableOperationsImpl.CLONE_EXCLUDE_PREFIX)) {
-              propertiesToExclude.add(entry.getKey().substring(TableOperationsImpl.CLONE_EXCLUDE_PREFIX.length()));
-              continue;
-            }
-
-            if (!TablePropUtil.isPropertyValid(entry.getKey(), entry.getValue())) {
-              throw new ThriftTableOperationException(null, tableName, TableOperation.CLONE, TableOperationExceptionType.OTHER, "Property or value not valid "
-                  + entry.getKey() + "=" + entry.getValue());
-            }
-
-            propertiesToSet.put(entry.getKey(), entry.getValue());
-          }
-
-          fate.seedTransaction(opid, new TraceRepo<Master>(new CloneTable(c.getPrincipal(), srcTableId, tableName, propertiesToSet, propertiesToExclude)),
-              autoCleanup);
-
-          break;
-        }
-        case DELETE: {
-          String tableName = ByteBufferUtil.toString(arguments.get(0));
-          final String tableId = checkTableId(tableName, TableOperation.DELETE);
-          checkNotMetadataTable(tableName, TableOperation.DELETE);
-          if (!security.canDeleteTable(c, tableId))
-            throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-          fate.seedTransaction(opid, new TraceRepo<Master>(new DeleteTable(tableId)), autoCleanup);
-          break;
-        }
-        case ONLINE: {
-          final String tableId = ByteBufferUtil.toString(arguments.get(0));
-          checkNotRootID(tableId, TableOperation.ONLINE);
-
-          if (!security.canOnlineOfflineTable(c, tableId, op))
-            throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-
-          fate.seedTransaction(opid, new TraceRepo<Master>(new ChangeTableState(tableId, TableOperation.ONLINE)), autoCleanup);
-          break;
-        }
-        case OFFLINE: {
-          final String tableId = ByteBufferUtil.toString(arguments.get(0));
-          checkNotRootID(tableId, TableOperation.OFFLINE);
-
-          if (!security.canOnlineOfflineTable(c, tableId, op))
-            throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-
-          fate.seedTransaction(opid, new TraceRepo<Master>(new ChangeTableState(tableId, TableOperation.OFFLINE)), autoCleanup);
-          break;
-        }
-        case MERGE: {
-          String tableName = ByteBufferUtil.toString(arguments.get(0));
-          Text startRow = ByteBufferUtil.toText(arguments.get(1));
-          Text endRow = ByteBufferUtil.toText(arguments.get(2));
-          final String tableId = checkTableId(tableName, TableOperation.MERGE);
-          log.debug("Creating merge op: " + tableId + " " + startRow + " " + endRow);
-
-          if (!security.canMerge(c, tableId))
-            throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-
-          fate.seedTransaction(opid, new TraceRepo<Master>(new TableRangeOp(MergeInfo.Operation.MERGE, tableId, startRow, endRow)), autoCleanup);
-          break;
-        }
-        case DELETE_RANGE: {
-          String tableName = ByteBufferUtil.toString(arguments.get(0));
-          Text startRow = ByteBufferUtil.toText(arguments.get(1));
-          Text endRow = ByteBufferUtil.toText(arguments.get(2));
-
-          final String tableId = checkTableId(tableName, TableOperation.DELETE_RANGE);
-          checkNotMetadataTable(tableName, TableOperation.DELETE_RANGE);
-
-          if (!security.canDeleteRange(c, tableId, tableName, startRow, endRow))
-            throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-
-          fate.seedTransaction(opid, new TraceRepo<Master>(new TableRangeOp(MergeInfo.Operation.DELETE, tableId, startRow, endRow)), autoCleanup);
-          break;
-        }
-        case BULK_IMPORT: {
-          String tableName = ByteBufferUtil.toString(arguments.get(0));
-          String dir = ByteBufferUtil.toString(arguments.get(1));
-          String failDir = ByteBufferUtil.toString(arguments.get(2));
-          boolean setTime = Boolean.parseBoolean(ByteBufferUtil.toString(arguments.get(3)));
-
-          final String tableId = checkTableId(tableName, TableOperation.BULK_IMPORT);
-          checkNotMetadataTable(tableName, TableOperation.BULK_IMPORT);
-
-          if (!security.canBulkImport(c, tableId, tableName, dir, failDir))
-            throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-
-          fate.seedTransaction(opid, new TraceRepo<Master>(new BulkImport(tableId, dir, failDir, setTime)), autoCleanup);
-          break;
-        }
-        case COMPACT: {
-          String tableId = ByteBufferUtil.toString(arguments.get(0));
-          byte[] startRow = ByteBufferUtil.toBytes(arguments.get(1));
-          byte[] endRow = ByteBufferUtil.toBytes(arguments.get(2));
-          List<IteratorSetting> iterators = IteratorUtil.decodeIteratorSettings(ByteBufferUtil.toBytes(arguments.get(3)));
-
-          if (!security.canCompact(c, tableId))
-            throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-
-          fate.seedTransaction(opid, new TraceRepo<Master>(new CompactRange(tableId, startRow, endRow, iterators)), autoCleanup);
-          break;
-        }
-        case COMPACT_CANCEL: {
-          String tableId = ByteBufferUtil.toString(arguments.get(0));
-
-          if (!security.canCompact(c, tableId))
-            throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-
-          fate.seedTransaction(opid, new TraceRepo<Master>(new CancelCompactions(tableId)), autoCleanup);
-          break;
-        }
-        case IMPORT: {
-          String tableName = ByteBufferUtil.toString(arguments.get(0));
-          String exportDir = ByteBufferUtil.toString(arguments.get(1));
-
-          if (!security.canImport(c, tableName, exportDir))
-            throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-
-          checkNotMetadataTable(tableName, TableOperation.CREATE);
-          checkTableName(tableName, TableOperation.CREATE);
-
-          try {
-            fate.seedTransaction(opid, new TraceRepo<Master>(new ImportTable(c.getPrincipal(), tableName, exportDir)), autoCleanup);
-          } catch (NamespaceNotFoundException e) {
-            throw new TException(e.getMessage(), e);
-          }
-          break;
-        }
-        case EXPORT: {
-          String tableName = ByteBufferUtil.toString(arguments.get(0));
-          String exportDir = ByteBufferUtil.toString(arguments.get(1));
-
-          String tableId = checkTableId(tableName, TableOperation.EXPORT);
-
-          if (!security.canExport(c, tableId, tableName, exportDir))
-            throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-
-          checkNotMetadataTable(tableName, TableOperation.EXPORT);
-
-          fate.seedTransaction(opid, new TraceRepo<Master>(new ExportTable(tableName, tableId, exportDir)), autoCleanup);
-          break;
-        }
-
-        default:
-          throw new UnsupportedOperationException();
-      }
-
-    }
-
-    @Override
-    public String waitForTableOperation(TInfo tinfo, TCredentials credentials, long opid) throws ThriftSecurityException, ThriftTableOperationException,
-        TException {
-      authenticate(credentials);
-
-      TStatus status = fate.waitForCompletion(opid);
-      if (status == TStatus.FAILED) {
-        Exception e = fate.getException(opid);
-        if (e instanceof ThriftTableOperationException)
-          throw (ThriftTableOperationException) e;
-        if (e instanceof ThriftSecurityException)
-          throw (ThriftSecurityException) e;
-        else if (e instanceof RuntimeException)
-          throw (RuntimeException) e;
-        else
-          throw new RuntimeException(e);
-      }
-
-      String ret = fate.getReturn(opid);
-      if (ret == null)
-        ret = ""; // thrift does not like returning null
-      return ret;
-    }
-
-    @Override
-    public void finishTableOperation(TInfo tinfo, TCredentials credentials, long opid) throws ThriftSecurityException, TException {
-      authenticate(credentials);
-      fate.delete(opid);
-    }
-
-    @Override
-    public long beginNamespaceOperation(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException, TException {
-      return beginTableOperation(tinfo, credentials);
-    }
-
-    @Override
-    public void executeNamespaceOperation(TInfo tinfo, TCredentials c, long opid, org.apache.accumulo.core.master.thrift.TableOperation op,
-        List<ByteBuffer> arguments, Map<String,String> options, boolean autoCleanup) throws ThriftSecurityException, ThriftTableOperationException, TException {
-      authenticate(c);
-
-      switch (op) {
-        case CREATE: {
-          String namespace = ByteBufferUtil.toString(arguments.get(0));
-          if (!security.canCreateNamespace(c, namespace))
-            throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-
-          checkNotSystemNamespace(namespace, TableOperation.CREATE);
-          checkNamespaceName(namespace, TableOperation.CREATE);
-          fate.seedTransaction(opid, new TraceRepo<Master>(new CreateNamespace(c.getPrincipal(), namespace, options)), autoCleanup);
-          break;
-        }
-        case RENAME: {
-
-          String oldName = ByteBufferUtil.toString(arguments.get(0));
-          String newName = ByteBufferUtil.toString(arguments.get(1));
-          String namespaceId = checkNamespaceId(oldName, TableOperation.RENAME);
-
-          checkNotSystemNamespace(oldName, TableOperation.RENAME);
-          checkNotSystemNamespace(newName, TableOperation.RENAME);
-          checkNamespaceName(newName, TableOperation.RENAME);
-          if (!security.canRenameNamespace(c, namespaceId, oldName, newName))
-            throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-
-          fate.seedTransaction(opid, new TraceRepo<Master>(new RenameNamespace(namespaceId, oldName, newName)), autoCleanup);
-          break;
-        }
-        case DELETE: {
-          String namespace = ByteBufferUtil.toString(arguments.get(0));
-          checkNotSystemNamespace(namespace, TableOperation.DELETE);
-          String namespaceId = checkNamespaceId(namespace, TableOperation.DELETE);
-          if (!security.canDeleteNamespace(c, namespaceId))
-            throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-
-          fate.seedTransaction(opid, new TraceRepo<Master>(new DeleteNamespace(namespaceId)), autoCleanup);
-          break;
-        }
-        default:
-          throw new UnsupportedOperationException();
-      }
-
-    }
-
-    @Override
-    public String waitForNamespaceOperation(TInfo tinfo, TCredentials credentials, long opid) throws ThriftSecurityException, ThriftTableOperationException,
-        TException {
-      return waitForTableOperation(tinfo, credentials, opid);
-    }
-
-    @Override
-    public void finishNamespaceOperation(TInfo tinfo, TCredentials credentials, long opid) throws ThriftSecurityException, TException {
-      finishTableOperation(tinfo, credentials, opid);
-    }
-
-    @Override
-    public void setNamespaceProperty(TInfo tinfo, TCredentials credentials, String ns, String property, String value) throws ThriftSecurityException,
-        ThriftTableOperationException, TException {
-      alterNamespaceProperty(credentials, ns, property, value, TableOperation.SET_PROPERTY);
-    }
-
-    @Override
-    public void removeNamespaceProperty(TInfo tinfo, TCredentials credentials, String ns, String property) throws ThriftSecurityException,
-        ThriftTableOperationException, TException {
-      alterNamespaceProperty(credentials, ns, property, null, TableOperation.REMOVE_PROPERTY);
-    }
+    return tserverSet.getConnection(server);
   }
 
   public MergeInfo getMergeInfo(KeyExtent tablet) {
@@ -1289,7 +503,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
     nextEvent.event("Merge state of %s cleared", tableId);
   }
 
-  private void setMasterGoalState(MasterGoalState state) {
+  void setMasterGoalState(MasterGoalState state) {
     try {
       ZooReaderWriter.getInstance().putPersistentData(ZooUtil.getRoot(instance) + Constants.ZMASTER_GOAL_STATE, state.name().getBytes(),
           NodeExistsPolicy.OVERWRITE);
@@ -1309,17 +523,6 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       }
   }
 
-  private void shutdown(boolean stopTabletServers) {
-    if (stopTabletServers) {
-      setMasterGoalState(MasterGoalState.CLEAN_STOP);
-      EventCoordinator.Listener eventListener = nextEvent.getListener();
-      do {
-        waitAround(eventListener);
-      } while (tserverSet.size() > 0);
-    }
-    setMasterState(MasterState.STOP);
-  }
-
   public boolean hasCycled(long time) {
     for (TabletGroupWatcher watcher : watchers) {
       if (watcher.stats.lastScanFinished() < time)
@@ -1720,7 +923,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       watcher.start();
     }
 
-    Processor<Iface> processor = new Processor<Iface>(TraceWrap.service(new MasterClientServiceHandler()));
+    Processor<Iface> processor = new Processor<Iface>(TraceWrap.service(new MasterClientServiceHandler(this)));
     ServerAddress sa = TServerUtils.startServer(getSystemConfiguration(), hostname, Property.MASTER_CLIENTPORT, processor, "Master",
         "Master Client Service Handler", null, Property.MASTER_MINTHREADS, Property.MASTER_THREADCHECK, Property.GENERAL_MAX_MESSAGE_SIZE);
     clientService = sa.server;
@@ -1858,8 +1061,6 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
     }
   }
 
-  static final String I_DONT_KNOW_WHY = "unexpected failure";
-
   @Override
   public void update(LiveTServerSet current, Set<TServerInstance> deleted, Set<TServerInstance> added) {
     DeadServerList obit = new DeadServerList(ZooUtil.getRoot(instance) + Constants.ZDEADTSERVERS);
@@ -1869,7 +1070,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
         obit.delete(up.hostPort());
     }
     for (TServerInstance dead : deleted) {
-      String cause = I_DONT_KNOW_WHY;
+      String cause = "unexpected failure";
       if (serversToShutdown.contains(dead))
         cause = "clean shutdown"; // maybe an incorrect assumption
       if (!getMasterGoalState().equals(MasterGoalState.CLEAN_STOP))
@@ -1972,11 +1173,6 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
     return result;
   }
 
-  public void killTServer(TServerInstance server) {
-    nextEvent.event("Forcing server down %s", server);
-    serversToShutdown.add(server);
-  }
-
   // recovers state from the persistent transaction to shutdown a server
   public void shutdownTServer(TServerInstance server) {
     nextEvent.event("Tablet Server shutdown requested for %s", server);
@@ -2003,10 +1199,6 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
     return this.fs;
   }
 
-  public void updateRecoveryInProgress(String file) {
-    recoveriesInProgress.add(file);
-  }
-
   public void assignedTablet(KeyExtent extent) {
     if (extent.isMeta()) {
       if (getMasterState().equals(MasterState.UNLOAD_ROOT_TABLET)) {


[5/7] ACCUMULO-1965 Fix exception handling for namespaces

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterClientService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterClientService.java b/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterClientService.java
index a43fb03..f6f1e0e 100644
--- a/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterClientService.java
+++ b/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterClientService.java
@@ -48,7 +48,7 @@ import org.slf4j.LoggerFactory;
 
 @SuppressWarnings("all") public class MasterClientService {
 
-  public interface Iface {
+  public interface Iface extends FateService.Iface {
 
     public long initiateFlush(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String tableName) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException;
 
@@ -78,25 +78,9 @@ import org.slf4j.LoggerFactory;
 
     public void reportTabletStatus(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String serverName, TabletLoadState status, org.apache.accumulo.core.data.thrift.TKeyExtent tablet) throws org.apache.thrift.TException;
 
-    public long beginTableOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
-
-    public void executeTableOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException;
-
-    public String waitForTableOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException;
-
-    public void finishTableOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
-
-    public long beginNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
-
-    public void executeNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException;
-
-    public String waitForNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException;
-
-    public void finishNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
-
   }
 
-  public interface AsyncIface {
+  public interface AsyncIface extends FateService .AsyncIface {
 
     public void initiateFlush(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String tableName, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.initiateFlush_call> resultHandler) throws org.apache.thrift.TException;
 
@@ -126,25 +110,9 @@ import org.slf4j.LoggerFactory;
 
     public void reportTabletStatus(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String serverName, TabletLoadState status, org.apache.accumulo.core.data.thrift.TKeyExtent tablet, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.reportTabletStatus_call> resultHandler) throws org.apache.thrift.TException;
 
-    public void beginTableOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.beginTableOperation_call> resultHandler) throws org.apache.thrift.TException;
-
-    public void executeTableOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.executeTableOperation_call> resultHandler) throws org.apache.thrift.TException;
-
-    public void waitForTableOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.waitForTableOperation_call> resultHandler) throws org.apache.thrift.TException;
-
-    public void finishTableOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.finishTableOperation_call> resultHandler) throws org.apache.thrift.TException;
-
-    public void beginNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.beginNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException;
-
-    public void executeNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.executeNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException;
-
-    public void waitForNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.waitForNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException;
-
-    public void finishNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.finishNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException;
-
   }
 
-  public static class Client extends org.apache.thrift.TServiceClient implements Iface {
+  public static class Client extends FateService.Client implements Iface {
     public static class Factory implements org.apache.thrift.TServiceClientFactory<Client> {
       public Factory() {}
       public Client getClient(org.apache.thrift.protocol.TProtocol prot) {
@@ -530,238 +498,8 @@ import org.slf4j.LoggerFactory;
       sendBase("reportTabletStatus", args);
     }
 
-    public long beginTableOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      send_beginTableOperation(tinfo, credentials);
-      return recv_beginTableOperation();
-    }
-
-    public void send_beginTableOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.thrift.TException
-    {
-      beginTableOperation_args args = new beginTableOperation_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      sendBase("beginTableOperation", args);
-    }
-
-    public long recv_beginTableOperation() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      beginTableOperation_result result = new beginTableOperation_result();
-      receiveBase(result, "beginTableOperation");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.sec != null) {
-        throw result.sec;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "beginTableOperation failed: unknown result");
-    }
-
-    public void executeTableOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
-    {
-      send_executeTableOperation(tinfo, credentials, opid, op, arguments, options, autoClean);
-      recv_executeTableOperation();
-    }
-
-    public void send_executeTableOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean) throws org.apache.thrift.TException
-    {
-      executeTableOperation_args args = new executeTableOperation_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setOpid(opid);
-      args.setOp(op);
-      args.setArguments(arguments);
-      args.setOptions(options);
-      args.setAutoClean(autoClean);
-      sendBase("executeTableOperation", args);
-    }
-
-    public void recv_executeTableOperation() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
-    {
-      executeTableOperation_result result = new executeTableOperation_result();
-      receiveBase(result, "executeTableOperation");
-      if (result.sec != null) {
-        throw result.sec;
-      }
-      if (result.tope != null) {
-        throw result.tope;
-      }
-      return;
-    }
-
-    public String waitForTableOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
-    {
-      send_waitForTableOperation(tinfo, credentials, opid);
-      return recv_waitForTableOperation();
-    }
-
-    public void send_waitForTableOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
-    {
-      waitForTableOperation_args args = new waitForTableOperation_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setOpid(opid);
-      sendBase("waitForTableOperation", args);
-    }
-
-    public String recv_waitForTableOperation() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
-    {
-      waitForTableOperation_result result = new waitForTableOperation_result();
-      receiveBase(result, "waitForTableOperation");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.sec != null) {
-        throw result.sec;
-      }
-      if (result.tope != null) {
-        throw result.tope;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "waitForTableOperation failed: unknown result");
-    }
-
-    public void finishTableOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      send_finishTableOperation(tinfo, credentials, opid);
-      recv_finishTableOperation();
-    }
-
-    public void send_finishTableOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
-    {
-      finishTableOperation_args args = new finishTableOperation_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setOpid(opid);
-      sendBase("finishTableOperation", args);
-    }
-
-    public void recv_finishTableOperation() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      finishTableOperation_result result = new finishTableOperation_result();
-      receiveBase(result, "finishTableOperation");
-      if (result.sec != null) {
-        throw result.sec;
-      }
-      return;
-    }
-
-    public long beginNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      send_beginNamespaceOperation(tinfo, credentials);
-      return recv_beginNamespaceOperation();
-    }
-
-    public void send_beginNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.thrift.TException
-    {
-      beginNamespaceOperation_args args = new beginNamespaceOperation_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      sendBase("beginNamespaceOperation", args);
-    }
-
-    public long recv_beginNamespaceOperation() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      beginNamespaceOperation_result result = new beginNamespaceOperation_result();
-      receiveBase(result, "beginNamespaceOperation");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.sec != null) {
-        throw result.sec;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "beginNamespaceOperation failed: unknown result");
-    }
-
-    public void executeNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
-    {
-      send_executeNamespaceOperation(tinfo, credentials, opid, op, arguments, options, autoClean);
-      recv_executeNamespaceOperation();
-    }
-
-    public void send_executeNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean) throws org.apache.thrift.TException
-    {
-      executeNamespaceOperation_args args = new executeNamespaceOperation_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setOpid(opid);
-      args.setOp(op);
-      args.setArguments(arguments);
-      args.setOptions(options);
-      args.setAutoClean(autoClean);
-      sendBase("executeNamespaceOperation", args);
-    }
-
-    public void recv_executeNamespaceOperation() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
-    {
-      executeNamespaceOperation_result result = new executeNamespaceOperation_result();
-      receiveBase(result, "executeNamespaceOperation");
-      if (result.sec != null) {
-        throw result.sec;
-      }
-      if (result.tope != null) {
-        throw result.tope;
-      }
-      return;
-    }
-
-    public String waitForNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
-    {
-      send_waitForNamespaceOperation(tinfo, credentials, opid);
-      return recv_waitForNamespaceOperation();
-    }
-
-    public void send_waitForNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
-    {
-      waitForNamespaceOperation_args args = new waitForNamespaceOperation_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setOpid(opid);
-      sendBase("waitForNamespaceOperation", args);
-    }
-
-    public String recv_waitForNamespaceOperation() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
-    {
-      waitForNamespaceOperation_result result = new waitForNamespaceOperation_result();
-      receiveBase(result, "waitForNamespaceOperation");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.sec != null) {
-        throw result.sec;
-      }
-      if (result.tope != null) {
-        throw result.tope;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "waitForNamespaceOperation failed: unknown result");
-    }
-
-    public void finishNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      send_finishNamespaceOperation(tinfo, credentials, opid);
-      recv_finishNamespaceOperation();
-    }
-
-    public void send_finishNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
-    {
-      finishNamespaceOperation_args args = new finishNamespaceOperation_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setOpid(opid);
-      sendBase("finishNamespaceOperation", args);
-    }
-
-    public void recv_finishNamespaceOperation() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      finishNamespaceOperation_result result = new finishNamespaceOperation_result();
-      receiveBase(result, "finishNamespaceOperation");
-      if (result.sec != null) {
-        throw result.sec;
-      }
-      return;
-    }
-
   }
-  public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
+  public static class AsyncClient extends FateService.AsyncClient implements AsyncIface {
     public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
       private org.apache.thrift.async.TAsyncClientManager clientManager;
       private org.apache.thrift.protocol.TProtocolFactory protocolFactory;
@@ -1350,416 +1088,86 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public void beginTableOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<beginTableOperation_call> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      beginTableOperation_call method_call = new beginTableOperation_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
+  }
+
+  public static class Processor<I extends Iface> extends FateService.Processor<I> implements org.apache.thrift.TProcessor {
+    private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName());
+    public Processor(I iface) {
+      super(iface, getProcessMap(new HashMap<String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
     }
 
-    public static class beginTableOperation_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private org.apache.accumulo.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.security.thrift.TCredentials credentials;
-      public beginTableOperation_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<beginTableOperation_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-        this.tinfo = tinfo;
-        this.credentials = credentials;
+    protected Processor(I iface, Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
+      super(iface, getProcessMap(processMap));
+    }
+
+    private static <I extends Iface> Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> getProcessMap(Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
+      processMap.put("initiateFlush", new initiateFlush());
+      processMap.put("waitForFlush", new waitForFlush());
+      processMap.put("setTableProperty", new setTableProperty());
+      processMap.put("removeTableProperty", new removeTableProperty());
+      processMap.put("setNamespaceProperty", new setNamespaceProperty());
+      processMap.put("removeNamespaceProperty", new removeNamespaceProperty());
+      processMap.put("setMasterGoalState", new setMasterGoalState());
+      processMap.put("shutdown", new shutdown());
+      processMap.put("shutdownTabletServer", new shutdownTabletServer());
+      processMap.put("setSystemProperty", new setSystemProperty());
+      processMap.put("removeSystemProperty", new removeSystemProperty());
+      processMap.put("getMasterStats", new getMasterStats());
+      processMap.put("reportSplitExtent", new reportSplitExtent());
+      processMap.put("reportTabletStatus", new reportTabletStatus());
+      return processMap;
+    }
+
+    public static class initiateFlush<I extends Iface> extends org.apache.thrift.ProcessFunction<I, initiateFlush_args> {
+      public initiateFlush() {
+        super("initiateFlush");
       }
 
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("beginTableOperation", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        beginTableOperation_args args = new beginTableOperation_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.write(prot);
-        prot.writeMessageEnd();
+      public initiateFlush_args getEmptyArgsInstance() {
+        return new initiateFlush_args();
       }
 
-      public long getResult() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException {
-        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new IllegalStateException("Method call not finished!");
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public initiateFlush_result getResult(I iface, initiateFlush_args args) throws org.apache.thrift.TException {
+        initiateFlush_result result = new initiateFlush_result();
+        try {
+          result.success = iface.initiateFlush(args.tinfo, args.credentials, args.tableName);
+          result.setSuccessIsSet(true);
+        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException tope) {
+          result.tope = tope;
         }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_beginTableOperation();
+        return result;
       }
     }
 
-    public void executeTableOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<executeTableOperation_call> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      executeTableOperation_call method_call = new executeTableOperation_call(tinfo, credentials, opid, op, arguments, options, autoClean, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
+    public static class waitForFlush<I extends Iface> extends org.apache.thrift.ProcessFunction<I, waitForFlush_args> {
+      public waitForFlush() {
+        super("waitForFlush");
+      }
 
-    public static class executeTableOperation_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private org.apache.accumulo.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.security.thrift.TCredentials credentials;
-      private long opid;
-      private TableOperation op;
-      private List<ByteBuffer> arguments;
-      private Map<String,String> options;
-      private boolean autoClean;
-      public executeTableOperation_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<executeTableOperation_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-        this.tinfo = tinfo;
-        this.credentials = credentials;
-        this.opid = opid;
-        this.op = op;
-        this.arguments = arguments;
-        this.options = options;
-        this.autoClean = autoClean;
+      public waitForFlush_args getEmptyArgsInstance() {
+        return new waitForFlush_args();
       }
 
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("executeTableOperation", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        executeTableOperation_args args = new executeTableOperation_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setOpid(opid);
-        args.setOp(op);
-        args.setArguments(arguments);
-        args.setOptions(options);
-        args.setAutoClean(autoClean);
-        args.write(prot);
-        prot.writeMessageEnd();
+      protected boolean isOneway() {
+        return false;
       }
 
-      public void getResult() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException {
-        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new IllegalStateException("Method call not finished!");
+      public waitForFlush_result getResult(I iface, waitForFlush_args args) throws org.apache.thrift.TException {
+        waitForFlush_result result = new waitForFlush_result();
+        try {
+          iface.waitForFlush(args.tinfo, args.credentials, args.tableName, args.startRow, args.endRow, args.flushID, args.maxLoops);
+        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException tope) {
+          result.tope = tope;
         }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        (new Client(prot)).recv_executeTableOperation();
-      }
-    }
-
-    public void waitForTableOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<waitForTableOperation_call> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      waitForTableOperation_call method_call = new waitForTableOperation_call(tinfo, credentials, opid, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class waitForTableOperation_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private org.apache.accumulo.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.security.thrift.TCredentials credentials;
-      private long opid;
-      public waitForTableOperation_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<waitForTableOperation_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-        this.tinfo = tinfo;
-        this.credentials = credentials;
-        this.opid = opid;
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("waitForTableOperation", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        waitForTableOperation_args args = new waitForTableOperation_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setOpid(opid);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public String getResult() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException {
-        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new IllegalStateException("Method call not finished!");
-        }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_waitForTableOperation();
-      }
-    }
-
-    public void finishTableOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<finishTableOperation_call> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      finishTableOperation_call method_call = new finishTableOperation_call(tinfo, credentials, opid, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class finishTableOperation_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private org.apache.accumulo.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.security.thrift.TCredentials credentials;
-      private long opid;
-      public finishTableOperation_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<finishTableOperation_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-        this.tinfo = tinfo;
-        this.credentials = credentials;
-        this.opid = opid;
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("finishTableOperation", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        finishTableOperation_args args = new finishTableOperation_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setOpid(opid);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public void getResult() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException {
-        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new IllegalStateException("Method call not finished!");
-        }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        (new Client(prot)).recv_finishTableOperation();
-      }
-    }
-
-    public void beginNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<beginNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      beginNamespaceOperation_call method_call = new beginNamespaceOperation_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class beginNamespaceOperation_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private org.apache.accumulo.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.security.thrift.TCredentials credentials;
-      public beginNamespaceOperation_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<beginNamespaceOperation_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-        this.tinfo = tinfo;
-        this.credentials = credentials;
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("beginNamespaceOperation", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        beginNamespaceOperation_args args = new beginNamespaceOperation_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public long getResult() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException {
-        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new IllegalStateException("Method call not finished!");
-        }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_beginNamespaceOperation();
-      }
-    }
-
-    public void executeNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<executeNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      executeNamespaceOperation_call method_call = new executeNamespaceOperation_call(tinfo, credentials, opid, op, arguments, options, autoClean, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class executeNamespaceOperation_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private org.apache.accumulo.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.security.thrift.TCredentials credentials;
-      private long opid;
-      private TableOperation op;
-      private List<ByteBuffer> arguments;
-      private Map<String,String> options;
-      private boolean autoClean;
-      public executeNamespaceOperation_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<executeNamespaceOperation_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-        this.tinfo = tinfo;
-        this.credentials = credentials;
-        this.opid = opid;
-        this.op = op;
-        this.arguments = arguments;
-        this.options = options;
-        this.autoClean = autoClean;
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("executeNamespaceOperation", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        executeNamespaceOperation_args args = new executeNamespaceOperation_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setOpid(opid);
-        args.setOp(op);
-        args.setArguments(arguments);
-        args.setOptions(options);
-        args.setAutoClean(autoClean);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public void getResult() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException {
-        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new IllegalStateException("Method call not finished!");
-        }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        (new Client(prot)).recv_executeNamespaceOperation();
-      }
-    }
-
-    public void waitForNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<waitForNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      waitForNamespaceOperation_call method_call = new waitForNamespaceOperation_call(tinfo, credentials, opid, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class waitForNamespaceOperation_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private org.apache.accumulo.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.security.thrift.TCredentials credentials;
-      private long opid;
-      public waitForNamespaceOperation_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<waitForNamespaceOperation_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-        this.tinfo = tinfo;
-        this.credentials = credentials;
-        this.opid = opid;
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("waitForNamespaceOperation", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        waitForNamespaceOperation_args args = new waitForNamespaceOperation_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setOpid(opid);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public String getResult() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException {
-        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new IllegalStateException("Method call not finished!");
-        }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_waitForNamespaceOperation();
-      }
-    }
-
-    public void finishNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<finishNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      finishNamespaceOperation_call method_call = new finishNamespaceOperation_call(tinfo, credentials, opid, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class finishNamespaceOperation_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private org.apache.accumulo.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.security.thrift.TCredentials credentials;
-      private long opid;
-      public finishNamespaceOperation_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<finishNamespaceOperation_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-        this.tinfo = tinfo;
-        this.credentials = credentials;
-        this.opid = opid;
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("finishNamespaceOperation", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        finishNamespaceOperation_args args = new finishNamespaceOperation_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setOpid(opid);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public void getResult() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException {
-        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new IllegalStateException("Method call not finished!");
-        }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        (new Client(prot)).recv_finishNamespaceOperation();
-      }
-    }
-
-  }
-
-  public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
-    private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName());
-    public Processor(I iface) {
-      super(iface, getProcessMap(new HashMap<String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
-    }
-
-    protected Processor(I iface, Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
-      super(iface, getProcessMap(processMap));
-    }
-
-    private static <I extends Iface> Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> getProcessMap(Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
-      processMap.put("initiateFlush", new initiateFlush());
-      processMap.put("waitForFlush", new waitForFlush());
-      processMap.put("setTableProperty", new setTableProperty());
-      processMap.put("removeTableProperty", new removeTableProperty());
-      processMap.put("setNamespaceProperty", new setNamespaceProperty());
-      processMap.put("removeNamespaceProperty", new removeNamespaceProperty());
-      processMap.put("setMasterGoalState", new setMasterGoalState());
-      processMap.put("shutdown", new shutdown());
-      processMap.put("shutdownTabletServer", new shutdownTabletServer());
-      processMap.put("setSystemProperty", new setSystemProperty());
-      processMap.put("removeSystemProperty", new removeSystemProperty());
-      processMap.put("getMasterStats", new getMasterStats());
-      processMap.put("reportSplitExtent", new reportSplitExtent());
-      processMap.put("reportTabletStatus", new reportTabletStatus());
-      processMap.put("beginTableOperation", new beginTableOperation());
-      processMap.put("executeTableOperation", new executeTableOperation());
-      processMap.put("waitForTableOperation", new waitForTableOperation());
-      processMap.put("finishTableOperation", new finishTableOperation());
-      processMap.put("beginNamespaceOperation", new beginNamespaceOperation());
-      processMap.put("executeNamespaceOperation", new executeNamespaceOperation());
-      processMap.put("waitForNamespaceOperation", new waitForNamespaceOperation());
-      processMap.put("finishNamespaceOperation", new finishNamespaceOperation());
-      return processMap;
-    }
-
-    public static class initiateFlush<I extends Iface> extends org.apache.thrift.ProcessFunction<I, initiateFlush_args> {
-      public initiateFlush() {
-        super("initiateFlush");
-      }
-
-      public initiateFlush_args getEmptyArgsInstance() {
-        return new initiateFlush_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public initiateFlush_result getResult(I iface, initiateFlush_args args) throws org.apache.thrift.TException {
-        initiateFlush_result result = new initiateFlush_result();
-        try {
-          result.success = iface.initiateFlush(args.tinfo, args.credentials, args.tableName);
-          result.setSuccessIsSet(true);
-        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException tope) {
-          result.tope = tope;
-        }
-        return result;
-      }
-    }
-
-    public static class waitForFlush<I extends Iface> extends org.apache.thrift.ProcessFunction<I, waitForFlush_args> {
-      public waitForFlush() {
-        super("waitForFlush");
-      }
-
-      public waitForFlush_args getEmptyArgsInstance() {
-        return new waitForFlush_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public waitForFlush_result getResult(I iface, waitForFlush_args args) throws org.apache.thrift.TException {
-        waitForFlush_result result = new waitForFlush_result();
-        try {
-          iface.waitForFlush(args.tinfo, args.credentials, args.tableName, args.startRow, args.endRow, args.flushID, args.maxLoops);
-        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException tope) {
-          result.tope = tope;
-        }
-        return result;
+        return result;
       }
     }
 
@@ -2049,212 +1457,10 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public static class beginTableOperation<I extends Iface> extends org.apache.thrift.ProcessFunction<I, beginTableOperation_args> {
-      public beginTableOperation() {
-        super("beginTableOperation");
-      }
-
-      public beginTableOperation_args getEmptyArgsInstance() {
-        return new beginTableOperation_args();
-      }
+  }
 
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public beginTableOperation_result getResult(I iface, beginTableOperation_args args) throws org.apache.thrift.TException {
-        beginTableOperation_result result = new beginTableOperation_result();
-        try {
-          result.success = iface.beginTableOperation(args.tinfo, args.credentials);
-          result.setSuccessIsSet(true);
-        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        }
-        return result;
-      }
-    }
-
-    public static class executeTableOperation<I extends Iface> extends org.apache.thrift.ProcessFunction<I, executeTableOperation_args> {
-      public executeTableOperation() {
-        super("executeTableOperation");
-      }
-
-      public executeTableOperation_args getEmptyArgsInstance() {
-        return new executeTableOperation_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public executeTableOperation_result getResult(I iface, executeTableOperation_args args) throws org.apache.thrift.TException {
-        executeTableOperation_result result = new executeTableOperation_result();
-        try {
-          iface.executeTableOperation(args.tinfo, args.credentials, args.opid, args.op, args.arguments, args.options, args.autoClean);
-        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException tope) {
-          result.tope = tope;
-        }
-        return result;
-      }
-    }
-
-    public static class waitForTableOperation<I extends Iface> extends org.apache.thrift.ProcessFunction<I, waitForTableOperation_args> {
-      public waitForTableOperation() {
-        super("waitForTableOperation");
-      }
-
-      public waitForTableOperation_args getEmptyArgsInstance() {
-        return new waitForTableOperation_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public waitForTableOperation_result getResult(I iface, waitForTableOperation_args args) throws org.apache.thrift.TException {
-        waitForTableOperation_result result = new waitForTableOperation_result();
-        try {
-          result.success = iface.waitForTableOperation(args.tinfo, args.credentials, args.opid);
-        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException tope) {
-          result.tope = tope;
-        }
-        return result;
-      }
-    }
-
-    public static class finishTableOperation<I extends Iface> extends org.apache.thrift.ProcessFunction<I, finishTableOperation_args> {
-      public finishTableOperation() {
-        super("finishTableOperation");
-      }
-
-      public finishTableOperation_args getEmptyArgsInstance() {
-        return new finishTableOperation_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public finishTableOperation_result getResult(I iface, finishTableOperation_args args) throws org.apache.thrift.TException {
-        finishTableOperation_result result = new finishTableOperation_result();
-        try {
-          iface.finishTableOperation(args.tinfo, args.credentials, args.opid);
-        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        }
-        return result;
-      }
-    }
-
-    public static class beginNamespaceOperation<I extends Iface> extends org.apache.thrift.ProcessFunction<I, beginNamespaceOperation_args> {
-      public beginNamespaceOperation() {
-        super("beginNamespaceOperation");
-      }
-
-      public beginNamespaceOperation_args getEmptyArgsInstance() {
-        return new beginNamespaceOperation_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public beginNamespaceOperation_result getResult(I iface, beginNamespaceOperation_args args) throws org.apache.thrift.TException {
-        beginNamespaceOperation_result result = new beginNamespaceOperation_result();
-        try {
-          result.success = iface.beginNamespaceOperation(args.tinfo, args.credentials);
-          result.setSuccessIsSet(true);
-        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        }
-        return result;
-      }
-    }
-
-    public static class executeNamespaceOperation<I extends Iface> extends org.apache.thrift.ProcessFunction<I, executeNamespaceOperation_args> {
-      public executeNamespaceOperation() {
-        super("executeNamespaceOperation");
-      }
-
-      public executeNamespaceOperation_args getEmptyArgsInstance() {
-        return new executeNamespaceOperation_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public executeNamespaceOperation_result getResult(I iface, executeNamespaceOperation_args args) throws org.apache.thrift.TException {
-        executeNamespaceOperation_result result = new executeNamespaceOperation_result();
-        try {
-          iface.executeNamespaceOperation(args.tinfo, args.credentials, args.opid, args.op, args.arguments, args.options, args.autoClean);
-        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException tope) {
-          result.tope = tope;
-        }
-        return result;
-      }
-    }
-
-    public static class waitForNamespaceOperation<I extends Iface> extends org.apache.thrift.ProcessFunction<I, waitForNamespaceOperation_args> {
-      public waitForNamespaceOperation() {
-        super("waitForNamespaceOperation");
-      }
-
-      public waitForNamespaceOperation_args getEmptyArgsInstance() {
-        return new waitForNamespaceOperation_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public waitForNamespaceOperation_result getResult(I iface, waitForNamespaceOperation_args args) throws org.apache.thrift.TException {
-        waitForNamespaceOperation_result result = new waitForNamespaceOperation_result();
-        try {
-          result.success = iface.waitForNamespaceOperation(args.tinfo, args.credentials, args.opid);
-        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException tope) {
-          result.tope = tope;
-        }
-        return result;
-      }
-    }
-
-    public static class finishNamespaceOperation<I extends Iface> extends org.apache.thrift.ProcessFunction<I, finishNamespaceOperation_args> {
-      public finishNamespaceOperation() {
-        super("finishNamespaceOperation");
-      }
-
-      public finishNamespaceOperation_args getEmptyArgsInstance() {
-        return new finishNamespaceOperation_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public finishNamespaceOperation_result getResult(I iface, finishNamespaceOperation_args args) throws org.apache.thrift.TException {
-        finishNamespaceOperation_result result = new finishNamespaceOperation_result();
-        try {
-          iface.finishNamespaceOperation(args.tinfo, args.credentials, args.opid);
-        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        }
-        return result;
-      }
-    }
-
-  }
-
-  public static class initiateFlush_args implements org.apache.thrift.TBase<initiateFlush_args, initiateFlush_args._Fields>, java.io.Serializable, Cloneable   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("initiateFlush_args");
+  public static class initiateFlush_args implements org.apache.thrift.TBase<initiateFlush_args, initiateFlush_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("initiateFlush_args");
 
     private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)3);
     private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)1);
@@ -16690,9014 +15896,4 @@ import org.slf4j.LoggerFactory;
 
   }
 
-  public static class beginTableOperation_args implements org.apache.thrift.TBase<beginTableOperation_args, beginTableOperation_args._Fields>, java.io.Serializable, Cloneable   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("beginTableOperation_args");
-
-    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)2);
-    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-
-    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
-    static {
-      schemes.put(StandardScheme.class, new beginTableOperation_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new beginTableOperation_argsTupleSchemeFactory());
-    }
-
-    public org.apache.accumulo.trace.thrift.TInfo tinfo; // required
-    public org.apache.accumulo.core.security.thrift.TCredentials credentials; // 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 {
-      TINFO((short)2, "tinfo"),
-      CREDENTIALS((short)1, "credentials");
-
-      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 2: // TINFO
-            return TINFO;
-          case 1: // CREDENTIALS
-            return CREDENTIALS;
-          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
-    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.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.trace.thrift.TInfo.class)));
-      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.security.thrift.TCredentials.class)));
-      metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(beginTableOperation_args.class, metaDataMap);
-    }
-
-    public beginTableOperation_args() {
-    }
-
-    public beginTableOperation_args(
-      org.apache.accumulo.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.security.thrift.TCredentials credentials)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public beginTableOperation_args(beginTableOperation_args other) {
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.trace.thrift.TInfo(other.tinfo);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.security.thrift.TCredentials(other.credentials);
-      }
-    }
-
-    public beginTableOperation_args deepCopy() {
-      return new beginTableOperation_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-    }
-
-    public org.apache.accumulo.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public beginTableOperation_args setTinfo(org.apache.accumulo.trace.thrift.TInfo tinfo) {
-      this.tinfo = tinfo;
-      return this;
-    }
-
-    public void unsetTinfo() {
-      this.tinfo = null;
-    }
-
-    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
-    public boolean isSetTinfo() {
-      return this.tinfo != null;
-    }
-
-    public void setTinfoIsSet(boolean value) {
-      if (!value) {
-        this.tinfo = null;
-      }
-    }
-
-    public org.apache.accumulo.core.security.thrift.TCredentials getCredentials() {
-      return this.credentials;
-    }
-
-    public beginTableOperation_args setCredentials(org.apache.accumulo.core.security.thrift.TCredentials credentials) {
-      this.credentials = credentials;
-      return this;
-    }
-
-    public void unsetCredentials() {
-      this.credentials = null;
-    }
-
-    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
-    public boolean isSetCredentials() {
-      return this.credentials != null;
-    }
-
-    public void setCredentialsIsSet(boolean value) {
-      if (!value) {
-        this.credentials = null;
-      }
-    }
-
-    public void setFieldValue(_Fields field, Object value) {
-      switch (field) {
-      case TINFO:
-        if (value == null) {
-          unsetTinfo();
-        } else {
-          setTinfo((org.apache.accumulo.trace.thrift.TInfo)value);
-        }
-        break;
-
-      case CREDENTIALS:
-        if (value == null) {
-          unsetCredentials();
-        } else {
-          setCredentials((org.apache.accumulo.core.security.thrift.TCredentials)value);
-        }
-        break;
-
-      }
-    }
-
-    public Object getFieldValue(_Fields field) {
-      switch (field) {
-      case TINFO:
-        return getTinfo();
-
-      case CREDENTIALS:
-        return getCredentials();
-
-      }
-      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 TINFO:
-        return isSetTinfo();
-      case CREDENTIALS:
-        return isSetCredentials();
-      }
-      throw new IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(Object that) {
-      if (that == null)
-        return false;
-      if (that instanceof beginTableOperation_args)
-        return this.equals((beginTableOperation_args)that);
-      return false;
-    }
-
-    public boolean equals(beginTableOperation_args that) {
-      if (that == null)
-        return false;
-
-      boolean this_present_tinfo = true && this.isSetTinfo();
-      boolean that_present_tinfo = true && that.isSetTinfo();
-      if (this_present_tinfo || that_present_tinfo) {
-        if (!(this_present_tinfo && that_present_tinfo))
-          return false;
-        if (!this.tinfo.equals(that.tinfo))
-          return false;
-      }
-
-      boolean this_present_credentials = true && this.isSetCredentials();
-      boolean that_present_credentials = true && that.isSetCredentials();
-      if (this_present_credentials || that_present_credentials) {
-        if (!(this_present_credentials && that_present_credentials))
-          return false;
-        if (!this.credentials.equals(that.credentials))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      return 0;
-    }
-
-    public int compareTo(beginTableOperation_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-      beginTableOperation_args typedOther = (beginTableOperation_args)other;
-
-      lastComparison = Boolean.valueOf(isSetTinfo()).compareTo(typedOther.isSetTinfo());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTinfo()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, typedOther.tinfo);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = Boolean.valueOf(isSetCredentials()).compareTo(typedOther.isSetCredentials());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetCredentials()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, typedOther.credentials);
-        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("beginTableOperation_args(");
-      boolean first = true;
-
-      sb.append("tinfo:");
-      if (this.tinfo == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tinfo);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("credentials:");
-      if (this.credentials == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.credentials);
-      }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-      if (tinfo != null) {
-        tinfo.validate();
-      }
-      if (credentials != null) {
-        credentials.validate();
-      }
-    }
-
-    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 {
-        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 beginTableOperation_argsStandardSchemeFactory implements SchemeFactory {
-      public beginTableOperation_argsStandardScheme getScheme() {
-        return new beginTableOperation_argsStandardScheme();
-      }
-    }
-
-    private static class beginTableOperation_argsStandardScheme extends StandardScheme<beginTableOperation_args> {
-
-      public void read(org.apache.thrift.protocol.TProtocol iprot, beginTableOperation_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 2: // TINFO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 1: // CREDENTIALS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.credentials = new org.apache.accumulo.core.security.thrift.TCredentials();
-                struct.credentials.read(iprot);
-                struct.setCredentialsIsSet(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, beginTableOperation_args struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.credentials != null) {
-          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
-          struct.credentials.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        if (struct.tinfo != null) {
-          oprot.writeFieldBegin(TINFO_FIELD_DESC);
-          struct.tinfo.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class beginTableOperation_argsTupleSchemeFactory implements SchemeFactory {
-      public beginTableOperation_argsTupleScheme getScheme() {
-        return new beginTableOperation_argsTupleScheme();
-      }
-    }
-
-    private static class beginTableOperation_argsTupleScheme extends TupleScheme<beginTableOperation_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, beginTableOperation_args struct) throws org.apache.thrift.TException {
-        TTupleProtocol oprot = (TTupleProtocol) prot;
-        BitSet optionals = new BitSet();
-        if (struct.isSetTinfo()) {
-          optionals.set(0);
-        }
-        if (struct.isSetCredentials()) {
-          optionals.set(1);
-        }
-        oprot.writeBitSet(optionals, 2);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, beginTableOperation_args struct) throws org.apache.thrift.TException {
-        TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(2);
-        if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.trace.thrift.TInfo();
-          struct.tinfo.read(iprot);
-          struct.setTinfoIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.credentials = new org.apache.accumulo.core.security.thrift.TCredentials();
-          struct.credentials.read(iprot);
-          struct.setCredentialsIsSet(true);
-        }
-      }
-    }
-
-  }
-
-  public static class beginTableOperation_result implements org.apache.thrift.TBase<beginTableOperation_result, beginTableOperation_result._Fields>, java.io.Serializable, Cloneable   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("beginTableOperation_result");
-
-    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I64, (short)0);
-    private static final org.apache.thrift.protocol.TField SEC_FIELD_DESC = new org.apache.thrift.protocol.TField("sec", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-
-    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
-    static {
-      schemes.put(StandardScheme.class, new beginTableOperation_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new beginTableOperation_resultTupleSchemeFactory());
-    }
-
-    public long success; // required
-    public org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec; // 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 {
-      SUCCESS((short)0, "success"),
-      SEC((short)1, "sec");
-
-      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 0: // SUCCESS
-            return SUCCESS;
-          case 1: // SEC
-            return SEC;
-          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 __SUCCESS_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.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
-      tmpMap.put(_Fields.SEC, new org.apache.thrift.meta_data.FieldMetaData("sec", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(beginTableOperation_result.class, metaDataMap);
-    }
-
-    public beginTableOperation_result() {
-    }
-
-    public beginTableOperation_result(
-      long success,
-      org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec)
-    {
-      this();
-      this.success = success;
-      setSuccessIsSet(true);
-      this.sec = sec;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public beginTableOperation_result(beginTableOperation_result other) {
-      __isset_bitfield = other.__isset_bitfield;
-      this.success = other.success;
-      if (other.isSetSec()) {
-        this.sec = new org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException(other.sec);
-      }
-    }
-
-    public beginTableOperation_result deepCopy() {
-      return new beginTableOperation_result(this);
-    }
-
-    @Override
-    public void clear() {
-      setSuccessIsSet(false);
-      this.success = 0;
-      this.sec = null;
-    }
-
-    public long getSuccess() {
-      return this.success;
-    }
-
-    public beginTableOperation_result setSuccess(long success) {
-      this.success = success;
-      setSuccessIsSet(true);
-      return this;
-    }
-
-    public void unsetSuccess() {
-      __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID);
-    }
-
-    /** Returns true if field success is set (has been assigned a value) and false otherwise */
-    public boolean isSetSuccess() {
-      return EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID);
-    }
-
-    public void setSuccessIsSet(boolean value) {
-      __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value);
-    }
-
-    public org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException getSec() {
-      return this.sec;
-    }
-
-    public beginTableOperation_result setSec(org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
-      this.sec = sec;
-      return this;
-    }
-
-    public void unsetSec() {
-      this.sec = null;
-    }
-
-    /** Returns true if field sec is set (has been assigned a value) and false otherwise */
-    public boolean isSetSec() {
-      return this.sec != null;
-    }
-
-    public void setSecIsSet(boolean value) {
-      if (!value) {
-        this.sec = null;
-      }
-    }
-
-    public void setFieldValue(_Fields field, Object value) {
-      switch (field) {
-      case SUCCESS:
-        if (value == null) {
-          unsetSuccess();
-        } else {
-          setSuccess((Long)value);
-        }
-        break;
-
-      case SEC:
-        if (value == null) {
-          unsetSec();
-        } else {
-          setSec((org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException)value);
-        }
-        break;
-
-      }
-    }
-
-    public Object getFieldValue(_Fields field) {
-      switch (field) {
-      case SUCCESS:
-        return Long.valueOf(getSuccess());
-
-      case SEC:
-        return getSec();
-
-      }
-      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 SUCCESS:
-        return isSetSuccess();
-      case SEC:
-        return isSetSec();
-      }
-      throw new IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(Object that) {
-      if (that == null)
-        return false;
-      if (that instanceof beginTableOperation_result)
-        return this.equals((beginTableOperation_result)that);
-      return false;
-    }
-
-    public boolean equals(beginTableOperation_result that) {
-      if (that == null)
-        return false;
-
-      boolean this_present_success = true;
-      boolean that_present_success = true;
-      if (this_present_success || that_present_success) {
-        if (!(this_present_success && that_present_success))
-          return false;
-        if (this.success != that.success)
-          return false;
-      }
-
-      boolean this_present_sec = true && this.isSetSec();
-      boolean that_present_sec = true && that.isSetSec();
-      if (this_present_sec || that_present_sec) {
-        if (!(this_present_sec && that_present_sec))
-          return false;
-        if (!this.sec.equals(that.sec))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      return 0;
-    }
-
-    public int compareTo(beginTableOperation_result other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-      beginTableOperation_result typedOther = (beginTableOperation_result)other;
-
-      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetSuccess()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = Boolean.valueOf(isSetSec()).compareTo(typedOther.isSetSec());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetSec()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sec, typedOther.sec);
-        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("beginTableOperation_result(");
-      boolean first = true;
-
-      sb.append("success:");
-      sb.append(this.success);
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("sec:");
-      if (this.sec == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.sec);
-      }
-      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 beginTableOperation_resultStandardSchemeFactory implements SchemeFactory {
-      public beginTableOperation_resultStandardScheme getScheme() {
-        return new beginTableOperation_resultStandardScheme();
-      }
-    }
-
-    private static class beginTableOperation_resultStandardScheme extends StandardScheme<beginTableOperation_result> {
-
-      public v

<TRUNCATED>

[2/7] ACCUMULO-1965 Fix exception handling for namespaces

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java b/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
new file mode 100644
index 0000000..dc654ae
--- /dev/null
+++ b/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
@@ -0,0 +1,463 @@
+/*
+ * 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.master;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map.Entry;
+import java.util.Set;
+
+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.IsolatedScanner;
+import org.apache.accumulo.core.client.RowIterator;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.client.impl.thrift.TableOperation;
+import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
+import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.data.thrift.TKeyExtent;
+import org.apache.accumulo.core.master.thrift.MasterClientService;
+import org.apache.accumulo.core.master.thrift.MasterGoalState;
+import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
+import org.apache.accumulo.core.master.thrift.MasterState;
+import org.apache.accumulo.core.master.thrift.TableInfo;
+import org.apache.accumulo.core.master.thrift.TabletLoadState;
+import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.master.thrift.TabletSplit;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.core.util.ByteBufferUtil;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
+import org.apache.accumulo.fate.zookeeper.IZooReaderWriter.Mutator;
+import org.apache.accumulo.master.tableOps.TraceRepo;
+import org.apache.accumulo.master.tserverOps.ShutdownTServer;
+import org.apache.accumulo.server.client.ClientServiceHandler;
+import org.apache.accumulo.server.conf.ServerConfiguration;
+import org.apache.accumulo.server.master.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.master.balancer.DefaultLoadBalancer;
+import org.apache.accumulo.server.master.balancer.TabletBalancer;
+import org.apache.accumulo.server.master.state.DeadServerList;
+import org.apache.accumulo.server.master.state.TServerInstance;
+import org.apache.accumulo.server.master.state.TabletServerState;
+import org.apache.accumulo.server.util.DefaultMap;
+import org.apache.accumulo.server.util.NamespacePropUtil;
+import org.apache.accumulo.server.util.SystemPropUtil;
+import org.apache.accumulo.server.util.TableInfoUtil;
+import org.apache.accumulo.server.util.TablePropUtil;
+import org.apache.accumulo.server.util.TabletIterator.TabletDeletedException;
+import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.trace.thrift.TInfo;
+import org.apache.hadoop.io.Text;
+import org.apache.log4j.Logger;
+import org.apache.thrift.TException;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.NoNodeException;
+
+class MasterClientServiceHandler extends FateServiceHandler implements MasterClientService.Iface {
+
+  private static final Logger log = Master.log;
+
+  MasterClientServiceHandler(Master master) {
+    super(master);
+  }
+
+  @Override
+  public long initiateFlush(TInfo tinfo, TCredentials c, String tableId) throws ThriftSecurityException, ThriftTableOperationException {
+    master.security.canFlush(c, tableId);
+
+    String zTablePath = Constants.ZROOT + "/" + master.getConfiguration().getInstance().getInstanceID() + Constants.ZTABLES + "/" + tableId
+        + Constants.ZTABLE_FLUSH_ID;
+
+    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    byte fid[];
+    try {
+      fid = zoo.mutate(zTablePath, null, null, new Mutator() {
+        @Override
+        public byte[] mutate(byte[] currentValue) throws Exception {
+          long flushID = Long.parseLong(new String(currentValue));
+          flushID++;
+          return ("" + flushID).getBytes();
+        }
+      });
+    } catch (NoNodeException nne) {
+      throw new ThriftTableOperationException(tableId, null, TableOperation.FLUSH, TableOperationExceptionType.NOTFOUND, null);
+    } catch (Exception e) {
+      Master.log.warn(e.getMessage(), e);
+      throw new ThriftTableOperationException(tableId, null, TableOperation.FLUSH, TableOperationExceptionType.OTHER, null);
+    }
+    return Long.parseLong(new String(fid));
+  }
+
+  @Override
+  public void waitForFlush(TInfo tinfo, TCredentials c, String tableId, ByteBuffer startRow, ByteBuffer endRow, long flushID, long maxLoops)
+      throws ThriftSecurityException, ThriftTableOperationException {
+    master.security.canFlush(c, tableId);
+
+    if (endRow != null && startRow != null && ByteBufferUtil.toText(startRow).compareTo(ByteBufferUtil.toText(endRow)) >= 0)
+      throw new ThriftTableOperationException(tableId, null, TableOperation.FLUSH, TableOperationExceptionType.BAD_RANGE, "start row must be less than end row");
+
+    Set<TServerInstance> serversToFlush = new HashSet<TServerInstance>(master.tserverSet.getCurrentServers());
+
+    for (long l = 0; l < maxLoops; l++) {
+
+      for (TServerInstance instance : serversToFlush) {
+        try {
+          final TServerConnection server = master.tserverSet.getConnection(instance);
+          if (server != null)
+            server.flush(master.masterLock, tableId, ByteBufferUtil.toBytes(startRow), ByteBufferUtil.toBytes(endRow));
+        } catch (TException ex) {
+          Master.log.error(ex.toString());
+        }
+      }
+
+      if (l == maxLoops - 1)
+        break;
+
+      UtilWaitThread.sleep(50);
+
+      serversToFlush.clear();
+
+      try {
+        Connector conn = master.getConnector();
+        Scanner scanner;
+        if (tableId.equals(MetadataTable.ID)) {
+          scanner = new IsolatedScanner(conn.createScanner(RootTable.NAME, Authorizations.EMPTY));
+          scanner.setRange(MetadataSchema.TabletsSection.getRange());
+        } else {
+          scanner = new IsolatedScanner(conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY));
+          scanner.setRange(new KeyExtent(new Text(tableId), null, ByteBufferUtil.toText(startRow)).toMetadataRange());
+        }
+        TabletsSection.ServerColumnFamily.FLUSH_COLUMN.fetch(scanner);
+        TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.fetch(scanner);
+        scanner.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);
+        scanner.fetchColumnFamily(LogColumnFamily.NAME);
+
+        RowIterator ri = new RowIterator(scanner);
+
+        int tabletsToWaitFor = 0;
+        int tabletCount = 0;
+
+        Text ert = ByteBufferUtil.toText(endRow);
+
+        while (ri.hasNext()) {
+          Iterator<Entry<Key,Value>> row = ri.next();
+          long tabletFlushID = -1;
+          int logs = 0;
+          boolean online = false;
+
+          TServerInstance server = null;
+
+          Entry<Key,Value> entry = null;
+          while (row.hasNext()) {
+            entry = row.next();
+            Key key = entry.getKey();
+
+            if (TabletsSection.ServerColumnFamily.FLUSH_COLUMN.equals(key.getColumnFamily(), key.getColumnQualifier())) {
+              tabletFlushID = Long.parseLong(entry.getValue().toString());
+            }
+
+            if (LogColumnFamily.NAME.equals(key.getColumnFamily()))
+              logs++;
+
+            if (TabletsSection.CurrentLocationColumnFamily.NAME.equals(key.getColumnFamily())) {
+              online = true;
+              server = new TServerInstance(entry.getValue(), key.getColumnQualifier());
+            }
+
+          }
+
+          // when tablet is not online and has no logs, there is no reason to wait for it
+          if ((online || logs > 0) && tabletFlushID < flushID) {
+            tabletsToWaitFor++;
+            if (server != null)
+              serversToFlush.add(server);
+          }
+
+          tabletCount++;
+
+          Text tabletEndRow = new KeyExtent(entry.getKey().getRow(), (Text) null).getEndRow();
+          if (tabletEndRow == null || (ert != null && tabletEndRow.compareTo(ert) >= 0))
+            break;
+        }
+
+        if (tabletsToWaitFor == 0)
+          break;
+
+        // TODO detect case of table offline AND tablets w/ logs? - ACCUMULO-1296
+
+        if (tabletCount == 0 && !Tables.exists(master.getInstance(), tableId))
+          throw new ThriftTableOperationException(tableId, null, TableOperation.FLUSH, TableOperationExceptionType.NOTFOUND, null);
+
+      } catch (AccumuloException e) {
+        Master.log.debug("Failed to scan " + MetadataTable.NAME + " table to wait for flush " + tableId, e);
+      } catch (TabletDeletedException tde) {
+        Master.log.debug("Failed to scan " + MetadataTable.NAME + " table to wait for flush " + tableId, tde);
+      } catch (AccumuloSecurityException e) {
+        Master.log.warn(e.getMessage(), e);
+        throw new ThriftSecurityException();
+      } catch (TableNotFoundException e) {
+        Master.log.error(e.getMessage(), e);
+        throw new ThriftTableOperationException();
+      }
+    }
+
+  }
+
+  @Override
+  public MasterMonitorInfo getMasterStats(TInfo info, TCredentials credentials) throws ThriftSecurityException {
+    final MasterMonitorInfo result = new MasterMonitorInfo();
+
+    result.tServerInfo = new ArrayList<TabletServerStatus>();
+    result.tableMap = new DefaultMap<String,TableInfo>(new TableInfo());
+    for (Entry<TServerInstance,TabletServerStatus> serverEntry : master.tserverStatus.entrySet()) {
+      final TabletServerStatus status = serverEntry.getValue();
+      result.tServerInfo.add(status);
+      for (Entry<String,TableInfo> entry : status.tableMap.entrySet()) {
+        TableInfoUtil.add(result.tableMap.get(entry.getKey()), entry.getValue());
+      }
+    }
+    result.badTServers = new HashMap<String,Byte>();
+    synchronized (master.badServers) {
+      for (TServerInstance bad : master.badServers.keySet()) {
+        result.badTServers.put(bad.hostPort(), TabletServerState.UNRESPONSIVE.getId());
+      }
+    }
+    result.state = master.getMasterState();
+    result.goalState = master.getMasterGoalState();
+    result.unassignedTablets = master.displayUnassigned();
+    result.serversShuttingDown = new HashSet<String>();
+    synchronized (master.serversToShutdown) {
+      for (TServerInstance server : master.serversToShutdown)
+        result.serversShuttingDown.add(server.hostPort());
+    }
+    DeadServerList obit = new DeadServerList(ZooUtil.getRoot(master.getInstance()) + Constants.ZDEADTSERVERS);
+    result.deadTabletServers = obit.getList();
+    return result;
+  }
+
+  @Override
+  public void removeTableProperty(TInfo info, TCredentials credentials, String tableName, String property) throws ThriftSecurityException,
+      ThriftTableOperationException {
+    alterTableProperty(credentials, tableName, property, null, TableOperation.REMOVE_PROPERTY);
+  }
+
+  @Override
+  public void setTableProperty(TInfo info, TCredentials credentials, String tableName, String property, String value) throws ThriftSecurityException,
+      ThriftTableOperationException {
+    alterTableProperty(credentials, tableName, property, value, TableOperation.SET_PROPERTY);
+  }
+
+  @Override
+  public void shutdown(TInfo info, TCredentials c, boolean stopTabletServers) throws ThriftSecurityException {
+    master.security.canPerformSystemActions(c);
+    if (stopTabletServers) {
+      master.setMasterGoalState(MasterGoalState.CLEAN_STOP);
+      EventCoordinator.Listener eventListener = master.nextEvent.getListener();
+      do {
+        eventListener.waitForEvents(Master.ONE_SECOND);
+      } while (master.tserverSet.size() > 0);
+    }
+    master.setMasterState(MasterState.STOP);
+  }
+
+  @Override
+  public void shutdownTabletServer(TInfo info, TCredentials c, String tabletServer, boolean force) throws ThriftSecurityException {
+    master.security.canPerformSystemActions(c);
+
+    final TServerInstance doomed = master.tserverSet.find(tabletServer);
+    if (!force) {
+      final TServerConnection server = master.tserverSet.getConnection(doomed);
+      if (server == null) {
+        Master.log.warn("No server found for name " + tabletServer);
+        return;
+      }
+    }
+
+    long tid = master.fate.startTransaction();
+    master.fate.seedTransaction(tid, new TraceRepo<Master>(new ShutdownTServer(doomed, force)), false);
+    master.fate.waitForCompletion(tid);
+    master.fate.delete(tid);
+  }
+
+  @Override
+  public void reportSplitExtent(TInfo info, TCredentials credentials, String serverName, TabletSplit split) {
+    KeyExtent oldTablet = new KeyExtent(split.oldTablet);
+    if (master.migrations.remove(oldTablet) != null) {
+      Master.log.info("Canceled migration of " + split.oldTablet);
+    }
+    for (TServerInstance instance : master.tserverSet.getCurrentServers()) {
+      if (serverName.equals(instance.hostPort())) {
+        master.nextEvent.event("%s reported split %s, %s", serverName, new KeyExtent(split.newTablets.get(0)), new KeyExtent(split.newTablets.get(1)));
+        return;
+      }
+    }
+    Master.log.warn("Got a split from a server we don't recognize: " + serverName);
+  }
+
+  @Override
+  public void reportTabletStatus(TInfo info, TCredentials credentials, String serverName, TabletLoadState status, TKeyExtent ttablet) {
+    KeyExtent tablet = new KeyExtent(ttablet);
+
+    switch (status) {
+      case LOAD_FAILURE:
+        Master.log.error(serverName + " reports assignment failed for tablet " + tablet);
+        break;
+      case LOADED:
+        master.nextEvent.event("tablet %s was loaded on %s", tablet, serverName);
+        break;
+      case UNLOADED:
+        master.nextEvent.event("tablet %s was unloaded from %s", tablet, serverName);
+        break;
+      case UNLOAD_ERROR:
+        Master.log.error(serverName + " reports unload failed for tablet " + tablet);
+        break;
+      case UNLOAD_FAILURE_NOT_SERVING:
+        if (Master.log.isTraceEnabled()) {
+          Master.log.trace(serverName + " reports unload failed: not serving tablet, could be a split: " + tablet);
+        }
+        break;
+      case CHOPPED:
+        master.nextEvent.event("tablet %s chopped", tablet);
+        break;
+    }
+  }
+
+  @Override
+  public void setMasterGoalState(TInfo info, TCredentials c, MasterGoalState state) throws ThriftSecurityException {
+    master.security.canPerformSystemActions(c);
+
+    master.setMasterGoalState(state);
+  }
+
+  @Override
+  public void removeSystemProperty(TInfo info, TCredentials c, String property) throws ThriftSecurityException {
+    master.security.canPerformSystemActions(c);
+
+    try {
+      SystemPropUtil.removeSystemProperty(property);
+      updatePlugins(property);
+    } catch (Exception e) {
+      Master.log.error("Problem removing config property in zookeeper", e);
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  @Override
+  public void setSystemProperty(TInfo info, TCredentials c, String property, String value) throws ThriftSecurityException, TException {
+    master.security.canPerformSystemActions(c);
+
+    try {
+      SystemPropUtil.setSystemProperty(property, value);
+      updatePlugins(property);
+    } catch (Exception e) {
+      Master.log.error("Problem setting config property in zookeeper", e);
+      throw new TException(e.getMessage());
+    }
+  }
+
+  @Override
+  public void setNamespaceProperty(TInfo tinfo, TCredentials credentials, String ns, String property, String value) throws ThriftSecurityException,
+      ThriftTableOperationException {
+    alterNamespaceProperty(credentials, ns, property, value, TableOperation.SET_PROPERTY);
+  }
+
+  @Override
+  public void removeNamespaceProperty(TInfo tinfo, TCredentials credentials, String ns, String property) throws ThriftSecurityException,
+      ThriftTableOperationException {
+    alterNamespaceProperty(credentials, ns, property, null, TableOperation.REMOVE_PROPERTY);
+  }
+
+  private void alterNamespaceProperty(TCredentials c, String namespace, String property, String value, TableOperation op) throws ThriftSecurityException,
+      ThriftTableOperationException {
+
+    String namespaceId = null;
+    namespaceId = checkNamespaceId(namespace, op);
+
+    if (!master.security.canAlterNamespace(c, namespaceId))
+      throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
+    try {
+      if (value == null) {
+        NamespacePropUtil.removeNamespaceProperty(namespaceId, property);
+      } else {
+        NamespacePropUtil.setNamespaceProperty(namespaceId, property, value);
+      }
+    } catch (KeeperException.NoNodeException e) {
+      // race condition... namespace no longer exists? This call will throw an exception if the namespace was deleted:
+      checkNamespaceId(namespaceId, op);
+      log.info("Error altering namespace property", e);
+      throw new ThriftTableOperationException(namespaceId, namespace, op, TableOperationExceptionType.OTHER, "Problem altering namespaceproperty");
+    } catch (Exception e) {
+      log.error("Problem altering namespace property", e);
+      throw new ThriftTableOperationException(namespaceId, namespace, op, TableOperationExceptionType.OTHER, "Problem altering namespace property");
+    }
+  }
+
+  private void alterTableProperty(TCredentials c, String tableName, String property, String value, TableOperation op) throws ThriftSecurityException,
+      ThriftTableOperationException {
+    final String tableId = ClientServiceHandler.checkTableId(master.getInstance(), tableName, op);
+    if (!master.security.canAlterTable(c, tableId))
+      throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
+    try {
+      if (value == null || value.isEmpty()) {
+        TablePropUtil.removeTableProperty(tableId, property);
+      } else if (!TablePropUtil.setTableProperty(tableId, property, value)) {
+        throw new Exception("Invalid table property.");
+      }
+    } catch (KeeperException.NoNodeException e) {
+      // race condition... table no longer exists? This call will throw an exception if the table was deleted:
+      ClientServiceHandler.checkTableId(master.getInstance(), tableName, op);
+      log.info("Error altering table property", e);
+      throw new ThriftTableOperationException(tableId, tableName, op, TableOperationExceptionType.OTHER, "Problem altering table property");
+    } catch (Exception e) {
+      log.error("Problem altering table property", e);
+      throw new ThriftTableOperationException(tableId, tableName, op, TableOperationExceptionType.OTHER, "Problem altering table property");
+    }
+  }
+
+  private void updatePlugins(String property) {
+    if (property.equals(Property.MASTER_TABLET_BALANCER.getKey())) {
+      TabletBalancer balancer = ServerConfiguration.getSystemConfiguration(master.getInstance()).instantiateClassProperty(Property.MASTER_TABLET_BALANCER,
+          TabletBalancer.class, new DefaultLoadBalancer());
+      balancer.init(master.getConfiguration());
+      master.tabletBalancer = balancer;
+      log.info("tablet balancer changed to " + master.tabletBalancer.getClass().getName());
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTable.java
index fae2c3b..cba1492 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTable.java
@@ -25,7 +25,9 @@ import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.TableOperation;
+import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.fate.Repo;
@@ -210,7 +212,12 @@ class ClonePermissions extends MasterRepo {
     // setup permissions in zookeeper before table info in zookeeper
     // this way concurrent users will not get a spurious pemission denied
     // error
-    return new CloneZookeeper(cloneInfo);
+    try {
+      return new CloneZookeeper(cloneInfo);
+    } catch (NamespaceNotFoundException e) {
+      throw new ThriftTableOperationException(null, cloneInfo.tableName, TableOperation.CLONE, TableOperationExceptionType.NAMESPACE_NOTFOUND,
+          "Namespace for target table not found");
+    }
   }
 
   @Override
@@ -237,7 +244,6 @@ public class CloneTable extends MasterRepo {
 
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-
     long val = Utils.reserveNamespace(cloneInfo.srcNamespaceId, tid, false, true, TableOperation.CLONE);
     val += Utils.reserveTable(cloneInfo.srcTableId, tid, false, true, TableOperation.CLONE);
     return val;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameNamespace.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameNamespace.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameNamespace.java
index 30adef1..41f24cd 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameNamespace.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameNamespace.java
@@ -68,7 +68,8 @@ public class RenameNamespace extends MasterRepo {
           if (currentName.equals(newName))
             return null; // assume in this case the operation is running again, so we are done
           if (!currentName.equals(oldName)) {
-            throw new ThriftTableOperationException(null, oldName, TableOperation.RENAME, TableOperationExceptionType.NOTFOUND, "Name changed while processing");
+            throw new ThriftTableOperationException(null, oldName, TableOperation.RENAME, TableOperationExceptionType.NAMESPACE_NOTFOUND,
+                "Name changed while processing");
           }
           return newName.getBytes();
         }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
index eee05aa..7e3a6f9 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
@@ -64,7 +64,8 @@ public class RenameTable extends MasterRepo {
 
     // ensure no attempt is made to rename across namespaces
     if (newTableName.contains(".") && !namespaceId.equals(Namespaces.getNamespaceId(instance, qualifiedNewTableName.getFirst())))
-      throw new IllegalArgumentException("Namespace in new table name does not match the old table name");
+      throw new ThriftTableOperationException(tableId, oldTableName, TableOperation.RENAME, TableOperationExceptionType.INVALID_NAME,
+          "Namespace in new table name does not match the old table name");
 
     IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java
index 9ff1b6a..564d939 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java
@@ -104,7 +104,7 @@ public class Utils {
         Instance instance = HdfsZooInstance.getInstance();
         IZooReaderWriter zk = ZooReaderWriter.getRetryingInstance();
         if (!zk.exists(ZooUtil.getRoot(instance) + Constants.ZNAMESPACES + "/" + namespaceId))
-          throw new ThriftTableOperationException(namespaceId, "", op, TableOperationExceptionType.NOTFOUND, "Namespace does not exist");
+          throw new ThriftTableOperationException(namespaceId, "", op, TableOperationExceptionType.NAMESPACE_NOTFOUND, "Namespace does not exist");
       }
       log.info("namespace " + namespaceId + " (" + Long.toHexString(id) + ") locked for " + (writeLock ? "write" : "read") + " operation: " + op);
       return 0;
@@ -155,6 +155,6 @@ public class Utils {
     String n = Namespaces.getNameToIdMap(instance).get(namespace);
 
     if (n != null && !n.equals(namespaceId))
-      throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.EXISTS, null);
+      throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.NAMESPACE_EXISTS, null);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
index 74526fa..402f139 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
@@ -147,7 +147,7 @@ public class Config extends Test {
         } catch (AccumuloException ex) {
           if (ex.getCause() instanceof ThriftTableOperationException) {
             ThriftTableOperationException ttoe = (ThriftTableOperationException) ex.getCause();
-            if (ttoe.type == TableOperationExceptionType.NOTFOUND)
+            if (ttoe.type == TableOperationExceptionType.NAMESPACE_NOTFOUND)
               return;
           }
           throw ex;
@@ -215,7 +215,7 @@ public class Config extends Test {
     } catch (AccumuloException ex) {
       if (ex.getCause() instanceof ThriftTableOperationException) {
         ThriftTableOperationException ttoe = (ThriftTableOperationException) ex.getCause();
-        if (ttoe.type == TableOperationExceptionType.NOTFOUND)
+        if (ttoe.type == TableOperationExceptionType.NAMESPACE_NOTFOUND)
           return;
       }
       throw ex;


[4/7] ACCUMULO-1965 Fix exception handling for namespaces

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/master/thrift/TableOperation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/master/thrift/TableOperation.java b/core/src/main/java/org/apache/accumulo/core/master/thrift/TableOperation.java
deleted file mode 100644
index bafddcd..0000000
--- a/core/src/main/java/org/apache/accumulo/core/master/thrift/TableOperation.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * 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.master.thrift;
-
-
-import java.util.Map;
-import java.util.HashMap;
-import org.apache.thrift.TEnum;
-
-@SuppressWarnings("all") public enum TableOperation implements org.apache.thrift.TEnum {
-  CREATE(0),
-  CLONE(1),
-  DELETE(2),
-  RENAME(3),
-  ONLINE(4),
-  OFFLINE(5),
-  MERGE(6),
-  DELETE_RANGE(7),
-  BULK_IMPORT(8),
-  COMPACT(9),
-  IMPORT(10),
-  EXPORT(11),
-  COMPACT_CANCEL(12);
-
-  private final int value;
-
-  private TableOperation(int value) {
-    this.value = value;
-  }
-
-  /**
-   * Get the integer value of this enum value, as defined in the Thrift IDL.
-   */
-  public int getValue() {
-    return value;
-  }
-
-  /**
-   * Find a the enum type by its integer value, as defined in the Thrift IDL.
-   * @return null if the value is not found.
-   */
-  public static TableOperation findByValue(int value) { 
-    switch (value) {
-      case 0:
-        return CREATE;
-      case 1:
-        return CLONE;
-      case 2:
-        return DELETE;
-      case 3:
-        return RENAME;
-      case 4:
-        return ONLINE;
-      case 5:
-        return OFFLINE;
-      case 6:
-        return MERGE;
-      case 7:
-        return DELETE_RANGE;
-      case 8:
-        return BULK_IMPORT;
-      case 9:
-        return COMPACT;
-      case 10:
-        return IMPORT;
-      case 11:
-        return EXPORT;
-      case 12:
-        return COMPACT_CANCEL;
-      default:
-        return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java
index c31cb0d..56966d3 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java
@@ -16,13 +16,13 @@
  */
 package org.apache.accumulo.core.metadata;
 
-import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.impl.Namespaces;
 
 public class MetadataTable {
 
   public static final String OLD_NAME = "!METADATA";
 
   public static final String ID = "!0";
-  public static final String NAME = Constants.ACCUMULO_NAMESPACE + ".metadata";
+  public static final String NAME = Namespaces.ACCUMULO_NAMESPACE + ".metadata";
 
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
index 527f242..85219eb 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
@@ -16,7 +16,7 @@
  */
 package org.apache.accumulo.core.metadata;
 
-import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.hadoop.io.Text;
 
@@ -26,7 +26,7 @@ import org.apache.hadoop.io.Text;
 public class RootTable {
 
   public static final String ID = "+r";
-  public static final String NAME = Constants.ACCUMULO_NAMESPACE + ".root";
+  public static final String NAME = Namespaces.ACCUMULO_NAMESPACE + ".root";
 
   /**
    * DFS location relative to the Accumulo directory

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java
index cbbc105..7f41c32 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java
@@ -22,9 +22,9 @@ import java.util.Map;
 import java.util.Map.Entry;
 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.impl.Namespaces;
 import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
 import org.apache.commons.cli.CommandLine;
@@ -47,7 +47,7 @@ public class NamespacesCommand extends Command {
       @Override
       public String apply(Map.Entry<String,String> entry) {
         String name = entry.getKey();
-        if (Constants.DEFAULT_NAMESPACE.equals(name))
+        if (Namespaces.DEFAULT_NAMESPACE.equals(name))
           name = DEFAULT_NAMESPACE_DISPLAY_NAME;
         String id = entry.getValue();
         if (cl.hasOption(namespaceIdOption.getOpt()))

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/thrift/client.thrift
----------------------------------------------------------------------
diff --git a/core/src/main/thrift/client.thrift b/core/src/main/thrift/client.thrift
index 601a7b2..38a8076 100644
--- a/core/src/main/thrift/client.thrift
+++ b/core/src/main/thrift/client.thrift
@@ -21,128 +21,131 @@ include "security.thrift"
 include "trace.thrift"
 
 enum TableOperation {
-    CREATE,
-    DELETE,
-    RENAME,
-    SET_PROPERTY,
-    REMOVE_PROPERTY,
-    OFFLINE,
-    ONLINE,
-    FLUSH,
-    PERMISSION,
-    CLONE,
-    MERGE,
-    DELETE_RANGE,
-    BULK_IMPORT,
-    COMPACT
-    IMPORT
-    EXPORT
-    COMPACT_CANCEL
+  CREATE
+  DELETE
+  RENAME
+  SET_PROPERTY
+  REMOVE_PROPERTY
+  OFFLINE
+  ONLINE
+  FLUSH
+  PERMISSION
+  CLONE
+  MERGE
+  DELETE_RANGE
+  BULK_IMPORT
+  COMPACT
+  IMPORT
+  EXPORT
+  COMPACT_CANCEL
 }
 
 enum TableOperationExceptionType {
-    EXISTS,
-    NOTFOUND,
-    OFFLINE,
-    BULK_BAD_INPUT_DIRECTORY,
-    BULK_BAD_ERROR_DIRECTORY,
-    BAD_RANGE,
-    OTHER
+  EXISTS
+  NOTFOUND
+  OFFLINE
+  BULK_BAD_INPUT_DIRECTORY
+  BULK_BAD_ERROR_DIRECTORY
+  BAD_RANGE
+  OTHER
+  NAMESPACE_EXISTS
+  NAMESPACE_NOTFOUND
+  INVALID_NAME
 }
 
 enum ConfigurationType {
-    CURRENT,
-    SITE,
-    DEFAULT
+  CURRENT
+  SITE
+  DEFAULT
 }
 
 enum SecurityErrorCode {
-    DEFAULT_SECURITY_ERROR = 0,
-    BAD_CREDENTIALS = 1,
-    PERMISSION_DENIED = 2,
-    USER_DOESNT_EXIST = 3,
-    CONNECTION_ERROR = 4,
-    USER_EXISTS = 5,
-    GRANT_INVALID = 6,
-    BAD_AUTHORIZATIONS = 7,
-    INVALID_INSTANCEID = 8,
-    TABLE_DOESNT_EXIST = 9,
-    UNSUPPORTED_OPERATION = 10,
-    INVALID_TOKEN = 11,
-    AUTHENTICATOR_FAILED = 12,
-    AUTHORIZOR_FAILED = 13,
-    PERMISSIONHANDLER_FAILED = 14,
-    TOKEN_EXPIRED = 15,
-    SERIALIZATION_ERROR = 16,
-    INSUFFICIENT_PROPERTIES = 17,
-    NAMESPACE_DOESNT_EXIST = 18;
+  DEFAULT_SECURITY_ERROR = 0
+  BAD_CREDENTIALS = 1
+  PERMISSION_DENIED = 2
+  USER_DOESNT_EXIST = 3
+  CONNECTION_ERROR = 4
+  USER_EXISTS = 5
+  GRANT_INVALID = 6
+  BAD_AUTHORIZATIONS = 7
+  INVALID_INSTANCEID = 8
+  TABLE_DOESNT_EXIST = 9
+  UNSUPPORTED_OPERATION = 10
+  INVALID_TOKEN = 11
+  AUTHENTICATOR_FAILED = 12
+  AUTHORIZOR_FAILED = 13
+  PERMISSIONHANDLER_FAILED = 14
+  TOKEN_EXPIRED = 15
+  SERIALIZATION_ERROR = 16
+  INSUFFICIENT_PROPERTIES = 17
+  NAMESPACE_DOESNT_EXIST = 18;
 }
 
 exception ThriftSecurityException {
-    1:string user,
-    2:SecurityErrorCode code
+  1:string user
+  2:SecurityErrorCode code
 }
 
 exception ThriftTableOperationException {
-    1:string tableId,
-    2:string tableName,
-    3:TableOperation op,
-    4:TableOperationExceptionType type,
-    5:string description
+  1:string tableId
+  2:string tableName
+  3:TableOperation op
+  4:TableOperationExceptionType type
+  5:string description
 }
 
 struct TDiskUsage {
-    1:list<string> tables
-    2:i64 usage
+  1:list<string> tables
+  2:i64 usage
 }
 
 service ClientService {
 
-    // system management methods
-    string getRootTabletLocation()
-    string getInstanceId()
-    string getZooKeepers()
-    
-    list<string> bulkImportFiles(1:trace.TInfo tinfo, 8:security.TCredentials credentials, 3:i64 tid, 4:string tableId, 5:list<string> files, 6:string errorDir, 7:bool setTime) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope);
-    // ensures that nobody is working on the transaction id above
-    bool isActive(1:trace.TInfo tinfo, 2:i64 tid),
-
-    void ping(2:security.TCredentials credentials) throws (1:ThriftSecurityException sec)
-
-    list<TDiskUsage> getDiskUsage(2:set<string> tables, 1:security.TCredentials credentials) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException toe)
-
-    // user management methods
-    set<string> listLocalUsers(2:trace.TInfo tinfo, 3:security.TCredentials credentials) throws (1:ThriftSecurityException sec)
-    void createLocalUser(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:binary password) throws (1:ThriftSecurityException sec)
-    void dropLocalUser(3:trace.TInfo tinfo, 4:security.TCredentials credentials, 2:string principal) throws (1:ThriftSecurityException sec)
-    void changeLocalUserPassword(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:binary password) throws (1:ThriftSecurityException sec)
-
-    // authentication-related methods
-    bool authenticate(1:trace.TInfo tinfo, 2:security.TCredentials credentials) throws (1:ThriftSecurityException sec)
-    bool authenticateUser(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:security.TCredentials toAuth) throws (1:ThriftSecurityException sec)
-
-    // authorization-related methods
-    void changeAuthorizations(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:list<binary> authorizations) throws (1:ThriftSecurityException sec)
-    list<binary> getUserAuthorizations(3:trace.TInfo tinfo, 4:security.TCredentials credentials, 2:string principal) throws (1:ThriftSecurityException sec)
-
-    // permissions-related methods
-    bool hasSystemPermission(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:byte sysPerm) throws (1:ThriftSecurityException sec)
-    bool hasTablePermission(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:string tableName, 4:byte tblPerm) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
-    bool hasNamespacePermission(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string principal, 4:string ns, 5:byte tblNspcPerm) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
-    void grantSystemPermission(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:byte permission) throws (1:ThriftSecurityException sec)
-    void revokeSystemPermission(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:byte permission) throws (1:ThriftSecurityException sec)
-    void grantTablePermission(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:string tableName, 4:byte permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
-    void revokeTablePermission(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:string tableName, 4:byte permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
-    void grantNamespacePermission(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string principal, 4:string ns, 5:byte permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
-    void revokeNamespacePermission(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string principal, 4:string ns, 5:byte permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
-
-    // configuration methods
-    map<string, string> getConfiguration(2:trace.TInfo tinfo, 3:security.TCredentials credentials, 1:ConfigurationType type);
-    map<string, string> getTableConfiguration(1:trace.TInfo tinfo, 3:security.TCredentials credentials, 2:string tableName) throws (1:ThriftTableOperationException tope);
-    map<string, string> getNamespaceConfiguration(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string ns) throws (1:ThriftTableOperationException tope);
-    bool checkClass(1:trace.TInfo tinfo, 4:security.TCredentials credentials, 2:string className, 3:string interfaceMatch);
-    bool checkTableClass(1:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string tableId, 3:string className, 4:string interfaceMatch) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope);
-    bool checkNamespaceClass(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string namespaceId, 4:string className, 5:string interfaceMatch) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope);
+  // system management methods
+  string getRootTabletLocation()
+  string getInstanceId()
+  string getZooKeepers()
+
+  list<string> bulkImportFiles(1:trace.TInfo tinfo, 8:security.TCredentials credentials, 3:i64 tid, 4:string tableId, 5:list<string> files, 6:string errorDir, 7:bool setTime) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope);
+  // ensures that nobody is working on the transaction id above
+  bool isActive(1:trace.TInfo tinfo, 2:i64 tid)
+
+  void ping(2:security.TCredentials credentials) throws (1:ThriftSecurityException sec)
+
+  list<TDiskUsage> getDiskUsage(2:set<string> tables, 1:security.TCredentials credentials) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException toe)
+
+  // user management methods
+  set<string> listLocalUsers(2:trace.TInfo tinfo, 3:security.TCredentials credentials) throws (1:ThriftSecurityException sec)
+  void createLocalUser(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:binary password) throws (1:ThriftSecurityException sec)
+  void dropLocalUser(3:trace.TInfo tinfo, 4:security.TCredentials credentials, 2:string principal) throws (1:ThriftSecurityException sec)
+  void changeLocalUserPassword(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:binary password) throws (1:ThriftSecurityException sec)
+
+  // authentication-related methods
+  bool authenticate(1:trace.TInfo tinfo, 2:security.TCredentials credentials) throws (1:ThriftSecurityException sec)
+  bool authenticateUser(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:security.TCredentials toAuth) throws (1:ThriftSecurityException sec)
+
+  // authorization-related methods
+  void changeAuthorizations(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:list<binary> authorizations) throws (1:ThriftSecurityException sec)
+  list<binary> getUserAuthorizations(3:trace.TInfo tinfo, 4:security.TCredentials credentials, 2:string principal) throws (1:ThriftSecurityException sec)
+
+  // permissions-related methods
+  bool hasSystemPermission(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:byte sysPerm) throws (1:ThriftSecurityException sec)
+  bool hasTablePermission(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:string tableName, 4:byte tblPerm) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+  bool hasNamespacePermission(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string principal, 4:string ns, 5:byte tblNspcPerm) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+  void grantSystemPermission(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:byte permission) throws (1:ThriftSecurityException sec)
+  void revokeSystemPermission(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:byte permission) throws (1:ThriftSecurityException sec)
+  void grantTablePermission(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:string tableName, 4:byte permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+  void revokeTablePermission(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:string tableName, 4:byte permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+  void grantNamespacePermission(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string principal, 4:string ns, 5:byte permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+  void revokeNamespacePermission(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string principal, 4:string ns, 5:byte permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+
+  // configuration methods
+  map<string, string> getConfiguration(2:trace.TInfo tinfo, 3:security.TCredentials credentials, 1:ConfigurationType type);
+  map<string, string> getTableConfiguration(1:trace.TInfo tinfo, 3:security.TCredentials credentials, 2:string tableName) throws (1:ThriftTableOperationException tope);
+  map<string, string> getNamespaceConfiguration(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string ns) throws (1:ThriftTableOperationException tope);
+  bool checkClass(1:trace.TInfo tinfo, 4:security.TCredentials credentials, 2:string className, 3:string interfaceMatch);
+  bool checkTableClass(1:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string tableId, 3:string className, 4:string interfaceMatch) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope);
+  bool checkNamespaceClass(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string namespaceId, 4:string className, 5:string interfaceMatch) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope);
 }
 
 // Only used for a unit test

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/thrift/master.thrift
----------------------------------------------------------------------
diff --git a/core/src/main/thrift/master.thrift b/core/src/main/thrift/master.thrift
index 1ec5c33..38e9227 100644
--- a/core/src/main/thrift/master.thrift
+++ b/core/src/main/thrift/master.thrift
@@ -23,148 +23,150 @@ include "client.thrift"
 include "trace.thrift"
 
 struct Compacting {
-    1:i32 running
-    2:i32 queued
+  1:i32 running
+  2:i32 queued
 }
 
 struct TableInfo {
-    1:i64 recs
-    2:i64 recsInMemory
-    3:i32 tablets
-    4:i32 onlineTablets
-    5:double ingestRate
-    6:double ingestByteRate
-    7:double queryRate
-    8:double queryByteRate
-    9:Compacting minors;
-    10:Compacting majors;
-    11:Compacting scans;
-    12:double scanRate;
+  1:i64 recs
+  2:i64 recsInMemory
+  3:i32 tablets
+  4:i32 onlineTablets
+  5:double ingestRate
+  6:double ingestByteRate
+  7:double queryRate
+  8:double queryByteRate
+  9:Compacting minors;
+  10:Compacting majors;
+  11:Compacting scans;
+  12:double scanRate;
 }
 
 struct RecoveryStatus {
-    2:string name
-    5:i32 runtime                   // in millis
-    6:double progress
+  2:string name
+  5:i32 runtime                   // in millis
+  6:double progress
 }
 
 struct TabletServerStatus {
-    1:map<string, TableInfo> tableMap
-    2:i64 lastContact
-    3:string name
-    5:double osLoad
-    7:i64 holdTime
-    8:i64 lookups
-    10:i64 indexCacheHits    
-    11:i64 indexCacheRequest   
-    12:i64 dataCacheHits   
-    13:i64 dataCacheRequest
-    14:list<RecoveryStatus> logSorts
+  1:map<string, TableInfo> tableMap
+  2:i64 lastContact
+  3:string name
+  5:double osLoad
+  7:i64 holdTime
+  8:i64 lookups
+  10:i64 indexCacheHits
+  11:i64 indexCacheRequest
+  12:i64 dataCacheHits
+  13:i64 dataCacheRequest
+  14:list<RecoveryStatus> logSorts
 }
 
 enum MasterState {
-    INITIAL,
-    HAVE_LOCK,
-    SAFE_MODE,
-    NORMAL, 
-    UNLOAD_METADATA_TABLETS, 
-    UNLOAD_ROOT_TABLET, 
-    STOP
+  INITIAL
+  HAVE_LOCK
+  SAFE_MODE
+  NORMAL
+  UNLOAD_METADATA_TABLETS
+  UNLOAD_ROOT_TABLET
+  STOP
 }
 
 enum MasterGoalState {
-    CLEAN_STOP,
-    SAFE_MODE,
-    NORMAL,
+  CLEAN_STOP
+  SAFE_MODE
+  NORMAL
 }
 
 struct DeadServer {
-    1:string server,
-    2:i64 lastStatus,
-    3:string status,
+  1:string server
+  2:i64 lastStatus
+  3:string status
 }
 
 struct MasterMonitorInfo {
-    1:map<string, TableInfo> tableMap
-    2:list<TabletServerStatus> tServerInfo
-    3:map<string, byte> badTServers
-    6:MasterState state
-    8:MasterGoalState goalState
-    7:i32 unassignedTablets
-    9:set<string> serversShuttingDown
-    10:list<DeadServer> deadTabletServers
+  1:map<string, TableInfo> tableMap
+  2:list<TabletServerStatus> tServerInfo
+  3:map<string, byte> badTServers
+  6:MasterState state
+  8:MasterGoalState goalState
+  7:i32 unassignedTablets
+  9:set<string> serversShuttingDown
+  10:list<DeadServer> deadTabletServers
 }
 
 struct TabletSplit {
-    1:data.TKeyExtent oldTablet
-    2:list<data.TKeyExtent> newTablets
+  1:data.TKeyExtent oldTablet
+  2:list<data.TKeyExtent> newTablets
 }
 
 exception RecoveryException {
-    1:string why
+  1:string why
 }
 
 enum TabletLoadState {
-    LOADED,
-    LOAD_FAILURE,
-    UNLOADED,
-    UNLOAD_FAILURE_NOT_SERVING,
-    UNLOAD_ERROR,
-    CHOPPED
+  LOADED
+  LOAD_FAILURE
+  UNLOADED
+  UNLOAD_FAILURE_NOT_SERVING
+  UNLOAD_ERROR
+  CHOPPED
 }
 
-enum TableOperation {
-  CREATE
-  CLONE
-  DELETE
-  RENAME
-  ONLINE
-  OFFLINE
-  MERGE
-  DELETE_RANGE
-  BULK_IMPORT
-  COMPACT
-  IMPORT
-  EXPORT
-  COMPACT_CANCEL
+enum FateOperation {
+  TABLE_CREATE
+  TABLE_CLONE
+  TABLE_DELETE
+  TABLE_RENAME
+  TABLE_ONLINE
+  TABLE_OFFLINE
+  TABLE_MERGE
+  TABLE_DELETE_RANGE
+  TABLE_BULK_IMPORT
+  TABLE_COMPACT
+  TABLE_IMPORT
+  TABLE_EXPORT
+  TABLE_CANCEL_COMPACT
+  NAMESPACE_CREATE
+  NAMESPACE_DELETE
+  NAMESPACE_RENAME
 }
 
-service MasterClientService {
-
-    // table management methods
-    i64 initiateFlush(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string tableName) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
-    void waitForFlush(5:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string tableName, 6:binary startRow, 7:binary endRow, 3:i64 flushID, 4:i64 maxLoops) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
-    
-    void setTableProperty(5:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string tableName, 3:string property, 4:string value) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
-    void removeTableProperty(4:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string tableName, 3:string property) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
-    
-    void setNamespaceProperty(5:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string ns, 3:string property, 4:string value) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
-    void removeNamespaceProperty(4:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string ns, 3:string property) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
-
-    // system management methods
-    void setMasterGoalState(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:MasterGoalState state) throws (1:client.ThriftSecurityException sec);
-    void shutdown(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:bool stopTabletServers) throws (1:client.ThriftSecurityException sec)
-    void shutdownTabletServer(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string tabletServer, 4:bool force) throws (1: client.ThriftSecurityException sec)
-    void setSystemProperty(4:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string property, 3:string value) throws (1:client.ThriftSecurityException sec)
-    void removeSystemProperty(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string property) throws (1:client.ThriftSecurityException sec)
-
-    // system monitoring methods
-    MasterMonitorInfo getMasterStats(2:trace.TInfo tinfo, 1:security.TCredentials credentials) throws (1:client.ThriftSecurityException sec)
-    
-    // tablet server reporting
-    oneway void reportSplitExtent(4:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string serverName, 3:TabletSplit split)
-    oneway void reportTabletStatus(5:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string serverName, 3:TabletLoadState status, 4:data.TKeyExtent tablet)
-
-   //table operations
-   i64 beginTableOperation(2:trace.TInfo tinfo, 1:security.TCredentials credentials) throws (1:client.ThriftSecurityException sec)
-   void executeTableOperation(7:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:i64 opid, 3:TableOperation op, 4:list<binary> arguments, 5:map<string, string> options, 6:bool autoClean) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
-   string waitForTableOperation(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:i64 opid) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
-   void finishTableOperation(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:i64 opid) throws (1:client.ThriftSecurityException sec)
-
-   //namespace operations
-   i64 beginNamespaceOperation(1:trace.TInfo tinfo, 2:security.TCredentials credentials) throws (1:client.ThriftSecurityException sec)
-   void executeNamespaceOperation(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:i64 opid, 4:TableOperation op, 5:list<binary> arguments, 6:map<string, string> options, 7:bool autoClean) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
-   string waitForNamespaceOperation(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:i64 opid) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
-   void finishNamespaceOperation(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:i64 opid) throws (1:client.ThriftSecurityException sec)
-   
+service FateService {
+  // register a fate operation by reserving an opid
+  i64 beginFateOperation(2:trace.TInfo tinfo, 1:security.TCredentials credentials) throws (1:client.ThriftSecurityException sec)
+  // initiate execution of the fate operation; set autoClean to true if not waiting for completion
+  void executeFateOperation(7:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:i64 opid, 3:FateOperation op, 4:list<binary> arguments, 5:map<string, string> options, 6:bool autoClean) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
+  // wait for completion of the operation and get the returned exception, if any
+  string waitForFateOperation(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:i64 opid) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
+  // clean up fate operation if autoClean was not set, after waiting
+  void finishFateOperation(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:i64 opid) throws (1:client.ThriftSecurityException sec)
+}
+
+service MasterClientService extends FateService {
+
+  // table management methods
+  i64 initiateFlush(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string tableName) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
+  void waitForFlush(5:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string tableName, 6:binary startRow, 7:binary endRow, 3:i64 flushID, 4:i64 maxLoops) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
+
+  void setTableProperty(5:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string tableName, 3:string property, 4:string value) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
+  void removeTableProperty(4:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string tableName, 3:string property) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
+
+  void setNamespaceProperty(5:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string ns, 3:string property, 4:string value) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
+  void removeNamespaceProperty(4:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string ns, 3:string property) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
+
+  // system management methods
+  void setMasterGoalState(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:MasterGoalState state) throws (1:client.ThriftSecurityException sec);
+  void shutdown(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:bool stopTabletServers) throws (1:client.ThriftSecurityException sec)
+  void shutdownTabletServer(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string tabletServer, 4:bool force) throws (1: client.ThriftSecurityException sec)
+  void setSystemProperty(4:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string property, 3:string value) throws (1:client.ThriftSecurityException sec)
+  void removeSystemProperty(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string property) throws (1:client.ThriftSecurityException sec)
+
+  // system monitoring methods
+  MasterMonitorInfo getMasterStats(2:trace.TInfo tinfo, 1:security.TCredentials credentials) throws (1:client.ThriftSecurityException sec)
+
+  // tablet server reporting
+  oneway void reportSplitExtent(4:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string serverName, 3:TabletSplit split)
+  oneway void reportTabletStatus(5:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string serverName, 3:TabletLoadState status, 4:data.TKeyExtent tablet)
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java b/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java
index 50a9fbe..009be17 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java
@@ -39,6 +39,7 @@ import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.NamespaceOperations;
+import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
@@ -66,7 +67,7 @@ public class MockNamespacesTest {
     Instance instance = new MockInstance("default");
     Connector c = instance.getConnector("user", new PasswordToken("pass"));
 
-    assertTrue(c.namespaceOperations().exists(Constants.DEFAULT_NAMESPACE));
+    assertTrue(c.namespaceOperations().exists(Namespaces.DEFAULT_NAMESPACE));
     c.tableOperations().create(tableName);
     assertTrue(c.tableOperations().exists(tableName));
   }
@@ -170,7 +171,7 @@ public class MockNamespacesTest {
     propVal = "13K";
     String tableName = "some_table";
     c.tableOperations().create(tableName);
-    c.namespaceOperations().setProperty(Constants.DEFAULT_NAMESPACE, propKey, propVal);
+    c.namespaceOperations().setProperty(Namespaces.DEFAULT_NAMESPACE, propKey, propVal);
 
     assertTrue(checkTableHasProp(c, tableName, propKey, propVal));
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java b/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java
index de24881..216b3ba 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java
@@ -61,43 +61,43 @@ import org.junit.Assert;
 import org.junit.Test;
 
 public class MockTableOperationsTest {
-  
+
   @Test
   public void testCreateUseVersions() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException {
     Instance instance = new MockInstance("topstest");
     Connector conn = instance.getConnector("user", new PasswordToken("pass"));
     String t = "tableName1";
-    
+
     {
       conn.tableOperations().create(t, false, TimeType.LOGICAL);
-      
+
       writeVersionable(conn, t, 3);
       assertVersionable(conn, t, 3);
-      
+
       IteratorSetting settings = new IteratorSetting(20, VersioningIterator.class);
       conn.tableOperations().attachIterator(t, settings);
-      
+
       assertVersionable(conn, t, 1);
-      
+
       conn.tableOperations().delete(t);
     }
-    
+
     {
       conn.tableOperations().create(t, true, TimeType.MILLIS);
-      
+
       try {
         IteratorSetting settings = new IteratorSetting(20, VersioningIterator.class);
         conn.tableOperations().attachIterator(t, settings);
         Assert.fail();
       } catch (AccumuloException ex) {}
-      
+
       writeVersionable(conn, t, 3);
       assertVersionable(conn, t, 1);
-      
+
       conn.tableOperations().delete(t);
     }
   }
-  
+
   protected void writeVersionable(Connector c, String tableName, int size) throws TableNotFoundException, MutationsRejectedException {
     for (int i = 0; i < size; i++) {
       BatchWriter w = c.createBatchWriter(tableName, new BatchWriterConfig());
@@ -107,7 +107,7 @@ public class MockTableOperationsTest {
       w.close();
     }
   }
-  
+
   protected void assertVersionable(Connector c, String tableName, int size) throws TableNotFoundException {
     BatchScanner s = c.createBatchScanner(tableName, Authorizations.EMPTY, 1);
     s.setRanges(Collections.singleton(Range.exact("row1", "cf", "cq")));
@@ -117,23 +117,24 @@ public class MockTableOperationsTest {
       Assert.assertEquals("cf", e.getKey().getColumnFamily().toString());
       Assert.assertEquals("cq", e.getKey().getColumnQualifier().toString());
       count++;
-      
+
     }
     Assert.assertEquals(size, count);
     s.close();
   }
-  
+
   @Test
   public void testTableNotFound() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException {
     Instance instance = new MockInstance("topstest");
     Connector conn = instance.getConnector("user", new PasswordToken("pass"));
+    IteratorSetting setting = new IteratorSetting(100, "myvers", VersioningIterator.class);
     String t = "tableName";
     try {
-      conn.tableOperations().attachIterator(t, null);
+      conn.tableOperations().attachIterator(t, setting);
       Assert.fail();
     } catch (TableNotFoundException e) {}
     try {
-      conn.tableOperations().checkIteratorConflicts(t, null, EnumSet.allOf(IteratorScope.class));
+      conn.tableOperations().checkIteratorConflicts(t, setting, EnumSet.allOf(IteratorScope.class));
       Assert.fail();
     } catch (TableNotFoundException e) {}
     try {
@@ -141,7 +142,7 @@ public class MockTableOperationsTest {
       Assert.fail();
     } catch (TableNotFoundException e) {}
     try {
-      conn.tableOperations().getIteratorSetting(t, null, null);
+      conn.tableOperations().getIteratorSetting(t, "myvers", IteratorScope.scan);
       Assert.fail();
     } catch (TableNotFoundException e) {}
     try {
@@ -174,13 +175,13 @@ public class MockTableOperationsTest {
       Assert.fail();
     } catch (TableExistsException e) {}
   }
-  
+
   private static class ImportTestFilesAndData {
     Path importPath;
     Path failurePath;
     List<Pair<Key,Value>> keyVals;
   }
-  
+
   @Test
   public void testImport() throws Throwable {
     ImportTestFilesAndData dataAndFiles = prepareTestFiles();
@@ -200,7 +201,7 @@ public class MockTableOperationsTest {
     }
     Assert.assertFalse(iterator.hasNext());
   }
-  
+
   private ImportTestFilesAndData prepareTestFiles() throws Throwable {
     Configuration defaultConf = new Configuration();
     Path tempFile = new Path("target/accumulo-test/import/sample.rf");
@@ -228,7 +229,7 @@ public class MockTableOperationsTest {
     files.keyVals = keyVals;
     return files;
   }
-  
+
   @Test(expected = TableNotFoundException.class)
   public void testFailsWithNoTable() throws Throwable {
     Instance instance = new MockInstance("foo");
@@ -237,7 +238,7 @@ public class MockTableOperationsTest {
     ImportTestFilesAndData testFiles = prepareTestFiles();
     tableOperations.importDirectory("doesnt_exist_table", testFiles.importPath.toString(), testFiles.failurePath.toString(), false);
   }
-  
+
   @Test(expected = IOException.class)
   public void testFailsWithNonEmptyFailureDirectory() throws Throwable {
     Instance instance = new MockInstance("foo");
@@ -248,7 +249,7 @@ public class MockTableOperationsTest {
     fs.open(testFiles.failurePath.suffix("/something")).close();
     tableOperations.importDirectory("doesnt_exist_table", testFiles.importPath.toString(), testFiles.failurePath.toString(), false);
   }
-  
+
   @Test
   public void testDeleteRows() throws Exception {
     Instance instance = new MockInstance("rows");
@@ -270,5 +271,5 @@ public class MockTableOperationsTest {
       Assert.assertTrue(entry.getKey().getRow().toString().charAt(0) != '1');
     }
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
index d522582..4df7c88 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
@@ -34,6 +34,7 @@ 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.NamespaceNotFoundException;
+import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.ClientService;
@@ -79,16 +80,16 @@ public class ClientServiceHandler implements ClientService.Iface {
     this.fs = fs;
   }
 
-  protected String checkTableId(String tableName, TableOperation operation) throws ThriftTableOperationException {
-    String tableId = Tables.getNameToIdMap(instance).get(tableName);
-    if (tableId == null) {
-      // maybe the table exist, but the cache was not updated yet... so try to clear the cache and check again
-      Tables.clearCache(instance);
-      tableId = Tables.getNameToIdMap(instance).get(tableName);
-      if (tableId == null)
-        throw new ThriftTableOperationException(null, tableName, operation, TableOperationExceptionType.NOTFOUND, null);
+  public static String checkTableId(Instance instance, String tableName, TableOperation operation) throws ThriftTableOperationException {
+    TableOperationExceptionType reason = null;
+    try {
+      return Tables._getTableId(instance, tableName);
+    } catch (NamespaceNotFoundException e) {
+      reason = TableOperationExceptionType.NAMESPACE_NOTFOUND;
+    } catch (TableNotFoundException e) {
+      reason = TableOperationExceptionType.NOTFOUND;
     }
-    return tableId;
+    throw new ThriftTableOperationException(null, tableName, operation, reason, null);
   }
 
   protected String checkNamespaceId(String namespace, TableOperation operation) throws ThriftTableOperationException {
@@ -98,7 +99,7 @@ public class ClientServiceHandler implements ClientService.Iface {
       Tables.clearCache(instance);
       namespaceId = Namespaces.getNameToIdMap(instance).get(namespace);
       if (namespaceId == null)
-        throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.NOTFOUND, null);
+        throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.NAMESPACE_NOTFOUND, null);
     }
     return namespaceId;
   }
@@ -181,7 +182,7 @@ public class ClientServiceHandler implements ClientService.Iface {
   @Override
   public void grantTablePermission(TInfo tinfo, TCredentials credentials, String user, String tableName, byte permission) throws ThriftSecurityException,
       ThriftTableOperationException {
-    String tableId = checkTableId(tableName, TableOperation.PERMISSION);
+    String tableId = checkTableId(instance, tableName, TableOperation.PERMISSION);
     security.grantTablePermission(credentials, user, tableId, TablePermission.getPermissionById(permission));
   }
 
@@ -200,7 +201,7 @@ public class ClientServiceHandler implements ClientService.Iface {
   @Override
   public void revokeTablePermission(TInfo tinfo, TCredentials credentials, String user, String tableName, byte permission) throws ThriftSecurityException,
       ThriftTableOperationException {
-    String tableId = checkTableId(tableName, TableOperation.PERMISSION);
+    String tableId = checkTableId(instance, tableName, TableOperation.PERMISSION);
     security.revokeTablePermission(credentials, user, tableId, TablePermission.getPermissionById(permission));
   }
 
@@ -212,7 +213,7 @@ public class ClientServiceHandler implements ClientService.Iface {
   @Override
   public boolean hasTablePermission(TInfo tinfo, TCredentials credentials, String user, String tableName, byte tblPerm) throws ThriftSecurityException,
       ThriftTableOperationException {
-    String tableId = checkTableId(tableName, TableOperation.PERMISSION);
+    String tableId = checkTableId(instance, tableName, TableOperation.PERMISSION);
     return security.hasTablePermission(credentials, user, tableId, TablePermission.getPermissionById(tblPerm));
   }
 
@@ -263,8 +264,8 @@ public class ClientServiceHandler implements ClientService.Iface {
 
   @Override
   public Map<String,String> getTableConfiguration(TInfo tinfo, TCredentials credentials, String tableName) throws TException, ThriftTableOperationException {
-    String tableId = checkTableId(tableName, null);
-    AccumuloConfiguration config = new ServerConfiguration(instance).getTableConfiguration(tableId);
+    String tableId = checkTableId(instance, tableName, null);
+    AccumuloConfiguration config = ServerConfiguration.getTableConfiguration(instance, tableId);
     return conf(credentials, config);
   }
 
@@ -326,7 +327,7 @@ public class ClientServiceHandler implements ClientService.Iface {
 
     security.authenticateUser(credentials, credentials);
 
-    String tableId = checkTableId(tableName, null);
+    String tableId = checkTableId(instance, tableName, null);
 
     ClassLoader loader = getClass().getClassLoader();
     Class<?> shouldMatch;
@@ -398,7 +399,7 @@ public class ClientServiceHandler implements ClientService.Iface {
 
       for (String table : tables) {
         // ensure that table table exists
-        String tableId = checkTableId(table, null);
+        String tableId = checkTableId(instance, table, null);
         tableIds.add(tableId);
         if (!security.canScan(credentials, tableId))
           throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
@@ -428,7 +429,7 @@ public class ClientServiceHandler implements ClientService.Iface {
       namespaceId = Namespaces.getNamespaceId(instance, ns);
     } catch (NamespaceNotFoundException e) {
       String why = "Could not find namespace while getting configuration.";
-      throw new ThriftTableOperationException(null, ns, null, TableOperationExceptionType.NOTFOUND, why);
+      throw new ThriftTableOperationException(null, ns, null, TableOperationExceptionType.NAMESPACE_NOTFOUND, why);
     }
     AccumuloConfiguration config = ServerConfiguration.getNamespaceConfiguration(instance, namespaceId);
     return conf(credentials, config);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
index 131426e..2a1f9a1 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
@@ -25,6 +25,7 @@ import java.util.Set;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationObserver;
 import org.apache.accumulo.core.conf.Property;
@@ -57,7 +58,7 @@ public class NamespaceConfiguration extends AccumuloConfiguration {
     if (value == null || !property.getType().isValidFormat(value)) {
       if (value != null)
         log.error("Using default value for " + key + " due to improperly formatted " + property.getType() + ": " + value);
-      if (!(namespaceId.equals(Constants.ACCUMULO_NAMESPACE_ID) && isIteratorOrConstraint(property.getKey()))) {
+      if (!(namespaceId.equals(Namespaces.ACCUMULO_NAMESPACE_ID) && isIteratorOrConstraint(property.getKey()))) {
         // ignore iterators from parent if system namespace
         value = parent.get(property);
       }
@@ -108,7 +109,7 @@ public class NamespaceConfiguration extends AccumuloConfiguration {
 
     // exclude system iterators/constraints from the system namespace
     // so they don't affect the metadata or root tables.
-    if (this.namespaceId.equals(Constants.ACCUMULO_NAMESPACE_ID))
+    if (getNamespaceId().equals(Namespaces.ACCUMULO_NAMESPACE_ID))
       parentFilter = new SystemNamespaceFilter(filter);
 
     parent.getProperties(props, parentFilter);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
index 7f5e44e..ce1547e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
@@ -29,6 +29,7 @@ import jline.console.ConsoleReader;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.cli.Help;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -395,10 +396,10 @@ public class Initialize {
     zoo.putPersistentData(zkInstanceRoot, new byte[0], NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZTABLES, Constants.ZTABLES_INITIAL_ID, NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZNAMESPACES, new byte[0], NodeExistsPolicy.FAIL);
-    TableManager.prepareNewNamespaceState(uuid, Constants.DEFAULT_NAMESPACE_ID, Constants.DEFAULT_NAMESPACE, NodeExistsPolicy.FAIL);
-    TableManager.prepareNewNamespaceState(uuid, Constants.ACCUMULO_NAMESPACE_ID, Constants.ACCUMULO_NAMESPACE, NodeExistsPolicy.FAIL);
-    TableManager.prepareNewTableState(uuid, RootTable.ID, Constants.ACCUMULO_NAMESPACE_ID, RootTable.NAME, TableState.ONLINE, NodeExistsPolicy.FAIL);
-    TableManager.prepareNewTableState(uuid, MetadataTable.ID, Constants.ACCUMULO_NAMESPACE_ID, MetadataTable.NAME, TableState.ONLINE, NodeExistsPolicy.FAIL);
+    TableManager.prepareNewNamespaceState(uuid, Namespaces.DEFAULT_NAMESPACE_ID, Namespaces.DEFAULT_NAMESPACE, NodeExistsPolicy.FAIL);
+    TableManager.prepareNewNamespaceState(uuid, Namespaces.ACCUMULO_NAMESPACE_ID, Namespaces.ACCUMULO_NAMESPACE, NodeExistsPolicy.FAIL);
+    TableManager.prepareNewTableState(uuid, RootTable.ID, Namespaces.ACCUMULO_NAMESPACE_ID, RootTable.NAME, TableState.ONLINE, NodeExistsPolicy.FAIL);
+    TableManager.prepareNewTableState(uuid, MetadataTable.ID, Namespaces.ACCUMULO_NAMESPACE_ID, MetadataTable.NAME, TableState.ONLINE, NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZTSERVERS, new byte[0], NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZPROBLEMS, new byte[0], NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + RootTable.ZROOT_TABLET, new byte[0], NodeExistsPolicy.FAIL);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/server/base/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java b/server/base/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java
index 6b81539..a9cdb15 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java
@@ -59,29 +59,29 @@ import org.apache.zookeeper.data.Stat;
 import com.google.common.net.HostAndPort;
 
 public class LiveTServerSet implements Watcher {
-  
+
   public interface Listener {
     void update(LiveTServerSet current, Set<TServerInstance> deleted, Set<TServerInstance> added);
   }
-  
+
   private static final Logger log = Logger.getLogger(LiveTServerSet.class);
-  
+
   private final Listener cback;
   private final Instance instance;
   private final AccumuloConfiguration conf;
   private ZooCache zooCache;
-  
+
   public class TServerConnection {
     private final HostAndPort address;
-    
+
     public TServerConnection(HostAndPort addr) throws TException {
       address = addr;
     }
-    
+
     private String lockString(ZooLock mlock) {
       return mlock.getLockID().serialize(ZooUtil.getRoot(instance) + Constants.ZMASTER_LOCK);
     }
-    
+
     public void assignTablet(ZooLock lock, KeyExtent extent) throws TException {
       TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
       try {
@@ -90,7 +90,7 @@ public class LiveTServerSet implements Watcher {
         ThriftUtil.returnClient(client);
       }
     }
-    
+
     public void unloadTablet(ZooLock lock, KeyExtent extent, boolean save) throws TException {
       TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
       try {
@@ -99,14 +99,14 @@ public class LiveTServerSet implements Watcher {
         ThriftUtil.returnClient(client);
       }
     }
-    
+
     public TabletServerStatus getTableMap(boolean usePooledConnection) throws TException, ThriftSecurityException {
-      
+
       if (usePooledConnection == true)
         throw new UnsupportedOperationException();
-      
+
       TTransport transport = ThriftUtil.createTransport(address, conf);
-      
+
       try {
         TabletClientService.Client client = ThriftUtil.createClient(new TabletClientService.Client.Factory(), transport);
         return client.getTabletServerStatus(Tracer.traceInfo(), SystemCredentials.get().toThrift(instance));
@@ -115,7 +115,7 @@ public class LiveTServerSet implements Watcher {
           transport.close();
       }
     }
-    
+
     public void halt(ZooLock lock) throws TException, ThriftSecurityException {
       TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
       try {
@@ -124,7 +124,7 @@ public class LiveTServerSet implements Watcher {
         ThriftUtil.returnClient(client);
       }
     }
-    
+
     public void fastHalt(ZooLock lock) throws TException {
       TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
       try {
@@ -133,7 +133,7 @@ public class LiveTServerSet implements Watcher {
         ThriftUtil.returnClient(client);
       }
     }
-    
+
     public void flush(ZooLock lock, String tableId, byte[] startRow, byte[] endRow) throws TException {
       TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
       try {
@@ -143,7 +143,7 @@ public class LiveTServerSet implements Watcher {
         ThriftUtil.returnClient(client);
       }
     }
-    
+
     public void chop(ZooLock lock, KeyExtent extent) throws TException {
       TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
       try {
@@ -152,7 +152,7 @@ public class LiveTServerSet implements Watcher {
         ThriftUtil.returnClient(client);
       }
     }
-    
+
     public void splitTablet(ZooLock lock, KeyExtent extent, Text splitPoint) throws TException, ThriftSecurityException, NotServingTabletException {
       TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
       try {
@@ -162,7 +162,7 @@ public class LiveTServerSet implements Watcher {
         ThriftUtil.returnClient(client);
       }
     }
-    
+
     public void flushTablet(ZooLock lock, KeyExtent extent) throws TException {
       TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
       try {
@@ -171,7 +171,7 @@ public class LiveTServerSet implements Watcher {
         ThriftUtil.returnClient(client);
       }
     }
-    
+
     public void compact(ZooLock lock, String tableId, byte[] startRow, byte[] endRow) throws TException {
       TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
       try {
@@ -181,7 +181,7 @@ public class LiveTServerSet implements Watcher {
         ThriftUtil.returnClient(client);
       }
     }
-    
+
     public boolean isActive(long tid) throws TException {
       TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, conf);
       try {
@@ -190,40 +190,40 @@ public class LiveTServerSet implements Watcher {
         ThriftUtil.returnClient(client);
       }
     }
-    
+
   }
-  
+
   static class TServerInfo {
     TServerConnection connection;
     TServerInstance instance;
-    
+
     TServerInfo(TServerInstance instance, TServerConnection connection) {
       this.connection = connection;
       this.instance = instance;
     }
   };
-  
+
   // The set of active tservers with locks, indexed by their name in zookeeper
   private Map<String,TServerInfo> current = new HashMap<String,TServerInfo>();
   // as above, indexed by TServerInstance
   private Map<TServerInstance,TServerInfo> currentInstances = new HashMap<TServerInstance,TServerInfo>();
-  
+
   // The set of entries in zookeeper without locks, and the first time each was noticed
   private Map<String,Long> locklessServers = new HashMap<String,Long>();
-  
+
   public LiveTServerSet(Instance instance, AccumuloConfiguration conf, Listener cback) {
     this.cback = cback;
     this.instance = instance;
     this.conf = conf;
-    
+
   }
-  
+
   public synchronized ZooCache getZooCache() {
     if (zooCache == null)
       zooCache = new ZooCache(this);
     return zooCache;
   }
-  
+
   public synchronized void startListeningForTabletServerChanges() {
     scanServers();
     SimpleTimer.getInstance().schedule(new Runnable() {
@@ -233,23 +233,23 @@ public class LiveTServerSet implements Watcher {
       }
     }, 0, 5000);
   }
-  
+
   public synchronized void scanServers() {
     try {
       final Set<TServerInstance> updates = new HashSet<TServerInstance>();
       final Set<TServerInstance> doomed = new HashSet<TServerInstance>();
-      
+
       final String path = ZooUtil.getRoot(instance) + Constants.ZTSERVERS;
-      
+
       HashSet<String> all = new HashSet<String>(current.keySet());
       all.addAll(getZooCache().getChildren(path));
-      
+
       locklessServers.keySet().retainAll(all);
-      
+
       for (String zPath : all) {
         checkServer(updates, doomed, path, zPath);
       }
-      
+
       // log.debug("Current: " + current.keySet());
       if (!doomed.isEmpty() || !updates.isEmpty())
         this.cback.update(this, doomed, updates);
@@ -257,7 +257,7 @@ public class LiveTServerSet implements Watcher {
       log.error(ex, ex);
     }
   }
-  
+
   private void deleteServerNode(String serverNode) throws InterruptedException, KeeperException {
     try {
       ZooReaderWriter.getInstance().delete(serverNode, -1);
@@ -267,23 +267,23 @@ public class LiveTServerSet implements Watcher {
       // someone else deleted it
     }
   }
-  
+
   private synchronized void checkServer(final Set<TServerInstance> updates, final Set<TServerInstance> doomed, final String path, final String zPath)
       throws TException, InterruptedException, KeeperException {
-    
+
     TServerInfo info = current.get(zPath);
-    
+
     final String lockPath = path + "/" + zPath;
     Stat stat = new Stat();
     byte[] lockData = ZooLock.getLockData(getZooCache(), lockPath, stat);
-    
+
     if (lockData == null) {
       if (info != null) {
         doomed.add(info.instance);
         current.remove(zPath);
         currentInstances.remove(info.instance);
       }
-      
+
       Long firstSeen = locklessServers.get(zPath);
       if (firstSeen == null) {
         locklessServers.put(zPath, System.currentTimeMillis());
@@ -296,7 +296,7 @@ public class LiveTServerSet implements Watcher {
       ServerServices services = new ServerServices(new String(lockData));
       HostAndPort client = services.getAddress(ServerServices.Service.TSERV_CLIENT);
       TServerInstance instance = new TServerInstance(client, stat.getEphemeralOwner());
-      
+
       if (info == null) {
         updates.add(instance);
         TServerInfo tServerInfo = new TServerInfo(instance, new TServerConnection(client));
@@ -311,29 +311,29 @@ public class LiveTServerSet implements Watcher {
       }
     }
   }
-  
+
   @Override
   public void process(WatchedEvent event) {
-    
+
     // its important that these event are propagated by ZooCache, because this ensures when reading zoocache that is has already processed the event and cleared
     // relevant nodes before code below reads from zoocache
-    
+
     if (event.getPath() != null) {
       if (event.getPath().endsWith(Constants.ZTSERVERS)) {
         scanServers();
       } else if (event.getPath().contains(Constants.ZTSERVERS)) {
         int pos = event.getPath().lastIndexOf('/');
-        
+
         // do only if ZTSERVER is parent
         if (pos >= 0 && event.getPath().substring(0, pos).endsWith(Constants.ZTSERVERS)) {
-          
+
           String server = event.getPath().substring(pos + 1);
-          
+
           final Set<TServerInstance> updates = new HashSet<TServerInstance>();
           final Set<TServerInstance> doomed = new HashSet<TServerInstance>();
-          
+
           final String path = ZooUtil.getRoot(instance) + Constants.ZTSERVERS;
-          
+
           try {
             checkServer(updates, doomed, path, server);
             if (!doomed.isEmpty() || !updates.isEmpty())
@@ -345,8 +345,8 @@ public class LiveTServerSet implements Watcher {
       }
     }
   }
-  
-  public synchronized TServerConnection getConnection(TServerInstance server) throws TException {
+
+  public synchronized TServerConnection getConnection(TServerInstance server) {
     if (server == null)
       return null;
     TServerInfo tServerInfo = currentInstances.get(server);
@@ -354,15 +354,15 @@ public class LiveTServerSet implements Watcher {
       return null;
     return tServerInfo.connection;
   }
-  
+
   public synchronized Set<TServerInstance> getCurrentServers() {
     return new HashSet<TServerInstance>(currentInstances.keySet());
   }
-  
+
   public synchronized int size() {
     return current.size();
   }
-  
+
   public synchronized TServerInstance find(String tabletServer) {
     HostAndPort addr = AddressUtil.parseAddress(tabletServer, false);
     for (Entry<String,TServerInfo> entry : current.entrySet()) {
@@ -371,7 +371,7 @@ public class LiveTServerSet implements Watcher {
     }
     return null;
   }
-  
+
   public synchronized void remove(TServerInstance server) {
     String zPath = null;
     for (Entry<String,TServerInfo> entry : current.entrySet()) {
@@ -384,7 +384,7 @@ public class LiveTServerSet implements Watcher {
       return;
     current.remove(zPath);
     currentInstances.remove(server);
-    
+
     log.info("Removing zookeeper lock for " + server);
     String fullpath = ZooUtil.getRoot(instance) + Constants.ZTSERVERS + "/" + zPath;
     try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java b/server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java
index eca7dfe..bbfa71b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java
@@ -35,7 +35,7 @@ import org.apache.accumulo.core.data.thrift.IterInfo;
 import org.apache.accumulo.core.data.thrift.TColumn;
 import org.apache.accumulo.core.data.thrift.TKeyExtent;
 import org.apache.accumulo.core.data.thrift.TRange;
-import org.apache.accumulo.core.master.thrift.TableOperation;
+import org.apache.accumulo.core.master.thrift.FateOperation;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.Credentials;
@@ -417,12 +417,12 @@ public class AuditedSecurityOperation extends SecurityOperation {
   public static final String CAN_ONLINE_OFFLINE_TABLE_AUDIT_TEMPLATE = "action: %s; targetTable: %s;";
 
   @Override
-  public boolean canOnlineOfflineTable(TCredentials credentials, String tableId, TableOperation op) throws ThriftSecurityException {
+  public boolean canOnlineOfflineTable(TCredentials credentials, String tableId, FateOperation op) throws ThriftSecurityException {
     String tableName = getTableName(tableId);
     String operation = null;
-    if (op == TableOperation.ONLINE)
+    if (op == FateOperation.TABLE_ONLINE)
       operation = "onlineTable";
-    if (op == TableOperation.OFFLINE)
+    if (op == FateOperation.TABLE_OFFLINE)
       operation = "offlineTable";
     try {
       boolean result = super.canOnlineOfflineTable(credentials, tableId, op);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
index 010efa8..4b302f0 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
@@ -37,7 +37,7 @@ import org.apache.accumulo.core.data.thrift.IterInfo;
 import org.apache.accumulo.core.data.thrift.TColumn;
 import org.apache.accumulo.core.data.thrift.TKeyExtent;
 import org.apache.accumulo.core.data.thrift.TRange;
-import org.apache.accumulo.core.master.thrift.TableOperation;
+import org.apache.accumulo.core.master.thrift.FateOperation;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.security.Authorizations;
@@ -356,7 +356,7 @@ public class SecurityOperation {
   protected boolean _hasNamespacePermission(String user, String namespace, NamespacePermission permission, boolean useCached) throws ThriftSecurityException {
     targetUserExists(user);
 
-    if (namespace.equals(Constants.ACCUMULO_NAMESPACE_ID) && permission.equals(NamespacePermission.READ))
+    if (namespace.equals(Namespaces.ACCUMULO_NAMESPACE_ID) && permission.equals(NamespacePermission.READ))
       return true;
 
     try {
@@ -478,7 +478,7 @@ public class SecurityOperation {
         || hasNamespacePermissionForTableId(c, tableId, NamespacePermission.DROP_TABLE, false);
   }
 
-  public boolean canOnlineOfflineTable(TCredentials c, String tableId, TableOperation op) throws ThriftSecurityException {
+  public boolean canOnlineOfflineTable(TCredentials c, String tableId, FateOperation op) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c, SystemPermission.SYSTEM, false) || hasSystemPermission(c, SystemPermission.ALTER_TABLE, false)
         || hasTablePermission(c, tableId, TablePermission.ALTER_TABLE, false)

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
index 1fb0b16..0ea556f 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
@@ -23,10 +23,10 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeSet;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
@@ -386,8 +386,8 @@ public class ZKPermHandler implements PermissionHandler {
     tablePerms.put(MetadataTable.ID, Collections.singleton(TablePermission.ALTER_TABLE));
     // essentially the same but on the system namespace, the ALTER_TABLE permission is now redundant
     Map<String,Set<NamespacePermission>> namespacePerms = new HashMap<String,Set<NamespacePermission>>();
-    namespacePerms.put(Constants.ACCUMULO_NAMESPACE_ID, Collections.singleton(NamespacePermission.ALTER_NAMESPACE));
-    namespacePerms.put(Constants.ACCUMULO_NAMESPACE_ID, Collections.singleton(NamespacePermission.ALTER_TABLE));
+    namespacePerms.put(Namespaces.ACCUMULO_NAMESPACE_ID, Collections.singleton(NamespacePermission.ALTER_NAMESPACE));
+    namespacePerms.put(Namespaces.ACCUMULO_NAMESPACE_ID, Collections.singleton(NamespacePermission.ALTER_TABLE));
 
     try {
       // prep parent node of users with root username


[6/7] ACCUMULO-1965 Fix exception handling for namespaces

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/master/thrift/FateService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/master/thrift/FateService.java b/core/src/main/java/org/apache/accumulo/core/master/thrift/FateService.java
new file mode 100644
index 0000000..cc4d175
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/master/thrift/FateService.java
@@ -0,0 +1,5017 @@
+/*
+ * 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.master.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 FateService {
+
+  public interface Iface {
+
+    public long beginFateOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+
+    public void executeFateOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, FateOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException;
+
+    public String waitForFateOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException;
+
+    public void finishFateOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+
+  }
+
+  public interface AsyncIface {
+
+    public void beginFateOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.beginFateOperation_call> resultHandler) throws org.apache.thrift.TException;
+
+    public void executeFateOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, FateOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.executeFateOperation_call> resultHandler) throws org.apache.thrift.TException;
+
+    public void waitForFateOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.waitForFateOperation_call> resultHandler) throws org.apache.thrift.TException;
+
+    public void finishFateOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.finishFateOperation_call> resultHandler) throws org.apache.thrift.TException;
+
+  }
+
+  public static class Client extends org.apache.thrift.TServiceClient implements Iface {
+    public static class Factory implements org.apache.thrift.TServiceClientFactory<Client> {
+      public Factory() {}
+      public Client getClient(org.apache.thrift.protocol.TProtocol prot) {
+        return new Client(prot);
+      }
+      public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
+        return new Client(iprot, oprot);
+      }
+    }
+
+    public Client(org.apache.thrift.protocol.TProtocol prot)
+    {
+      super(prot, prot);
+    }
+
+    public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
+      super(iprot, oprot);
+    }
+
+    public long beginFateOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      send_beginFateOperation(tinfo, credentials);
+      return recv_beginFateOperation();
+    }
+
+    public void send_beginFateOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    {
+      beginFateOperation_args args = new beginFateOperation_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      sendBase("beginFateOperation", args);
+    }
+
+    public long recv_beginFateOperation() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      beginFateOperation_result result = new beginFateOperation_result();
+      receiveBase(result, "beginFateOperation");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.sec != null) {
+        throw result.sec;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "beginFateOperation failed: unknown result");
+    }
+
+    public void executeFateOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, FateOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
+    {
+      send_executeFateOperation(tinfo, credentials, opid, op, arguments, options, autoClean);
+      recv_executeFateOperation();
+    }
+
+    public void send_executeFateOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, FateOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean) throws org.apache.thrift.TException
+    {
+      executeFateOperation_args args = new executeFateOperation_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setOpid(opid);
+      args.setOp(op);
+      args.setArguments(arguments);
+      args.setOptions(options);
+      args.setAutoClean(autoClean);
+      sendBase("executeFateOperation", args);
+    }
+
+    public void recv_executeFateOperation() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
+    {
+      executeFateOperation_result result = new executeFateOperation_result();
+      receiveBase(result, "executeFateOperation");
+      if (result.sec != null) {
+        throw result.sec;
+      }
+      if (result.tope != null) {
+        throw result.tope;
+      }
+      return;
+    }
+
+    public String waitForFateOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
+    {
+      send_waitForFateOperation(tinfo, credentials, opid);
+      return recv_waitForFateOperation();
+    }
+
+    public void send_waitForFateOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
+    {
+      waitForFateOperation_args args = new waitForFateOperation_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setOpid(opid);
+      sendBase("waitForFateOperation", args);
+    }
+
+    public String recv_waitForFateOperation() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
+    {
+      waitForFateOperation_result result = new waitForFateOperation_result();
+      receiveBase(result, "waitForFateOperation");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.sec != null) {
+        throw result.sec;
+      }
+      if (result.tope != null) {
+        throw result.tope;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "waitForFateOperation failed: unknown result");
+    }
+
+    public void finishFateOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      send_finishFateOperation(tinfo, credentials, opid);
+      recv_finishFateOperation();
+    }
+
+    public void send_finishFateOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
+    {
+      finishFateOperation_args args = new finishFateOperation_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setOpid(opid);
+      sendBase("finishFateOperation", args);
+    }
+
+    public void recv_finishFateOperation() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      finishFateOperation_result result = new finishFateOperation_result();
+      receiveBase(result, "finishFateOperation");
+      if (result.sec != null) {
+        throw result.sec;
+      }
+      return;
+    }
+
+  }
+  public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
+    public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
+      private org.apache.thrift.async.TAsyncClientManager clientManager;
+      private org.apache.thrift.protocol.TProtocolFactory protocolFactory;
+      public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) {
+        this.clientManager = clientManager;
+        this.protocolFactory = protocolFactory;
+      }
+      public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) {
+        return new AsyncClient(protocolFactory, clientManager, transport);
+      }
+    }
+
+    public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) {
+      super(protocolFactory, clientManager, transport);
+    }
+
+    public void beginFateOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<beginFateOperation_call> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      beginFateOperation_call method_call = new beginFateOperation_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class beginFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private org.apache.accumulo.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.security.thrift.TCredentials credentials;
+      public beginFateOperation_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<beginFateOperation_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("beginFateOperation", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        beginFateOperation_args args = new beginFateOperation_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public long getResult() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_beginFateOperation();
+      }
+    }
+
+    public void executeFateOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, FateOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<executeFateOperation_call> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      executeFateOperation_call method_call = new executeFateOperation_call(tinfo, credentials, opid, op, arguments, options, autoClean, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class executeFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private org.apache.accumulo.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.security.thrift.TCredentials credentials;
+      private long opid;
+      private FateOperation op;
+      private List<ByteBuffer> arguments;
+      private Map<String,String> options;
+      private boolean autoClean;
+      public executeFateOperation_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, FateOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<executeFateOperation_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.opid = opid;
+        this.op = op;
+        this.arguments = arguments;
+        this.options = options;
+        this.autoClean = autoClean;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("executeFateOperation", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        executeFateOperation_args args = new executeFateOperation_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setOpid(opid);
+        args.setOp(op);
+        args.setArguments(arguments);
+        args.setOptions(options);
+        args.setAutoClean(autoClean);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_executeFateOperation();
+      }
+    }
+
+    public void waitForFateOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<waitForFateOperation_call> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      waitForFateOperation_call method_call = new waitForFateOperation_call(tinfo, credentials, opid, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class waitForFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private org.apache.accumulo.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.security.thrift.TCredentials credentials;
+      private long opid;
+      public waitForFateOperation_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<waitForFateOperation_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.opid = opid;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("waitForFateOperation", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        waitForFateOperation_args args = new waitForFateOperation_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setOpid(opid);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public String getResult() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_waitForFateOperation();
+      }
+    }
+
+    public void finishFateOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<finishFateOperation_call> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      finishFateOperation_call method_call = new finishFateOperation_call(tinfo, credentials, opid, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class finishFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private org.apache.accumulo.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.security.thrift.TCredentials credentials;
+      private long opid;
+      public finishFateOperation_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<finishFateOperation_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.opid = opid;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("finishFateOperation", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        finishFateOperation_args args = new finishFateOperation_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setOpid(opid);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_finishFateOperation();
+      }
+    }
+
+  }
+
+  public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
+    private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName());
+    public Processor(I iface) {
+      super(iface, getProcessMap(new HashMap<String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
+    }
+
+    protected Processor(I iface, Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
+      super(iface, getProcessMap(processMap));
+    }
+
+    private static <I extends Iface> Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> getProcessMap(Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
+      processMap.put("beginFateOperation", new beginFateOperation());
+      processMap.put("executeFateOperation", new executeFateOperation());
+      processMap.put("waitForFateOperation", new waitForFateOperation());
+      processMap.put("finishFateOperation", new finishFateOperation());
+      return processMap;
+    }
+
+    public static class beginFateOperation<I extends Iface> extends org.apache.thrift.ProcessFunction<I, beginFateOperation_args> {
+      public beginFateOperation() {
+        super("beginFateOperation");
+      }
+
+      public beginFateOperation_args getEmptyArgsInstance() {
+        return new beginFateOperation_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public beginFateOperation_result getResult(I iface, beginFateOperation_args args) throws org.apache.thrift.TException {
+        beginFateOperation_result result = new beginFateOperation_result();
+        try {
+          result.success = iface.beginFateOperation(args.tinfo, args.credentials);
+          result.setSuccessIsSet(true);
+        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        }
+        return result;
+      }
+    }
+
+    public static class executeFateOperation<I extends Iface> extends org.apache.thrift.ProcessFunction<I, executeFateOperation_args> {
+      public executeFateOperation() {
+        super("executeFateOperation");
+      }
+
+      public executeFateOperation_args getEmptyArgsInstance() {
+        return new executeFateOperation_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public executeFateOperation_result getResult(I iface, executeFateOperation_args args) throws org.apache.thrift.TException {
+        executeFateOperation_result result = new executeFateOperation_result();
+        try {
+          iface.executeFateOperation(args.tinfo, args.credentials, args.opid, args.op, args.arguments, args.options, args.autoClean);
+        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException tope) {
+          result.tope = tope;
+        }
+        return result;
+      }
+    }
+
+    public static class waitForFateOperation<I extends Iface> extends org.apache.thrift.ProcessFunction<I, waitForFateOperation_args> {
+      public waitForFateOperation() {
+        super("waitForFateOperation");
+      }
+
+      public waitForFateOperation_args getEmptyArgsInstance() {
+        return new waitForFateOperation_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public waitForFateOperation_result getResult(I iface, waitForFateOperation_args args) throws org.apache.thrift.TException {
+        waitForFateOperation_result result = new waitForFateOperation_result();
+        try {
+          result.success = iface.waitForFateOperation(args.tinfo, args.credentials, args.opid);
+        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException tope) {
+          result.tope = tope;
+        }
+        return result;
+      }
+    }
+
+    public static class finishFateOperation<I extends Iface> extends org.apache.thrift.ProcessFunction<I, finishFateOperation_args> {
+      public finishFateOperation() {
+        super("finishFateOperation");
+      }
+
+      public finishFateOperation_args getEmptyArgsInstance() {
+        return new finishFateOperation_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public finishFateOperation_result getResult(I iface, finishFateOperation_args args) throws org.apache.thrift.TException {
+        finishFateOperation_result result = new finishFateOperation_result();
+        try {
+          iface.finishFateOperation(args.tinfo, args.credentials, args.opid);
+        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        }
+        return result;
+      }
+    }
+
+  }
+
+  public static class beginFateOperation_args implements org.apache.thrift.TBase<beginFateOperation_args, beginFateOperation_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("beginFateOperation_args");
+
+    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new beginFateOperation_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new beginFateOperation_argsTupleSchemeFactory());
+    }
+
+    public org.apache.accumulo.trace.thrift.TInfo tinfo; // required
+    public org.apache.accumulo.core.security.thrift.TCredentials credentials; // 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 {
+      TINFO((short)2, "tinfo"),
+      CREDENTIALS((short)1, "credentials");
+
+      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 2: // TINFO
+            return TINFO;
+          case 1: // CREDENTIALS
+            return CREDENTIALS;
+          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
+    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.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.trace.thrift.TInfo.class)));
+      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.security.thrift.TCredentials.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(beginFateOperation_args.class, metaDataMap);
+    }
+
+    public beginFateOperation_args() {
+    }
+
+    public beginFateOperation_args(
+      org.apache.accumulo.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.security.thrift.TCredentials credentials)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public beginFateOperation_args(beginFateOperation_args other) {
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.trace.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.security.thrift.TCredentials(other.credentials);
+      }
+    }
+
+    public beginFateOperation_args deepCopy() {
+      return new beginFateOperation_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+    }
+
+    public org.apache.accumulo.trace.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public beginFateOperation_args setTinfo(org.apache.accumulo.trace.thrift.TInfo tinfo) {
+      this.tinfo = tinfo;
+      return this;
+    }
+
+    public void unsetTinfo() {
+      this.tinfo = null;
+    }
+
+    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+    public boolean isSetTinfo() {
+      return this.tinfo != null;
+    }
+
+    public void setTinfoIsSet(boolean value) {
+      if (!value) {
+        this.tinfo = null;
+      }
+    }
+
+    public org.apache.accumulo.core.security.thrift.TCredentials getCredentials() {
+      return this.credentials;
+    }
+
+    public beginFateOperation_args setCredentials(org.apache.accumulo.core.security.thrift.TCredentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public void unsetCredentials() {
+      this.credentials = null;
+    }
+
+    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredentials() {
+      return this.credentials != null;
+    }
+
+    public void setCredentialsIsSet(boolean value) {
+      if (!value) {
+        this.credentials = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case TINFO:
+        if (value == null) {
+          unsetTinfo();
+        } else {
+          setTinfo((org.apache.accumulo.trace.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.security.thrift.TCredentials)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      }
+      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 TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof beginFateOperation_args)
+        return this.equals((beginFateOperation_args)that);
+      return false;
+    }
+
+    public boolean equals(beginFateOperation_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      return 0;
+    }
+
+    public int compareTo(beginFateOperation_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      beginFateOperation_args typedOther = (beginFateOperation_args)other;
+
+      lastComparison = Boolean.valueOf(isSetTinfo()).compareTo(typedOther.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, typedOther.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetCredentials()).compareTo(typedOther.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, typedOther.credentials);
+        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("beginFateOperation_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.validate();
+      }
+    }
+
+    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 {
+        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 beginFateOperation_argsStandardSchemeFactory implements SchemeFactory {
+      public beginFateOperation_argsStandardScheme getScheme() {
+        return new beginFateOperation_argsStandardScheme();
+      }
+    }
+
+    private static class beginFateOperation_argsStandardScheme extends StandardScheme<beginFateOperation_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, beginFateOperation_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 2: // TINFO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tinfo = new org.apache.accumulo.trace.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // CREDENTIALS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.credentials = new org.apache.accumulo.core.security.thrift.TCredentials();
+                struct.credentials.read(iprot);
+                struct.setCredentialsIsSet(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, beginFateOperation_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class beginFateOperation_argsTupleSchemeFactory implements SchemeFactory {
+      public beginFateOperation_argsTupleScheme getScheme() {
+        return new beginFateOperation_argsTupleScheme();
+      }
+    }
+
+    private static class beginFateOperation_argsTupleScheme extends TupleScheme<beginFateOperation_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, beginFateOperation_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, beginFateOperation_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.trace.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.credentials = new org.apache.accumulo.core.security.thrift.TCredentials();
+          struct.credentials.read(iprot);
+          struct.setCredentialsIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class beginFateOperation_result implements org.apache.thrift.TBase<beginFateOperation_result, beginFateOperation_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("beginFateOperation_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I64, (short)0);
+    private static final org.apache.thrift.protocol.TField SEC_FIELD_DESC = new org.apache.thrift.protocol.TField("sec", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new beginFateOperation_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new beginFateOperation_resultTupleSchemeFactory());
+    }
+
+    public long success; // required
+    public org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec; // 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 {
+      SUCCESS((short)0, "success"),
+      SEC((short)1, "sec");
+
+      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 0: // SUCCESS
+            return SUCCESS;
+          case 1: // SEC
+            return SEC;
+          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 __SUCCESS_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.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+      tmpMap.put(_Fields.SEC, new org.apache.thrift.meta_data.FieldMetaData("sec", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(beginFateOperation_result.class, metaDataMap);
+    }
+
+    public beginFateOperation_result() {
+    }
+
+    public beginFateOperation_result(
+      long success,
+      org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec)
+    {
+      this();
+      this.success = success;
+      setSuccessIsSet(true);
+      this.sec = sec;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public beginFateOperation_result(beginFateOperation_result other) {
+      __isset_bitfield = other.__isset_bitfield;
+      this.success = other.success;
+      if (other.isSetSec()) {
+        this.sec = new org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException(other.sec);
+      }
+    }
+
+    public beginFateOperation_result deepCopy() {
+      return new beginFateOperation_result(this);
+    }
+
+    @Override
+    public void clear() {
+      setSuccessIsSet(false);
+      this.success = 0;
+      this.sec = null;
+    }
+
+    public long getSuccess() {
+      return this.success;
+    }
+
+    public beginFateOperation_result setSuccess(long success) {
+      this.success = success;
+      setSuccessIsSet(true);
+      return this;
+    }
+
+    public void unsetSuccess() {
+      __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value);
+    }
+
+    public org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException getSec() {
+      return this.sec;
+    }
+
+    public beginFateOperation_result setSec(org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
+      this.sec = sec;
+      return this;
+    }
+
+    public void unsetSec() {
+      this.sec = null;
+    }
+
+    /** Returns true if field sec is set (has been assigned a value) and false otherwise */
+    public boolean isSetSec() {
+      return this.sec != null;
+    }
+
+    public void setSecIsSet(boolean value) {
+      if (!value) {
+        this.sec = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((Long)value);
+        }
+        break;
+
+      case SEC:
+        if (value == null) {
+          unsetSec();
+        } else {
+          setSec((org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return Long.valueOf(getSuccess());
+
+      case SEC:
+        return getSec();
+
+      }
+      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 SUCCESS:
+        return isSetSuccess();
+      case SEC:
+        return isSetSec();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof beginFateOperation_result)
+        return this.equals((beginFateOperation_result)that);
+      return false;
+    }
+
+    public boolean equals(beginFateOperation_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true;
+      boolean that_present_success = true;
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (this.success != that.success)
+          return false;
+      }
+
+      boolean this_present_sec = true && this.isSetSec();
+      boolean that_present_sec = true && that.isSetSec();
+      if (this_present_sec || that_present_sec) {
+        if (!(this_present_sec && that_present_sec))
+          return false;
+        if (!this.sec.equals(that.sec))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      return 0;
+    }
+
+    public int compareTo(beginFateOperation_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      beginFateOperation_result typedOther = (beginFateOperation_result)other;
+
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetSec()).compareTo(typedOther.isSetSec());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSec()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sec, typedOther.sec);
+        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("beginFateOperation_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      sb.append(this.success);
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("sec:");
+      if (this.sec == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.sec);
+      }
+      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 beginFateOperation_resultStandardSchemeFactory implements SchemeFactory {
+      public beginFateOperation_resultStandardScheme getScheme() {
+        return new beginFateOperation_resultStandardScheme();
+      }
+    }
+
+    private static class beginFateOperation_resultStandardScheme extends StandardScheme<beginFateOperation_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, beginFateOperation_result 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 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+                struct.success = iprot.readI64();
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // SEC
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.sec = new org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException();
+                struct.sec.read(iprot);
+                struct.setSecIsSet(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, beginFateOperation_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.isSetSuccess()) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          oprot.writeI64(struct.success);
+          oprot.writeFieldEnd();
+        }
+        if (struct.sec != null) {
+          oprot.writeFieldBegin(SEC_FIELD_DESC);
+          struct.sec.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class beginFateOperation_resultTupleSchemeFactory implements SchemeFactory {
+      public beginFateOperation_resultTupleScheme getScheme() {
+        return new beginFateOperation_resultTupleScheme();
+      }
+    }
+
+    private static class beginFateOperation_resultTupleScheme extends TupleScheme<beginFateOperation_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, beginFateOperation_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        if (struct.isSetSec()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetSuccess()) {
+          oprot.writeI64(struct.success);
+        }
+        if (struct.isSetSec()) {
+          struct.sec.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, beginFateOperation_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.success = iprot.readI64();
+          struct.setSuccessIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.sec = new org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException();
+          struct.sec.read(iprot);
+          struct.setSecIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class executeFateOperation_args implements org.apache.thrift.TBase<executeFateOperation_args, executeFateOperation_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("executeFateOperation_args");
+
+    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)7);
+    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField OPID_FIELD_DESC = new org.apache.thrift.protocol.TField("opid", org.apache.thrift.protocol.TType.I64, (short)2);
+    private static final org.apache.thrift.protocol.TField OP_FIELD_DESC = new org.apache.thrift.protocol.TField("op", org.apache.thrift.protocol.TType.I32, (short)3);
+    private static final org.apache.thrift.protocol.TField ARGUMENTS_FIELD_DESC = new org.apache.thrift.protocol.TField("arguments", org.apache.thrift.protocol.TType.LIST, (short)4);
+    private static final org.apache.thrift.protocol.TField OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("options", org.apache.thrift.protocol.TType.MAP, (short)5);
+    private static final org.apache.thrift.protocol.TField AUTO_CLEAN_FIELD_DESC = new org.apache.thrift.protocol.TField("autoClean", org.apache.thrift.protocol.TType.BOOL, (short)6);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new executeFateOperation_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new executeFateOperation_argsTupleSchemeFactory());
+    }
+
+    public org.apache.accumulo.trace.thrift.TInfo tinfo; // required
+    public org.apache.accumulo.core.security.thrift.TCredentials credentials; // required
+    public long opid; // required
+    /**
+     * 
+     * @see FateOperation
+     */
+    public FateOperation op; // required
+    public List<ByteBuffer> arguments; // required
+    public Map<String,String> options; // required
+    public boolean autoClean; // 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 {
+      TINFO((short)7, "tinfo"),
+      CREDENTIALS((short)1, "credentials"),
+      OPID((short)2, "opid"),
+      /**
+       * 
+       * @see FateOperation
+       */
+      OP((short)3, "op"),
+      ARGUMENTS((short)4, "arguments"),
+      OPTIONS((short)5, "options"),
+      AUTO_CLEAN((short)6, "autoClean");
+
+      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 7: // TINFO
+            return TINFO;
+          case 1: // CREDENTIALS
+            return CREDENTIALS;
+          case 2: // OPID
+            return OPID;
+          case 3: // OP
+            return OP;
+          case 4: // ARGUMENTS
+            return ARGUMENTS;
+          case 5: // OPTIONS
+            return OPTIONS;
+          case 6: // AUTO_CLEAN
+            return AUTO_CLEAN;
+          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 __OPID_ISSET_ID = 0;
+    private static final int __AUTOCLEAN_ISSET_ID = 1;
+    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.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.trace.thrift.TInfo.class)));
+      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.security.thrift.TCredentials.class)));
+      tmpMap.put(_Fields.OPID, new org.apache.thrift.meta_data.FieldMetaData("opid", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+      tmpMap.put(_Fields.OP, new org.apache.thrift.meta_data.FieldMetaData("op", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, FateOperation.class)));
+      tmpMap.put(_Fields.ARGUMENTS, new org.apache.thrift.meta_data.FieldMetaData("arguments", 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              , true))));
+      tmpMap.put(_Fields.OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("options", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+      tmpMap.put(_Fields.AUTO_CLEAN, new org.apache.thrift.meta_data.FieldMetaData("autoClean", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(executeFateOperation_args.class, metaDataMap);
+    }
+
+    public executeFateOperation_args() {
+    }
+
+    public executeFateOperation_args(
+      org.apache.accumulo.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.security.thrift.TCredentials credentials,
+      long opid,
+      FateOperation op,
+      List<ByteBuffer> arguments,
+      Map<String,String> options,
+      boolean autoClean)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.opid = opid;
+      setOpidIsSet(true);
+      this.op = op;
+      this.arguments = arguments;
+      this.options = options;
+      this.autoClean = autoClean;
+      setAutoCleanIsSet(true);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public executeFateOperation_args(executeFateOperation_args other) {
+      __isset_bitfield = other.__isset_bitfield;
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.trace.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.security.thrift.TCredentials(other.credentials);
+      }
+      this.opid = other.opid;
+      if (other.isSetOp()) {
+        this.op = other.op;
+      }
+      if (other.isSetArguments()) {
+        List<ByteBuffer> __this__arguments = new ArrayList<ByteBuffer>();
+        for (ByteBuffer other_element : other.arguments) {
+          ByteBuffer temp_binary_element = org.apache.thrift.TBaseHelper.copyBinary(other_element);
+;
+          __this__arguments.add(temp_binary_element);
+        }
+        this.arguments = __this__arguments;
+      }
+      if (other.isSetOptions()) {
+        Map<String,String> __this__options = new HashMap<String,String>();
+        for (Map.Entry<String, String> other_element : other.options.entrySet()) {
+
+          String other_element_key = other_element.getKey();
+          String other_element_value = other_element.getValue();
+
+          String __this__options_copy_key = other_element_key;
+
+          String __this__options_copy_value = other_element_value;
+
+          __this__options.put(__this__options_copy_key, __this__options_copy_value);
+        }
+        this.options = __this__options;
+      }
+      this.autoClean = other.autoClean;
+    }
+
+    public executeFateOperation_args deepCopy() {
+      return new executeFateOperation_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      setOpidIsSet(false);
+      this.opid = 0;
+      this.op = null;
+      this.arguments = null;
+      this.options = null;
+      setAutoCleanIsSet(false);
+      this.autoClean = false;
+    }
+
+    public org.apache.accumulo.trace.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public executeFateOperation_args setTinfo(org.apache.accumulo.trace.thrift.TInfo tinfo) {
+      this.tinfo = tinfo;
+      return this;
+    }
+
+    public void unsetTinfo() {
+      this.tinfo = null;
+    }
+
+    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+    public boolean isSetTinfo() {
+      return this.tinfo != null;
+    }
+
+    public void setTinfoIsSet(boolean value) {
+      if (!value) {
+        this.tinfo = null;
+      }
+    }
+
+    public org.apache.accumulo.core.security.thrift.TCredentials getCredentials() {
+      return this.credentials;
+    }
+
+    public executeFateOperation_args setCredentials(org.apache.accumulo.core.security.thrift.TCredentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public void unsetCredentials() {
+      this.credentials = null;
+    }
+
+    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredentials() {
+      return this.credentials != null;
+    }
+
+    public void setCredentialsIsSet(boolean value) {
+      if (!value) {
+        this.credentials = null;
+      }
+    }
+
+    public long getOpid() {
+      return this.opid;
+    }
+
+    public executeFateOperation_args setOpid(long opid) {
+      this.opid = opid;
+      setOpidIsSet(true);
+      return this;
+    }
+
+    public void unsetOpid() {
+      __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __OPID_ISSET_ID);
+    }
+
+    /** Returns true if field opid is set (has been assigned a value) and false otherwise */
+    public boolean isSetOpid() {
+      return EncodingUtils.testBit(__isset_bitfield, __OPID_ISSET_ID);
+    }
+
+    public void setOpidIsSet(boolean value) {
+      __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __OPID_ISSET_ID, value);
+    }
+
+    /**
+     * 
+     * @see FateOperation
+     */
+    public FateOperation getOp() {
+      return this.op;
+    }
+
+    /**
+     * 
+     * @see FateOperation
+     */
+    public executeFateOperation_args setOp(FateOperation op) {
+      this.op = op;
+      return this;
+    }
+
+    public void unsetOp() {
+      this.op = null;
+    }
+
+    /** Returns true if field op is set (has been assigned a value) and false otherwise */
+    public boolean isSetOp() {
+      return this.op != null;
+    }
+
+    public void setOpIsSet(boolean value) {
+      if (!value) {
+        this.op = null;
+      }
+    }
+
+    public int getArgumentsSize() {
+      return (this.arguments == null) ? 0 : this.arguments.size();
+    }
+
+    public java.util.Iterator<ByteBuffer> getArgumentsIterator() {
+      return (this.arguments == null) ? null : this.arguments.iterator();
+    }
+
+    public void addToArguments(ByteBuffer elem) {
+      if (this.arguments == null) {
+        this.arguments = new ArrayList<ByteBuffer>();
+      }
+      this.arguments.add(elem);
+    }
+
+    public List<ByteBuffer> getArguments() {
+      return this.arguments;
+    }
+
+    public executeFateOperation_args setArguments(List<ByteBuffer> arguments) {
+      this.arguments = arguments;
+      return this;
+    }
+
+    public void unsetArguments() {
+      this.arguments = null;
+    }
+
+    /** Returns true if field arguments is set (has been assigned a value) and false otherwise */
+    public boolean isSetArguments() {
+      return this.arguments != null;
+    }
+
+    public void setArgumentsIsSet(boolean value) {
+      if (!value) {
+        this.arguments = null;
+      }
+    }
+
+    public int getOptionsSize() {
+      return (this.options == null) ? 0 : this.options.size();
+    }
+
+    public void putToOptions(String key, String val) {
+      if (this.options == null) {
+        this.options = new HashMap<String,String>();
+      }
+      this.options.put(key, val);
+    }
+
+    public Map<String,String> getOptions() {
+      return this.options;
+    }
+
+    public executeFateOperation_args setOptions(Map<String,String> options) {
+      this.options = options;
+      return this;
+    }
+
+    public void unsetOptions() {
+      this.options = null;
+    }
+
+    /** Returns true if field options is set (has been assigned a value) and false otherwise */
+    public boolean isSetOptions() {
+      return this.options != null;
+    }
+
+    public void setOptionsIsSet(boolean value) {
+      if (!value) {
+        this.options = null;
+      }
+    }
+
+    public boolean isAutoClean() {
+      return this.autoClean;
+    }
+
+    public executeFateOperation_args setAutoClean(boolean autoClean) {
+      this.autoClean = autoClean;
+      setAutoCleanIsSet(true);
+      return this;
+    }
+
+    public void unsetAutoClean() {
+      __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __AUTOCLEAN_ISSET_ID);
+    }
+
+    /** Returns true if field autoClean is set (has been assigned a value) and false otherwise */
+    public boolean isSetAutoClean() {
+      return EncodingUtils.testBit(__isset_bitfield, __AUTOCLEAN_ISSET_ID);
+    }
+
+    public void setAutoCleanIsSet(boolean value) {
+      __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __AUTOCLEAN_ISSET_ID, value);
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case TINFO:
+        if (value == null) {
+          unsetTinfo();
+        } else {
+          setTinfo((org.apache.accumulo.trace.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.security.thrift.TCredentials)value);
+        }
+        break;
+
+      case OPID:
+        if (value == null) {
+          unsetOpid();
+        } else {
+          setOpid((Long)value);
+        }
+        break;
+
+      case OP:
+        if (value == null) {
+          unsetOp();
+        } else {
+          setOp((FateOperation)value);
+        }
+        break;
+
+      case ARGUMENTS:
+        if (value == null) {
+          unsetArguments();
+        } else {
+          setArguments((List<ByteBuffer>)value);
+        }
+        break;
+
+      case OPTIONS:
+        if (value == null) {
+          unsetOptions();
+        } else {
+          setOptions((Map<String,String>)value);
+        }
+        break;
+
+      case AUTO_CLEAN:
+        if (value == null) {
+          unsetAutoClean();
+        } else {
+          setAutoClean((Boolean)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      case OPID:
+        return Long.valueOf(getOpid());
+
+      case OP:
+        return getOp();
+
+      case ARGUMENTS:
+        return getArguments();
+
+      case OPTIONS:
+        return getOptions();
+
+      case AUTO_CLEAN:
+        return Boolean.valueOf(isAutoClean());
+
+      }
+      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 TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case OPID:
+        return isSetOpid();
+      case OP:
+        return isSetOp();
+      case ARGUMENTS:
+        return isSetArguments();
+      case OPTIONS:
+        return isSetOptions();
+      case AUTO_CLEAN:
+        return isSetAutoClean();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof executeFateOperation_args)
+        return this.equals((executeFateOperation_args)that);
+      return false;
+    }
+
+    public boolean equals(executeFateOperation_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      boolean this_present_opid = true;
+      boolean that_present_opid = true;
+      if (this_present_opid || that_present_opid) {
+        if (!(this_present_opid && that_present_opid))
+          return false;
+        if (this.opid != that.opid)
+          return false;
+      }
+
+      boolean this_present_op = true && this.isSetOp();
+      boolean that_present_op = true && that.isSetOp();
+      if (this_present_op || that_present_op) {
+        if (!(this_present_op && that_present_op))
+          return false;
+        if (!this.op.equals(that.op))
+          return false;
+      }
+
+      boolean this_present_arguments = true && this.isSetArguments();
+      boolean that_present_arguments = true && that.isSetArguments();
+      if (this_present_arguments || that_present_arguments) {
+        if (!(this_present_arguments && that_present_arguments))
+          return false;
+        if (!this.arguments.equals(that.arguments))
+          return false;
+      }
+
+      boolean this_present_options = true && this.isSetOptions();
+      boolean that_present_options = true && that.isSetOptions();
+      if (this_present_options || that_present_options) {
+        if (!(this_present_options && that_present_options))
+          return false;
+        if (!this.options.equals(that.options))
+          return false;
+      }
+
+      boolean this_present_autoClean = true;
+      boolean that_present_autoClean = true;
+      if (this_present_autoClean || that_present_autoClean) {
+        if (!(this_present_autoClean && that_present_autoClean))
+          return false;
+        if (this.autoClean != that.autoClean)
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      return 0;
+    }
+
+    public int compareTo(executeFateOperation_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      executeFateOperation_args typedOther = (executeFateOperation_args)other;
+
+      lastComparison = Boolean.valueOf(isSetTinfo()).compareTo(typedOther.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, typedOther.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetCredentials()).compareTo(typedOther.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, typedOther.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetOpid()).compareTo(typedOther.isSetOpid());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetOpid()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.opid, typedOther.opid);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetOp()).compareTo(typedOther.isSetOp());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetOp()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.op, typedOther.op);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetArguments()).compareTo(typedOther.isSetArguments());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetArguments()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.arguments, typedOther.arguments);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetOptions()).compareTo(typedOther.isSetOptions());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetOptions()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.options, typedOther.options);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetAutoClean()).compareTo(typedOther.isSetAutoClean());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetAutoClean()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.autoClean, typedOther.autoClean);
+        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("executeFateOperation_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("opid:");
+      sb.append(this.opid);
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("op:");
+      if (this.op == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.op);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("arguments:");
+      if (this.arguments == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.arguments);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("options:");
+      if (this.options == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.options);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("autoClean:");
+      sb.append(this.autoClean);
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.validate();
+      }
+    }
+
+    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 executeFateOperation_argsStandardSchemeFactory implements SchemeFactory {
+      public executeFateOperation_argsStandardScheme getScheme() {
+        return new executeFateOperation_argsStandardScheme();
+      }
+    }
+
+    private static class executeFateOperation_argsStandardScheme extends StandardScheme<executeFateOperation_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, executeFateOperation_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (sch

<TRUNCATED>