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:07:36 UTC

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

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)) {