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/05 00:58:54 UTC

[13/50] [abbrv] git commit: ACCUMULO-802 added namespace locking to the rest of the tableOps and put them in a consistent order (namespace then table), switched to using HdfsZooInstance to get the instance for the namespaceId

ACCUMULO-802 added namespace locking to the rest of the tableOps and put them in a consistent order (namespace then table), switched to using HdfsZooInstance to get the instance for the namespaceId


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

Branch: refs/heads/master
Commit: 3d707c8645619e349fefd9323a2c2c22c803d4d4
Parents: c2ed43e
Author: Sean Hickey <ta...@gmail.com>
Authored: Wed Aug 14 08:04:55 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:10 2013 -0500

----------------------------------------------------------------------
 .../core/client/admin/TableOperationsImpl.java  |  5 +-
 .../java/org/apache/accumulo/master/Master.java |  8 +-
 .../master/tableOps/CancelCompactions.java      | 17 ++--
 .../master/tableOps/ChangeTableState.java       | 26 ++++--
 .../accumulo/master/tableOps/CloneTable.java    | 11 +--
 .../accumulo/master/tableOps/CompactRange.java  | 95 +++++++++++++++++++-
 .../accumulo/master/tableOps/CreateTable.java   |  8 +-
 .../accumulo/master/tableOps/DeleteTable.java   |  6 +-
 .../accumulo/master/tableOps/ExportTable.java   | 10 ++-
 .../accumulo/master/tableOps/ImportTable.java   |  5 +-
 .../accumulo/master/tableOps/RenameTable.java   |  4 +-
 .../master/tableOps/RenameTableNamespace.java   |  2 +-
 .../accumulo/master/tableOps/TableRangeOp.java  | 56 +++++++-----
 .../test/randomwalk/concurrent/Merge.java       |  2 +-
 .../concurrent/RenameTableNamespace.java        |  2 +-
 15 files changed, 197 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/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 cfaa657..aa4c83e 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
@@ -129,7 +129,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   private Credentials credentials;
 
   public static final String CLONE_EXCLUDE_PREFIX = "!";
-
+  
   private static final Logger log = Logger.getLogger(TableOperations.class);
 
   /**
@@ -727,7 +727,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
   // get the properties that are only in the table 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 {
+  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 TableNamespaceOperationsImpl(instance, credentials).getProperties(namespace);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/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 338550a..5cfd3c3 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
@@ -929,7 +929,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
           TimeType timeType = TimeType.valueOf(ByteBufferUtil.toString(arguments.get(1)));
 
           try {
-            fate.seedTransaction(opid, new TraceRepo<Master>(new CreateTable(c.getPrincipal(), tableName, timeType, options, getInstance())), autoCleanup);
+            fate.seedTransaction(opid, new TraceRepo<Master>(new CreateTable(c.getPrincipal(), tableName, timeType, options)), autoCleanup);
           } catch (TableNamespaceNotFoundException e) {
             throw new ThriftTableOperationException(null, tableName, TableOperation.CREATE, TableOperationExceptionType.NOTFOUND, e.getMessage());
           }
@@ -946,7 +946,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
           if (!security.canRenameTable(c, tableId, oldTableName, newTableName))
             throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-          fate.seedTransaction(opid, new TraceRepo<Master>(new RenameTable(tableId, oldTableName, newTableName, getInstance())), autoCleanup);
+          fate.seedTransaction(opid, new TraceRepo<Master>(new RenameTable(tableId, oldTableName, newTableName)), autoCleanup);
           
           break;
         }
@@ -975,7 +975,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
             propertiesToSet.put(entry.getKey(), entry.getValue());
           }
 
-          fate.seedTransaction(opid, new TraceRepo<Master>(new CloneTable(c.getPrincipal(), srcTableId, tableName, propertiesToSet, propertiesToExclude, getInstance())),
+          fate.seedTransaction(opid, new TraceRepo<Master>(new CloneTable(c.getPrincipal(), srcTableId, tableName, propertiesToSet, propertiesToExclude)),
               autoCleanup);
 
           break;
@@ -986,7 +986,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
           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, getInstance())), autoCleanup);
+          fate.seedTransaction(opid, new TraceRepo<Master>(new DeleteTable(tableId)), autoCleanup);
           break;
         }
         case ONLINE: {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/server/master/src/main/java/org/apache/accumulo/master/tableOps/CancelCompactions.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CancelCompactions.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CancelCompactions.java
index dd4c229..9698344 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CancelCompactions.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CancelCompactions.java
@@ -17,6 +17,8 @@
 package org.apache.accumulo.master.tableOps;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.TableOperation;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
@@ -32,7 +34,7 @@ class FinishCancelCompaction extends MasterRepo {
   public FinishCancelCompaction(String tableId) {
     this.tableId = tableId;
   }
-
+  
   @Override
   public Repo<Master> call(long tid, Master environment) throws Exception {
     Utils.getReadLock(tableId, tid).unlock();
@@ -52,14 +54,18 @@ public class CancelCompactions extends MasterRepo {
   
   private static final long serialVersionUID = 1L;
   private String tableId;
+  private String namespaceId;
   
   public CancelCompactions(String tableId) {
     this.tableId = tableId;
+    Instance inst = HdfsZooInstance.getInstance();
+    this.namespaceId = Tables.getNamespace(inst, tableId);
   }
-
+  
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    return Utils.reserveTable(tableId, tid, false, true, TableOperation.COMPACT_CANCEL);
+    return Utils.reserveTableNamespace(namespaceId, tid, false, true, TableOperation.COMPACT_CANCEL)
+        + Utils.reserveTable(tableId, tid, false, true, TableOperation.COMPACT_CANCEL);
   }
   
   @Override
@@ -85,15 +91,16 @@ public class CancelCompactions extends MasterRepo {
           return (flushID + "").getBytes();
         else
           return (cid + "").getBytes();
-
+        
       }
     });
-
+    
     return new FinishCancelCompaction(tableId);
   }
   
   @Override
   public void undo(long tid, Master environment) throws Exception {
+    Utils.unreserveTableNamespace(namespaceId, tid, false);
     Utils.unreserveTable(tableId, tid, false);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/server/master/src/main/java/org/apache/accumulo/master/tableOps/ChangeTableState.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ChangeTableState.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ChangeTableState.java
index 697c15e..ffd61e4 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ChangeTableState.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ChangeTableState.java
@@ -16,49 +16,57 @@
  */
 package org.apache.accumulo.master.tableOps;
 
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.TableOperation;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.master.Master;
+import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.tables.TableManager;
 import org.apache.log4j.Logger;
 
 public class ChangeTableState extends MasterRepo {
-  
+
   private static final long serialVersionUID = 1L;
   private String tableId;
   private TableOperation top;
-  
+  private String namespaceId;
+
   public ChangeTableState(String tableId, TableOperation top) {
     this.tableId = tableId;
     this.top = top;
-    
+    Instance inst = HdfsZooInstance.getInstance();
+    this.namespaceId = Tables.getNamespace(inst, tableId);
+
     if (top != TableOperation.ONLINE && top != TableOperation.OFFLINE)
       throw new IllegalArgumentException(top.toString());
   }
-  
+
   @Override
   public long isReady(long tid, Master environment) throws Exception {
     // reserve the table so that this op does not run concurrently with create, clone, or delete table
-    return Utils.reserveTable(tableId, tid, true, true, top);
+    return Utils.reserveTableNamespace(namespaceId, tid, false, true, top) + Utils.reserveTable(tableId, tid, true, true, top);
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master env) throws Exception {
-    
+
     TableState ts = TableState.ONLINE;
     if (top == TableOperation.OFFLINE)
       ts = TableState.OFFLINE;
-    
+
     TableManager.getInstance().transitionTableState(tableId, ts);
+    Utils.unreserveTableNamespace(namespaceId, tid, false);
     Utils.unreserveTable(tableId, tid, true);
     Logger.getLogger(ChangeTableState.class).debug("Changed table state " + tableId + " " + ts);
     env.getEventCoordinator().event("Set table state of %s to %s", tableId, ts);
     return null;
   }
-  
+
   @Override
   public void undo(long tid, Master env) throws Exception {
+    Utils.unreserveTableNamespace(namespaceId, tid, false);
     Utils.unreserveTable(tableId, tid, true);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/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 46edc25..f370a3a 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
@@ -75,11 +75,11 @@ class FinishCloneTable extends MasterRepo {
     
     TableManager.getInstance().transitionTableState(cloneInfo.tableId, TableState.ONLINE);
     
-    Utils.unreserveTable(cloneInfo.srcTableId, tid, false);
-    Utils.unreserveTable(cloneInfo.tableId, tid, true);
     Utils.unreserveTableNamespace(cloneInfo.srcNamespaceId, tid, false);
     if (!cloneInfo.namespaceId.equals(cloneInfo.srcNamespaceId))
       Utils.unreserveTableNamespace(cloneInfo.namespaceId, tid, false);
+    Utils.unreserveTable(cloneInfo.srcTableId, tid, false);
+    Utils.unreserveTable(cloneInfo.tableId, tid, true);
     
     environment.getEventCoordinator().event("Cloned table %s from %s", cloneInfo.tableName, cloneInfo.srcTableId);
     
@@ -171,9 +171,9 @@ class CloneZookeeper extends MasterRepo {
   public void undo(long tid, Master environment) throws Exception {
     Instance instance = HdfsZooInstance.getInstance();
     TableManager.getInstance().removeTable(cloneInfo.tableId);
-    Utils.unreserveTable(cloneInfo.tableId, tid, true);
     if (!cloneInfo.namespaceId.equals(cloneInfo.srcNamespaceId))
       Utils.unreserveTableNamespace(cloneInfo.namespaceId, tid, false);
+    Utils.unreserveTable(cloneInfo.tableId, tid, true);
     Tables.clearCache(instance);
   }
   
@@ -224,13 +224,14 @@ public class CloneTable extends MasterRepo {
   private static final long serialVersionUID = 1L;
   private CloneInfo cloneInfo;
   
-  public CloneTable(String user, String srcTableId, String tableName, Map<String,String> propertiesToSet, Set<String> propertiesToExclude, Instance inst) {
+  public CloneTable(String user, String srcTableId, String tableName, Map<String,String> propertiesToSet, Set<String> propertiesToExclude) {
     cloneInfo = new CloneInfo();
     cloneInfo.user = user;
     cloneInfo.srcTableId = srcTableId;
     cloneInfo.tableName = tableName;
     cloneInfo.propertiesToExclude = propertiesToExclude;
     cloneInfo.propertiesToSet = propertiesToSet;
+    Instance inst = HdfsZooInstance.getInstance();
     cloneInfo.srcNamespaceId = Tables.getNamespace(inst, cloneInfo.srcTableId);
   }
   
@@ -257,8 +258,8 @@ public class CloneTable extends MasterRepo {
   
   @Override
   public void undo(long tid, Master environment) throws Exception {
-    Utils.unreserveTable(cloneInfo.srcTableId, tid, false);
     Utils.unreserveTableNamespace(cloneInfo.srcNamespaceId, tid, false);
+    Utils.unreserveTable(cloneInfo.srcTableId, tid, false);
   }
   
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactRange.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactRange.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactRange.java
index 160fc7e..1c3b398 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactRange.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactRange.java
@@ -16,6 +16,10 @@
  */
 package org.apache.accumulo.master.tableOps;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
@@ -49,10 +53,10 @@ import org.apache.accumulo.master.Master;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.master.LiveTServerSet.TServerConnection;
 import org.apache.accumulo.server.master.state.TServerInstance;
-import org.apache.accumulo.server.master.tableOps.CompactionIterators;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.commons.codec.binary.Hex;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.log4j.Logger;
 import org.apache.thrift.TException;
@@ -66,6 +70,7 @@ class CompactionDriver extends MasterRepo {
   private String tableId;
   private byte[] startRow;
   private byte[] endRow;
+  private String namespaceId;
 
   public CompactionDriver(long compactId, String tableId, byte[] startRow, byte[] endRow) {
 
@@ -73,6 +78,8 @@ class CompactionDriver extends MasterRepo {
     this.tableId = tableId;
     this.startRow = startRow;
     this.endRow = endRow;
+    Instance inst = HdfsZooInstance.getInstance();
+    this.namespaceId = Tables.getNamespace(inst, tableId);
   }
 
   @Override
@@ -179,6 +186,7 @@ class CompactionDriver extends MasterRepo {
   public Repo<Master> call(long tid, Master environment) throws Exception {
     CompactRange.removeIterators(tid, tableId);
     Utils.getReadLock(tableId, tid).unlock();
+    Utils.getReadLock(namespaceId, tid).unlock();
     return null;
   }
 
@@ -196,11 +204,92 @@ public class CompactRange extends MasterRepo {
   private byte[] startRow;
   private byte[] endRow;
   private byte[] iterators;
+  private String namespaceId;
+
+  public static class CompactionIterators implements Writable {
+    byte[] startRow;
+    byte[] endRow;
+    List<IteratorSetting> iterators;
+
+    public CompactionIterators(byte[] startRow, byte[] endRow, List<IteratorSetting> iterators) {
+      this.startRow = startRow;
+      this.endRow = endRow;
+      this.iterators = iterators;
+    }
+
+    public CompactionIterators() {
+      startRow = null;
+      endRow = null;
+      iterators = Collections.emptyList();
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      out.writeBoolean(startRow != null);
+      if (startRow != null) {
+        out.writeInt(startRow.length);
+        out.write(startRow);
+      }
+
+      out.writeBoolean(endRow != null);
+      if (endRow != null) {
+        out.writeInt(endRow.length);
+        out.write(endRow);
+      }
+
+      out.writeInt(iterators.size());
+      for (IteratorSetting is : iterators) {
+        is.write(out);
+      }
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      if (in.readBoolean()) {
+        startRow = new byte[in.readInt()];
+        in.readFully(startRow);
+      } else {
+        startRow = null;
+      }
+
+      if (in.readBoolean()) {
+        endRow = new byte[in.readInt()];
+        in.readFully(endRow);
+      } else {
+        endRow = null;
+      }
+
+      int num = in.readInt();
+      iterators = new ArrayList<IteratorSetting>(num);
+
+      for (int i = 0; i < num; i++) {
+        iterators.add(new IteratorSetting(in));
+      }
+    }
+
+    public Text getEndRow() {
+      if (endRow == null)
+        return null;
+      return new Text(endRow);
+    }
+
+    public Text getStartRow() {
+      if (startRow == null)
+        return null;
+      return new Text(startRow);
+    }
+
+    public List<IteratorSetting> getIterators() {
+      return iterators;
+    }
+  }
 
   public CompactRange(String tableId, byte[] startRow, byte[] endRow, List<IteratorSetting> iterators) throws ThriftTableOperationException {
     this.tableId = tableId;
     this.startRow = startRow.length == 0 ? null : startRow;
     this.endRow = endRow.length == 0 ? null : endRow;
+    Instance inst = HdfsZooInstance.getInstance();
+    this.namespaceId = Tables.getNamespace(inst, tableId);
 
     if (iterators.size() > 0) {
       this.iterators = WritableUtils.toByteArray(new CompactionIterators(this.startRow, this.endRow, iterators));
@@ -215,7 +304,8 @@ public class CompactRange extends MasterRepo {
 
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    return Utils.reserveTable(tableId, tid, false, true, TableOperation.COMPACT);
+    return Utils.reserveTableNamespace(namespaceId, tid, false, true, TableOperation.COMPACT)
+        + Utils.reserveTable(tableId, tid, false, true, TableOperation.COMPACT);
   }
 
   @Override
@@ -297,6 +387,7 @@ public class CompactRange extends MasterRepo {
     try {
       removeIterators(tid, tableId);
     } finally {
+      Utils.unreserveTableNamespace(namespaceId, tid, false);
       Utils.unreserveTable(tableId, tid, false);
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/server/master/src/main/java/org/apache/accumulo/master/tableOps/CreateTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CreateTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CreateTable.java
index f657a1b..10eba61 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CreateTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CreateTable.java
@@ -35,6 +35,7 @@ import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.master.Master;
 import org.apache.accumulo.server.ServerConstants;
+import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.security.AuditedSecurityOperation;
 import org.apache.accumulo.server.security.SecurityOperation;
@@ -81,8 +82,8 @@ class FinishCreateTable extends MasterRepo {
   public Repo<Master> call(long tid, Master env) throws Exception {
     TableManager.getInstance().transitionTableState(tableInfo.tableId, TableState.ONLINE);
     
-    Utils.unreserveTable(tableInfo.tableId, tid, true);
     Utils.unreserveTableNamespace(tableInfo.namespaceId, tid, false);
+    Utils.unreserveTable(tableInfo.tableId, tid, true);
     
     env.getEventCoordinator().event("Created table %s ", tableInfo.tableName);
     
@@ -280,12 +281,13 @@ public class CreateTable extends MasterRepo {
   
   private TableInfo tableInfo;
   
-  public CreateTable(String user, String tableName, TimeType timeType, Map<String,String> props, Instance inst) throws TableNamespaceNotFoundException {
+  public CreateTable(String user, String tableName, TimeType timeType, Map<String,String> props) throws TableNamespaceNotFoundException {
     tableInfo = new TableInfo();
     tableInfo.tableName = tableName;
     tableInfo.timeType = TabletTime.getTimeID(timeType);
     tableInfo.user = user;
     tableInfo.props = props;
+    Instance inst = HdfsZooInstance.getInstance();
     tableInfo.namespaceId = TableNamespaces.getNamespaceId(inst, Tables.extractNamespace(tableInfo.tableName));
   }
   
@@ -296,7 +298,7 @@ public class CreateTable extends MasterRepo {
   }
   
   @Override
-  public Repo<Master> call(long tid, Master master) throws Exception {   
+  public Repo<Master> call(long tid, Master master) throws Exception {
     // first step is to reserve a table id.. if the machine fails during this step
     // it is ok to retry... the only side effect is that a table id may not be used
     // or skipped

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteTable.java
index 98d1ae9..bf1d9ce 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteTable.java
@@ -41,6 +41,7 @@ import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.master.Master;
 import org.apache.accumulo.server.ServerConstants;
+import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.master.state.MetaDataTableScanner;
 import org.apache.accumulo.server.master.state.TabletLocationState;
@@ -221,8 +222,9 @@ public class DeleteTable extends MasterRepo {
   
   private String tableId, namespaceId;
   
-  public DeleteTable(String tableId, Instance inst) {
+  public DeleteTable(String tableId) {
     this.tableId = tableId;
+    Instance inst = HdfsZooInstance.getInstance();
     this.namespaceId = Tables.getNamespace(inst, tableId);
   }
   
@@ -242,8 +244,8 @@ public class DeleteTable extends MasterRepo {
   
   @Override
   public void undo(long tid, Master environment) throws Exception {
-    Utils.unreserveTable(tableId, tid, true);
     Utils.unreserveTableNamespace(namespaceId, tid, false);
+    Utils.unreserveTable(tableId, tid, true);
   }
   
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/server/master/src/main/java/org/apache/accumulo/master/tableOps/ExportTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ExportTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ExportTable.java
index 22df3b3..cad779c 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ExportTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ExportTable.java
@@ -32,6 +32,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.Connector;
+import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.Tables;
@@ -53,6 +54,7 @@ import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.master.Master;
 import org.apache.accumulo.server.ServerConstants;
+import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -67,6 +69,7 @@ class ExportInfo implements Serializable {
   public String tableName;
   public String tableID;
   public String exportDir;
+  public String namespaceID;
 }
 
 class WriteExportFiles extends MasterRepo {
@@ -91,7 +94,8 @@ class WriteExportFiles extends MasterRepo {
   @Override
   public long isReady(long tid, Master master) throws Exception {
     
-    long reserved = Utils.reserveTable(tableInfo.tableID, tid, false, true, TableOperation.EXPORT);
+    long reserved = Utils.reserveTableNamespace(tableInfo.namespaceID, tid, false, true, TableOperation.EXPORT)
+        + Utils.reserveTable(tableInfo.tableID, tid, false, true, TableOperation.EXPORT);
     if (reserved > 0)
       return reserved;
     
@@ -133,6 +137,7 @@ class WriteExportFiles extends MasterRepo {
       throw new ThriftTableOperationException(tableInfo.tableID, tableInfo.tableName, TableOperation.EXPORT, TableOperationExceptionType.OTHER,
           "Failed to create export files " + ioe.getMessage());
     }
+    Utils.unreserveTableNamespace(tableInfo.namespaceID, tid, false);
     Utils.unreserveTable(tableInfo.tableID, tid, false);
     Utils.unreserveHdfsDirectory(new Path(tableInfo.exportDir).toString(), tid);
     return null;
@@ -140,6 +145,7 @@ class WriteExportFiles extends MasterRepo {
   
   @Override
   public void undo(long tid, Master env) throws Exception {
+    Utils.unreserveTableNamespace(tableInfo.namespaceID, tid, false);
     Utils.unreserveTable(tableInfo.tableID, tid, false);
   }
   
@@ -283,6 +289,8 @@ public class ExportTable extends MasterRepo {
     tableInfo.tableName = tableName;
     tableInfo.exportDir = exportDir;
     tableInfo.tableID = tableId;
+    Instance inst = HdfsZooInstance.getInstance();
+    tableInfo.namespaceID = Tables.getNamespace(inst, tableId);
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
index 4032ce5..4405387 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
@@ -83,6 +83,7 @@ class ImportedTableInfo implements Serializable {
   public String tableName;
   public String tableId;
   public String importDir;
+  public String namespaceId;
 }
 
 class FinishImportTable extends MasterRepo {
@@ -107,6 +108,7 @@ class FinishImportTable extends MasterRepo {
     
     TableManager.getInstance().transitionTableState(tableInfo.tableId, TableState.ONLINE);
     
+    Utils.unreserveTableNamespace(tableInfo.namespaceId, tid, false);
     Utils.unreserveTable(tableInfo.tableId, tid, true);
     
     Utils.unreserveHdfsDirectory(new Path(tableInfo.exportDir).toString(), tid);
@@ -417,7 +419,7 @@ class ImportPopulateZookeeper extends MasterRepo {
   
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    return Utils.reserveTable(tableInfo.tableId, tid, true, false, TableOperation.IMPORT);
+    return Utils.reserveTableNamespace(tableInfo.namespaceId, tid, false, true, TableOperation.IMPORT) + Utils.reserveTable(tableInfo.tableId, tid, true, false, TableOperation.IMPORT);
   }
   
   private Map<String,String> getExportedProps(VolumeManager fs) throws Exception {
@@ -468,6 +470,7 @@ class ImportPopulateZookeeper extends MasterRepo {
   public void undo(long tid, Master env) throws Exception {
     Instance instance = HdfsZooInstance.getInstance();
     TableManager.getInstance().removeTable(tableInfo.tableId);
+    Utils.unreserveTableNamespace(tableInfo.namespaceId, tid, false);
     Utils.unreserveTable(tableInfo.tableId, tid, true);
     Tables.clearCache(instance);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/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 871c0a0..c34d082 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
@@ -28,6 +28,7 @@ import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter.Mutator;
 import org.apache.accumulo.master.Master;
+import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.tables.TableManager;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.log4j.Logger;
@@ -46,10 +47,11 @@ public class RenameTable extends MasterRepo {
         + Utils.reserveTable(tableId, tid, true, true, TableOperation.RENAME);
   }
 
-  public RenameTable(String tableId, String oldTableName, String newTableName, Instance inst) {
+  public RenameTable(String tableId, String oldTableName, String newTableName) {
     this.tableId = tableId;
     this.oldTableName = oldTableName;
     this.newTableName = newTableName;
+    Instance inst = HdfsZooInstance.getInstance();
     this.namespaceId = Tables.getNamespace(inst, tableId);
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTableNamespace.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTableNamespace.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTableNamespace.java
index 2da5d9f..9662fbb 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTableNamespace.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTableNamespace.java
@@ -75,7 +75,7 @@ public class RenameTableNamespace extends MasterRepo {
       Tables.clearCache(instance);
     } finally {
       Utils.tableNameLock.unlock();
-      Utils.unreserveTable(namespaceId, id, true);
+      Utils.unreserveTableNamespace(namespaceId, id, true);
     }
 
     Logger.getLogger(RenameTableNamespace.class).debug("Renamed table namespace " + namespaceId + " " + oldName + " " + newName);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableRangeOp.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableRangeOp.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableRangeOp.java
index 0ad2196..77a6606 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableRangeOp.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableRangeOp.java
@@ -16,6 +16,8 @@
  */
 package org.apache.accumulo.master.tableOps;
 
+import org.apache.accumulo.core.client.Instance;
+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;
@@ -24,9 +26,10 @@ import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.master.Master;
+import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.master.state.MergeInfo;
-import org.apache.accumulo.server.master.state.MergeState;
 import org.apache.accumulo.server.master.state.MergeInfo.Operation;
+import org.apache.accumulo.server.master.state.MergeState;
 import org.apache.hadoop.io.Text;
 
 /**
@@ -43,14 +46,17 @@ import org.apache.hadoop.io.Text;
  * and prevent merges (writes) while they run. Merge operations will lock out some operations while they run.
  */
 class TableRangeOpWait extends MasterRepo {
-  
+
   private static final long serialVersionUID = 1L;
   private String tableId;
-  
+  private String namespaceId;
+
   public TableRangeOpWait(String tableId) {
     this.tableId = tableId;
+    Instance inst = HdfsZooInstance.getInstance();
+    this.namespaceId = Tables.getNamespace(inst, tableId);
   }
-  
+
   @Override
   public long isReady(long tid, Master env) throws Exception {
     Text tableIdText = new Text(tableId);
@@ -59,69 +65,74 @@ class TableRangeOpWait extends MasterRepo {
     }
     return 0;
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master master) throws Exception {
     Text tableIdText = new Text(tableId);
     MergeInfo mergeInfo = master.getMergeInfo(tableIdText);
     log.info("removing merge information " + mergeInfo);
     master.clearMergeState(tableIdText);
+    Utils.unreserveTableNamespace(namespaceId, tid, false);
     Utils.unreserveTable(tableId, tid, true);
     return null;
   }
-  
+
 }
 
 public class TableRangeOp extends MasterRepo {
-  
+
   private static final long serialVersionUID = 1L;
-  
+
   private String tableId;
   private byte[] startRow;
   private byte[] endRow;
   private Operation op;
-  
+  private String namespaceId;
+
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    return Utils.reserveTable(tableId, tid, true, true, TableOperation.MERGE);
+    return Utils.reserveTableNamespace(namespaceId, tid, false, true, TableOperation.MERGE)
+        + Utils.reserveTable(tableId, tid, true, true, TableOperation.MERGE);
   }
-  
+
   public TableRangeOp(MergeInfo.Operation op, String tableId, Text startRow, Text endRow) throws ThriftTableOperationException {
-    
+
     this.tableId = tableId;
     this.startRow = TextUtil.getBytes(startRow);
     this.endRow = TextUtil.getBytes(endRow);
     this.op = op;
+    Instance inst = HdfsZooInstance.getInstance();
+    this.namespaceId = Tables.getNamespace(inst, tableId);
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master env) throws Exception {
-    
+
     if (RootTable.ID.equals(tableId) && TableOperation.MERGE.equals(op)) {
       log.warn("Attempt to merge tablets for " + RootTable.NAME + " does nothing. It is not splittable.");
     }
-    
+
     Text start = startRow.length == 0 ? null : new Text(startRow);
     Text end = endRow.length == 0 ? null : new Text(endRow);
     Text tableIdText = new Text(tableId);
-    
+
     if (start != null && end != null)
       if (start.compareTo(end) >= 0)
         throw new ThriftTableOperationException(tableId, null, TableOperation.MERGE, TableOperationExceptionType.BAD_RANGE,
             "start row must be less than end row");
-    
+
     env.mustBeOnline(tableId);
-    
+
     MergeInfo info = env.getMergeInfo(tableIdText);
-    
+
     if (info.getState() == MergeState.NONE) {
       KeyExtent range = new KeyExtent(tableIdText, end, start);
       env.setMergeState(new MergeInfo(range, op), MergeState.STARTED);
     }
-    
+
     return new TableRangeOpWait(tableId);
   }
-  
+
   @Override
   public void undo(long tid, Master env) throws Exception {
     // Not sure this is a good thing to do. The Master state engine should be the one to remove it.
@@ -130,7 +141,8 @@ public class TableRangeOp extends MasterRepo {
     if (mergeInfo.getState() != MergeState.NONE)
       log.info("removing merge information " + mergeInfo);
     env.clearMergeState(tableIdText);
+    Utils.unreserveTableNamespace(namespaceId, tid, false);
     Utils.unreserveTable(tableId, tid, true);
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Merge.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Merge.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Merge.java
index 8fcfab5..a16f6a6 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Merge.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Merge.java
@@ -42,7 +42,7 @@ public class Merge extends Test {
     tableNames = new ArrayList<String>(tableNames);
     tableNames.add(MetadataTable.NAME);
     String tableName = tableNames.get(rand.nextInt(tableNames.size()));
-        
+    
     List<Text> range = ConcurrentFixture.generateRange(rand);
     
     try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java
index 01329d0..fdd4cef 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java
@@ -35,7 +35,7 @@ public class RenameTableNamespace extends Test {
     Random rand = (Random) state.get("rand");
     
     @SuppressWarnings("unchecked")
-    List<String> namespaces = (List<String>) state.get("tables");
+    List<String> namespaces = (List<String>) state.get("namespaces");
     
     String srcName = namespaces.get(rand.nextInt(namespaces.size()));
     String newName = namespaces.get(rand.nextInt(namespaces.size()));