You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by bu...@apache.org on 2014/04/05 02:59:36 UTC

[01/15] git commit: ACCUMULO-2519 Aborts upgrade if there are Fate transactions from an old version.

Repository: accumulo
Updated Branches:
  refs/heads/1.5.2-SNAPSHOT f67c38614 -> 5a504b311
  refs/heads/1.6.0-SNAPSHOT 2cb526e5e -> f5a94f041
  refs/heads/master bdc2a994c -> ebbd5e62b


ACCUMULO-2519 Aborts upgrade if there are Fate transactions from an old version.


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

Branch: refs/heads/1.5.2-SNAPSHOT
Commit: 5a504b311c0e5f59ff5b14221c6bf61f43b4d093
Parents: a904f69
Author: Sean Busbey <bu...@cloudera.com>
Authored: Fri Mar 28 01:46:09 2014 -0500
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Apr 4 17:27:05 2014 -0700

----------------------------------------------------------------------
 README                                          |  14 +++
 .../org/apache/accumulo/server/Accumulo.java    |  31 ++++++
 .../apache/accumulo/server/master/Master.java   | 100 ++++++++++++-------
 .../server/tabletserver/TabletServer.java       |   5 +
 .../accumulo/server/util/MetadataTable.java     |   3 +
 5 files changed, 116 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/5a504b31/README
----------------------------------------------------------------------
diff --git a/README b/README
index 115a9b7..0bb1030 100644
--- a/README
+++ b/README
@@ -54,12 +54,26 @@ accumulo.
 
  This happens automatically the first time Accumulo 1.5 is started.  
 
+  * Verify that there are no outstanding FATE operations
+    - Under 1.4 you can list what's in FATE by running
+      $ACCUMULO_HOME/bin/accumulo org.apache.accumulo.server.fate.Admin print
+    - Note that operations in any state will prevent an upgrade. It is safe
+      to delete operations with status SUCCESSFUL. For others, you should restart
+      your 1.4 cluster and allow them to finish.
   * Stop the 1.4 instance.  
   * Configure 1.5 to use the hdfs directory, walog directories, and zookeepers
     that 1.4 was using.
   * Copy other 1.4 configuration options as needed.
   * Start Accumulo 1.5. 
 
+  The upgrade process must make changes to Accumulo's internal state in both ZooKeeper and
+  the table metadata. This process may take some time as Tablet Servers move write-ahead
+  logs to HDFS and then do recovery. During this time, the Monitor will claim that the
+  Master is down and some services may send the Monitor log messages about failure to
+  communicate with each other. These messages are safe to ignore. If you need detail on
+  the upgrade's progress you should view the local logs on the Tablet Servers and active
+  Master.
+
 ******************************************************************************
 4. Configuring
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5a504b31/server/src/main/java/org/apache/accumulo/server/Accumulo.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/Accumulo.java b/server/src/main/java/org/apache/accumulo/server/Accumulo.java
index 99ec7e4..420b6cc 100644
--- a/server/src/main/java/org/apache/accumulo/server/Accumulo.java
+++ b/server/src/main/java/org/apache/accumulo/server/Accumulo.java
@@ -27,11 +27,16 @@ import java.util.Map.Entry;
 import java.util.TreeMap;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.trace.DistributedTrace;
 import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.util.Version;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.ReadOnlyTStore;
+import org.apache.accumulo.fate.ReadOnlyStore;
+import org.apache.accumulo.fate.ZooStore;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.util.time.SimpleTimer;
@@ -53,6 +58,7 @@ public class Accumulo {
     try {
       if (getAccumuloPersistentVersion(fs) == Constants.PREV_DATA_VERSION) {
         fs.create(new Path(ServerConstants.getDataVersionLocation() + "/" + Constants.DATA_VERSION));
+        // TODO document failure mode & recovery if FS permissions cause above to work and below to fail ACCUMULO-2596
         fs.delete(new Path(ServerConstants.getDataVersionLocation() + "/" + Constants.PREV_DATA_VERSION), false);
       }
     } catch (IOException e) {
@@ -263,4 +269,29 @@ public class Accumulo {
       throw new RuntimeException("cannot find method setSafeMode", ex);
     }
   }
+
+  /**
+   * Exit loudly if there are outstanding Fate operations.
+   * Since Fate serializes class names, we need to make sure there are no queued
+   * transactions from a previous version before continuing an upgrade. The status of the operations is
+   * irrelevant; those in SUCCESSFUL status cause the same problem as those just queued.
+   *
+   * Note that the Master should not allow write access to Fate until after all upgrade steps are complete.
+   *
+   * Should be called as a guard before performing any upgrade steps, after determining that an upgrade is needed.
+   *
+   * see ACCUMULO-2519
+   */
+  public static void abortIfFateTransactions() {
+    try {
+      final ReadOnlyTStore<Accumulo> fate = new ReadOnlyStore<Accumulo>(new ZooStore<Accumulo>(ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZFATE,
+          ZooReaderWriter.getRetryingInstance()));
+      if (!(fate.list().isEmpty())) {
+        throw new AccumuloException("Aborting upgrade because there are outstanding FATE transactions from a previous Accumulo version. Please see the README document for instructions on what to do under your previous version.");
+      }
+    } catch (Exception exception) {
+      log.fatal("Problem verifying Fate readiness", exception);
+      System.exit(1);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5a504b31/server/src/main/java/org/apache/accumulo/server/master/Master.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/Master.java b/server/src/main/java/org/apache/accumulo/server/master/Master.java
index 270eb18..a2ad2e6 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/Master.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/Master.java
@@ -34,6 +34,7 @@ import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -271,7 +272,9 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       upgradeMetadata();
     }
   }
-  
+
+  private boolean haveUpgradedZooKeeper = false;
+
   private void upgradeZookeeper() {
     // 1.5.1 and 1.6.0 both do some state checking after obtaining the zoolock for the
     // monitor and before starting up. It's not tied to the data version at all (and would
@@ -279,59 +282,79 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
     // that the master is not the only thing that may alter zookeeper before starting.
 
     if (Accumulo.getAccumuloPersistentVersion(fs) == Constants.PREV_DATA_VERSION) {
+      // This Master hasn't started Fate yet, so any outstanding transactions must be from before the upgrade.
+      // Change to Guava's Verify once we use Guava 17.
+      if (null != fate) {
+        throw new IllegalStateException("Access to Fate should not have been initialized prior to the Master transitioning to active. Please save all logs and file a bug.");
+      }
+      Accumulo.abortIfFateTransactions();
       try {
         log.info("Upgrading zookeeper");
-        
+
         IZooReaderWriter zoo = ZooReaderWriter.getInstance();
-        
+
         zoo.recursiveDelete(ZooUtil.getRoot(instance) + "/loggers", NodeMissingPolicy.SKIP);
         zoo.recursiveDelete(ZooUtil.getRoot(instance) + "/dead/loggers", NodeMissingPolicy.SKIP);
 
         zoo.putPersistentData(ZooUtil.getRoot(instance) + Constants.ZRECOVERY, new byte[] {'0'}, NodeExistsPolicy.SKIP);
-        
+
         for (String id : Tables.getIdToNameMap(instance).keySet()) {
-          
+
           zoo.putPersistentData(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + id + Constants.ZTABLE_COMPACT_CANCEL_ID, "0".getBytes(Constants.UTF8),
               NodeExistsPolicy.SKIP);
         }
+        haveUpgradedZooKeeper = true;
       } catch (Exception ex) {
         log.fatal("Error performing upgrade", ex);
         System.exit(1);
       }
     }
   }
-  
+
   private final AtomicBoolean upgradeMetadataRunning = new AtomicBoolean(false);
-  
+  private final CountDownLatch waitForMetadataUpgrade = new CountDownLatch(1);
+
   private final ServerConfiguration serverConfig;
   
   private void upgradeMetadata() {
-    if (Accumulo.getAccumuloPersistentVersion(fs) == Constants.PREV_DATA_VERSION) {
-      if (upgradeMetadataRunning.compareAndSet(false, true)) {
+    // we make sure we're only doing the rest of this method once so that we can signal to other threads that an upgrade wasn't needed.
+    if (upgradeMetadataRunning.compareAndSet(false, true)) {
+      if (Accumulo.getAccumuloPersistentVersion(fs) == Constants.PREV_DATA_VERSION) {
+        // sanity check that we passed the Fate verification prior to ZooKeeper upgrade, and that Fate still hasn't been started.
+        // Change both to use Guava's Verify once we use Guava 17.
+        if (!haveUpgradedZooKeeper) {
+          throw new IllegalStateException("We should only attempt to upgrade Accumulo's !METADATA table if we've already upgraded ZooKeeper. Please save all logs and file a bug.");
+        }
+        if (null != fate) {
+          throw new IllegalStateException("Access to Fate should not have been initialized prior to the Master finishing upgrades. Please save all logs and file a bug.");
+        }
         Runnable upgradeTask = new Runnable() {
           @Override
           public void run() {
             try {
+              log.info("Starting to upgrade !METADATA table.");
               MetadataTable.moveMetaDeleteMarkers(instance, SecurityConstants.getSystemCredentials());
+              log.info("Updating persistent data version.");
               Accumulo.updateAccumuloVersion(fs);
-              
               log.info("Upgrade complete");
-              
+              waitForMetadataUpgrade.countDown();
             } catch (Exception ex) {
               log.fatal("Error performing upgrade", ex);
               System.exit(1);
             }
-            
+
           }
         };
-        
+
         // need to run this in a separate thread because a lock is held that prevents !METADATA tablets from being assigned and this task writes to the
         // !METADATA table
         new Thread(upgradeTask).start();
+      } else {
+        waitForMetadataUpgrade.countDown();
       }
     }
   }
-  
+
   private int assignedOrHosted(Text tableId) {
     int result = 0;
     for (TabletGroupWatcher watcher : watchers) {
@@ -2136,28 +2159,6 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
     
     tserverSet.startListeningForTabletServerChanges();
     
-    // TODO: add shutdown for fate object - ACCUMULO-1307
-    try {
-      final AgeOffStore<Master> store = new AgeOffStore<Master>(new org.apache.accumulo.fate.ZooStore<Master>(ZooUtil.getRoot(instance) + Constants.ZFATE,
-          ZooReaderWriter.getRetryingInstance()), 1000 * 60 * 60 * 8);
-      
-      int threads = this.getConfiguration().getConfiguration().getCount(Property.MASTER_FATE_THREADPOOL_SIZE);
-      
-      fate = new Fate<Master>(this, store, threads);
-      
-      SimpleTimer.getInstance().schedule(new Runnable() {
-        
-        @Override
-        public void run() {
-          store.ageOff();
-        }
-      }, 63000, 63000);
-    } catch (KeeperException e) {
-      throw new IOException(e);
-    } catch (InterruptedException e) {
-      throw new IOException(e);
-    }
-    
     ZooReaderWriter.getInstance().getChildren(zroot + Constants.ZRECOVERY, new Watcher() {
       @Override
       public void process(WatchedEvent event) {
@@ -2183,7 +2184,32 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
     for (TabletGroupWatcher watcher : watchers) {
       watcher.start();
     }
-    
+
+    // Once we are sure tablet servers are no longer checking for an empty Fate transaction queue before doing WAL upgrades, we can safely start using Fate ourselves.
+    waitForMetadataUpgrade.await();
+
+    // TODO: add shutdown for fate object - ACCUMULO-1307
+    try {
+      final AgeOffStore<Master> store = new AgeOffStore<Master>(new org.apache.accumulo.fate.ZooStore<Master>(ZooUtil.getRoot(instance) + Constants.ZFATE,
+          ZooReaderWriter.getRetryingInstance()), 1000 * 60 * 60 * 8);
+
+      int threads = this.getConfiguration().getConfiguration().getCount(Property.MASTER_FATE_THREADPOOL_SIZE);
+
+      fate = new Fate<Master>(this, store, threads);
+
+      SimpleTimer.getInstance().schedule(new Runnable() {
+
+        @Override
+        public void run() {
+          store.ageOff();
+        }
+      }, 63000, 63000);
+    } catch (KeeperException e) {
+      throw new IOException(e);
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
+
     Processor<Iface> processor = new Processor<Iface>(TraceWrap.service(new MasterClientServiceHandler()));
     ServerPort serverPort = TServerUtils.startServer(getSystemConfiguration(), Property.MASTER_CLIENTPORT, processor, "Master",
         "Master Client Service Handler", null, Property.MASTER_MINTHREADS, Property.MASTER_THREADCHECK, Property.GENERAL_MAX_MESSAGE_SIZE);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5a504b31/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java b/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
index d76946d..ad3d615 100644
--- a/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
+++ b/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
@@ -3322,6 +3322,11 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
    * 
    */
   public static void recoverLocalWriteAheadLogs(FileSystem fs, ServerConfiguration serverConf) throws IOException {
+    if (Accumulo.getAccumuloPersistentVersion(fs) == Constants.PREV_DATA_VERSION) {
+      // If the Master has not yet signaled a finish to upgrading, we need to make sure we can rollback in the
+      // event of outstanding transactions in Fate from the previous version.
+      Accumulo.abortIfFateTransactions();
+    }
     FileSystem localfs = FileSystem.getLocal(fs.getConf()).getRawFileSystem();
     AccumuloConfiguration conf = serverConf.getConfiguration();
     String localWalDirectories = conf.get(Property.LOGGER_DIR);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5a504b31/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java b/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
index 7328a55..d6e0a3c 100644
--- a/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
+++ b/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
@@ -1233,6 +1233,9 @@ public class MetadataTable extends org.apache.accumulo.core.util.MetadataTable {
     update(SecurityConstants.getSystemCredentials(), m);
   }
 
+  /**
+   * During an upgrade from Accumulo 1.4 -> 1.5, we need to move deletion requests for files under the !METADATA table to the root tablet.
+   */
   public static void moveMetaDeleteMarkers(Instance instance, TCredentials creds) {
     // move delete markers from the normal delete keyspace to the root tablet delete keyspace if the files are for the !METADATA table
     Scanner scanner = new ScannerImpl(instance, creds, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);


[08/15] git commit: Merge branch '1.5.2-SNAPSHOT' into 1.6.0-SNAPSHOT

Posted by bu...@apache.org.
Merge branch '1.5.2-SNAPSHOT' into 1.6.0-SNAPSHOT

  Conflicts:
      fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
      fate/src/main/java/org/apache/accumulo/fate/TStore.java
      server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
      server/src/main/java/org/apache/accumulo/server/master/Master.java
      server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
      server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java


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

Branch: refs/heads/master
Commit: 957c9d1b34afb8a5b7347921ba2d5b0b22bbc828
Parents: 2cb526e 5a504b3
Author: Sean Busbey <bu...@cloudera.com>
Authored: Fri Apr 4 17:27:39 2014 -0700
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Apr 4 17:28:32 2014 -0700

----------------------------------------------------------------------
 README                                          |  14 +++
 fate/pom.xml                                    |   9 ++
 .../org/apache/accumulo/fate/AdminUtil.java     |  14 +--
 .../java/org/apache/accumulo/fate/Fate.java     |   2 +-
 .../org/apache/accumulo/fate/ReadOnlyRepo.java  |  32 +++++
 .../org/apache/accumulo/fate/ReadOnlyStore.java | 111 ++++++++++++++++
 .../apache/accumulo/fate/ReadOnlyTStore.java    | 125 +++++++++++++++++++
 .../java/org/apache/accumulo/fate/Repo.java     |   5 +-
 .../java/org/apache/accumulo/fate/TStore.java   |  62 +--------
 .../apache/accumulo/fate/AgeOffStoreTest.java   |   2 +-
 .../apache/accumulo/fate/ReadOnlyStoreTest.java |  72 +++++++++++
 .../org/apache/accumulo/fate/SimpleStore.java   |   2 +-
 .../org/apache/accumulo/server/Accumulo.java    |  33 ++++-
 .../accumulo/server/util/MetadataTableUtil.java |   3 +
 .../java/org/apache/accumulo/master/Master.java |  77 ++++++++----
 .../apache/accumulo/master/util/FateAdmin.java  |   3 +-
 16 files changed, 464 insertions(+), 102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/957c9d1b/README
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/957c9d1b/fate/pom.xml
----------------------------------------------------------------------
diff --cc fate/pom.xml
index b339758,0868e4c..a3a7b42
--- a/fate/pom.xml
+++ b/fate/pom.xml
@@@ -24,11 -24,15 +24,15 @@@
    </parent>
    <artifactId>accumulo-fate</artifactId>
    <name>Fate</name>
 +  <description>A FAult-Tolerant Executor library used by Apache Accumulo.</description>
    <dependencies>
      <dependency>
+       <groupId>com.google.guava</groupId>
+       <artifactId>guava</artifactId>
+     </dependency>
+     <dependency>
        <groupId>commons-lang</groupId>
        <artifactId>commons-lang</artifactId>
 -      <scope>provided</scope>
      </dependency>
      <dependency>
        <groupId>log4j</groupId>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/957c9d1b/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
----------------------------------------------------------------------
diff --cc fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
index 8277c71,0238fde..1f8c32d
--- a/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
@@@ -25,9 -23,8 +25,9 @@@ import java.util.HashMap
  import java.util.List;
  import java.util.Map;
  import java.util.Map.Entry;
 +import java.util.Set;
  
- import org.apache.accumulo.fate.TStore.TStatus;
+ import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
  import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
  import org.apache.accumulo.fate.zookeeper.ZooLock;
  import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
@@@ -39,32 -36,7 +39,32 @@@ import org.apache.zookeeper.KeeperExcep
  public class AdminUtil<T> {
    private static final Charset UTF8 = Charset.forName("UTF-8");
    
 +  private boolean exitOnError = false;
 +  
 +  /**
 +   * Default constructor
 +   */
 +  public AdminUtil() {
 +    this(true);
 +  }
 +  
 +  /**
 +   * Constructor
 +   * 
 +   * @param exitOnError
 +   *          <code>System.exit(1)</code> on error if true
 +   */
 +  public AdminUtil(boolean exitOnError) {
 +    super();
 +    this.exitOnError = exitOnError;
 +  }
 +  
-   public void print(ZooStore<T> zs, IZooReaderWriter zk, String lockPath) throws KeeperException, InterruptedException {
+   public void print(ReadOnlyTStore<T> zs, IZooReaderWriter zk, String lockPath) throws KeeperException, InterruptedException {
 +    print(zs, zk, lockPath, new Formatter(System.out), null, null);
 +  }
 +  
-   public void print(ZooStore<T> zs, IZooReaderWriter zk, String lockPath, Formatter fmt, Set<Long> filterTxid, EnumSet<TStatus> filterStatus)
++  public void print(ReadOnlyTStore<T> zs, IZooReaderWriter zk, String lockPath, Formatter fmt, Set<Long> filterTxid, EnumSet<TStatus> filterStatus)
 +      throws KeeperException, InterruptedException {
      Map<Long,List<String>> heldLocks = new HashMap<Long,List<String>>();
      Map<Long,List<String>> waitingLocks = new HashMap<Long,List<String>>();
      
@@@ -169,79 -137,28 +169,79 @@@
        return false;
      }
      
 -    long txid = Long.parseLong(txidStr, 16);
 +    long txid;
 +    try {
 +      txid = Long.parseLong(txidStr, 16);
 +    } catch (NumberFormatException nfe) {
 +      System.out.printf("Invalid transaction ID format: %s%n", txidStr);
 +      return false;
 +    }
 +    boolean state = false;
      zs.reserve(txid);
 -    zs.delete(txid);
 -    zs.unreserve(txid, 0);
 +    TStatus ts = zs.getStatus(txid);
 +    switch (ts) {
 +      case UNKNOWN:
 +        System.out.printf("Invalid transaction ID: %016x%n", txid);
 +        break;
 +      
 +      case IN_PROGRESS:
 +      case NEW:
 +      case FAILED:
 +      case FAILED_IN_PROGRESS:
 +      case SUCCESSFUL:
 +        System.out.printf("Deleting transaction: %016x (%s)%n", txid, ts);
 +        zs.delete(txid);
 +        state = true;
 +        break;
 +    }
      
 -    return true;
 +    zs.unreserve(txid, 0);
 +    return state;
    }
    
-   public boolean prepFail(ZooStore<T> zs, IZooReaderWriter zk, String path, String txidStr) {
+   public boolean prepFail(TStore<T> zs, IZooReaderWriter zk, String path, String txidStr) {
      if (!checkGlobalLock(zk, path)) {
        return false;
      }
      
 -    long txid = Long.parseLong(txidStr, 16);
 +    long txid;
 +    try {
 +      txid = Long.parseLong(txidStr, 16);
 +    } catch (NumberFormatException nfe) {
 +      System.out.printf("Invalid transaction ID format: %s%n", txidStr);
 +      return false;
 +    }
 +    boolean state = false;
      zs.reserve(txid);
 -    zs.setStatus(txid, TStatus.FAILED_IN_PROGRESS);
 -    zs.unreserve(txid, 0);
 +    TStatus ts = zs.getStatus(txid);
 +    switch (ts) {
 +      case UNKNOWN:
 +        System.out.printf("Invalid transaction ID: %016x%n", txid);
 +        break;
 +      
 +      case IN_PROGRESS:
 +      case NEW:
 +        System.out.printf("Failing transaction: %016x (%s)%n", txid, ts);
 +        zs.setStatus(txid, TStatus.FAILED_IN_PROGRESS);
 +        state = true;
 +        break;
 +      
 +      case SUCCESSFUL:
 +        System.out.printf("Transaction already completed: %016x (%s)%n", txid, ts);
 +        break;
 +      
 +      case FAILED:
 +      case FAILED_IN_PROGRESS:
 +        System.out.printf("Transaction already failed: %016x (%s)%n", txid, ts);
 +        state = true;
 +        break;
 +    }
      
 -    return true;
 +    zs.unreserve(txid, 0);
 +    return state;
    }
    
-   public void deleteLocks(ZooStore<T> zs, IZooReaderWriter zk, String path, String txidStr) throws KeeperException, InterruptedException {
+   public void deleteLocks(TStore<T> zs, IZooReaderWriter zk, String path, String txidStr) throws KeeperException, InterruptedException {
      // delete any locks assoc w/ fate operation
      List<String> lockedIds = zk.getChildren(path);
      

http://git-wip-us.apache.org/repos/asf/accumulo/blob/957c9d1b/fate/src/main/java/org/apache/accumulo/fate/Fate.java
----------------------------------------------------------------------
diff --cc fate/src/main/java/org/apache/accumulo/fate/Fate.java
index 3561fc8,b2eb681..2b232ac
--- a/fate/src/main/java/org/apache/accumulo/fate/Fate.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/Fate.java
@@@ -17,9 -17,8 +17,9 @@@
  package org.apache.accumulo.fate;
  
  import java.util.EnumSet;
 +import java.util.concurrent.atomic.AtomicBoolean;
  
- import org.apache.accumulo.fate.TStore.TStatus;
+ import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
  import org.apache.accumulo.fate.util.Daemon;
  import org.apache.accumulo.fate.util.LoggingRunnable;
  import org.apache.log4j.Logger;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/957c9d1b/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
----------------------------------------------------------------------
diff --cc server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
index a9e4b5f,0000000..4e1eb35
mode 100644,000000..100644
--- a/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
@@@ -1,242 -1,0 +1,271 @@@
 +/*
 + * 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.server;
 +
 +import java.io.File;
 +import java.io.FileInputStream;
 +import java.io.IOException;
 +import java.io.InputStream;
 +import java.net.InetAddress;
 +import java.net.UnknownHostException;
 +import java.util.Map.Entry;
 +import java.util.TreeMap;
 +
 +import org.apache.accumulo.core.Constants;
++import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.trace.DistributedTrace;
 +import org.apache.accumulo.core.util.AddressUtil;
 +import org.apache.accumulo.core.util.UtilWaitThread;
 +import org.apache.accumulo.core.util.Version;
 +import org.apache.accumulo.core.volume.Volume;
++import org.apache.accumulo.core.zookeeper.ZooUtil;
++import org.apache.accumulo.fate.ReadOnlyTStore;
++import org.apache.accumulo.fate.ReadOnlyStore;
++import org.apache.accumulo.fate.ZooStore;
 +import org.apache.accumulo.server.client.HdfsZooInstance;
 +import org.apache.accumulo.server.conf.ServerConfiguration;
 +import org.apache.accumulo.server.fs.VolumeManager;
 +import org.apache.accumulo.server.util.time.SimpleTimer;
 +import org.apache.accumulo.server.watcher.MonitorLog4jWatcher;
 +import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 +import org.apache.hadoop.fs.FileStatus;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.log4j.Logger;
 +import org.apache.log4j.helpers.LogLog;
 +import org.apache.log4j.xml.DOMConfigurator;
 +import org.apache.zookeeper.KeeperException;
 +
 +public class Accumulo {
 +  
 +  private static final Logger log = Logger.getLogger(Accumulo.class);
 +  
 +  public static synchronized void updateAccumuloVersion(VolumeManager fs) {
 +    for (Volume volume : fs.getVolumes()) {
 +      try {
 +        if (getAccumuloPersistentVersion(fs) == ServerConstants.PREV_DATA_VERSION) {
 +          log.debug("Attempting to upgrade " + volume);
 +          Path dataVersionLocation = ServerConstants.getDataVersionLocation(volume);
 +          fs.create(new Path(dataVersionLocation, Integer.toString(ServerConstants.DATA_VERSION))).close();
- 
++          // TODO document failure mode & recovery if FS permissions cause above to work and below to fail ACCUMULO-2596
 +          Path prevDataVersionLoc = new Path(dataVersionLocation, Integer.toString(ServerConstants.PREV_DATA_VERSION));
 +          if (!fs.delete(prevDataVersionLoc)) {
 +            throw new RuntimeException("Could not delete previous data version location (" + prevDataVersionLoc + ") for " + volume);
 +          }
 +        }
 +      } catch (IOException e) {
 +        throw new RuntimeException("Unable to set accumulo version: an error occurred.", e);
 +      }
 +    }
 +  }
 +  
 +  public static synchronized int getAccumuloPersistentVersion(FileSystem fs, Path path) {
 +    int dataVersion;
 +    try {
 +      FileStatus[] files = fs.listStatus(path);
 +      if (files == null || files.length == 0) {
 +        dataVersion = -1; // assume it is 0.5 or earlier
 +      } else {
 +        dataVersion = Integer.parseInt(files[0].getPath().getName());
 +      }
 +      return dataVersion;
 +    } catch (IOException e) {
 +      throw new RuntimeException("Unable to read accumulo version: an error occurred.", e);
 +    }
 +  }
 +  
 +  public static synchronized int getAccumuloPersistentVersion(VolumeManager fs) {
 +    // It doesn't matter which Volume is used as they should all have the data version stored
 +    Volume v = fs.getVolumes().iterator().next();
 +    Path path = ServerConstants.getDataVersionLocation(v);
 +    return getAccumuloPersistentVersion(v.getFileSystem(), path);
 +  }
 +
 +  public static synchronized Path getAccumuloInstanceIdPath(VolumeManager fs) {
 +    // It doesn't matter which Volume is used as they should all have the instance ID stored
 +    Volume v = fs.getVolumes().iterator().next();
 +    return ServerConstants.getInstanceIdLocation(v);
 +  }
 +
 +  public static void enableTracing(String address, String application) {
 +    try {
 +      DistributedTrace.enable(HdfsZooInstance.getInstance(), ZooReaderWriter.getInstance(), application, address);
 +    } catch (Exception ex) {
 +      log.error("creating remote sink for trace spans", ex);
 +    }
 +  }
 +  
 +  public static void init(VolumeManager fs, ServerConfiguration config, String application) throws UnknownHostException {
 +    
 +    System.setProperty("org.apache.accumulo.core.application", application);
 +    
 +    if (System.getenv("ACCUMULO_LOG_DIR") != null)
 +      System.setProperty("org.apache.accumulo.core.dir.log", System.getenv("ACCUMULO_LOG_DIR"));
 +    else
 +      System.setProperty("org.apache.accumulo.core.dir.log", System.getenv("ACCUMULO_HOME") + "/logs/");
 +    
 +    String localhost = InetAddress.getLocalHost().getHostName();
 +    System.setProperty("org.apache.accumulo.core.ip.localhost.hostname", localhost);
 +    
 +    int logPort = config.getConfiguration().getPort(Property.MONITOR_LOG4J_PORT);
 +    System.setProperty("org.apache.accumulo.core.host.log.port", Integer.toString(logPort));
 +    
 +    // Use a specific log config, if it exists
 +    String logConfig = String.format("%s/%s_logger.xml", System.getenv("ACCUMULO_CONF_DIR"), application);
 +    if (!new File(logConfig).exists()) {
 +      // otherwise, use the generic config
 +      logConfig = String.format("%s/generic_logger.xml", System.getenv("ACCUMULO_CONF_DIR"));
 +    }
 +    // Turn off messages about not being able to reach the remote logger... we protect against that.
 +    LogLog.setQuietMode(true);
 +
 +    // Read the auditing config
 +    String auditConfig = String.format("%s/auditLog.xml", System.getenv("ACCUMULO_CONF_DIR"));
 +
 +    DOMConfigurator.configureAndWatch(auditConfig, 5000);
 +
 +    // Configure logging using information advertised in zookeeper by the monitor
 +    new MonitorLog4jWatcher(config.getInstance().getInstanceID(), logConfig, 5000).start();
 +
 +    log.info(application + " starting");
 +    log.info("Instance " + config.getInstance().getInstanceID());
 +    int dataVersion = Accumulo.getAccumuloPersistentVersion(fs);
 +    log.info("Data Version " + dataVersion);
 +    Accumulo.waitForZookeeperAndHdfs(fs);
 +    
 +    Version codeVersion = new Version(Constants.VERSION);
 +    if (dataVersion != ServerConstants.DATA_VERSION && dataVersion != ServerConstants.PREV_DATA_VERSION) {
 +      throw new RuntimeException("This version of accumulo (" + codeVersion + ") is not compatible with files stored using data version " + dataVersion);
 +    }
 +    
 +    TreeMap<String,String> sortedProps = new TreeMap<String,String>();
 +    for (Entry<String,String> entry : config.getConfiguration())
 +      sortedProps.put(entry.getKey(), entry.getValue());
 +    
 +    for (Entry<String,String> entry : sortedProps.entrySet()) {
 +      String key = entry.getKey();
 +      log.info(key + " = " + (Property.isSensitive(key) ? "<hidden>" : entry.getValue()));
 +    }
 +    
 +    monitorSwappiness();
 +  }
 +  
 +  /**
 +   * 
 +   */
 +  public static void monitorSwappiness() {
 +    SimpleTimer.getInstance().schedule(new Runnable() {
 +      @Override
 +      public void run() {
 +        try {
 +          String procFile = "/proc/sys/vm/swappiness";
 +          File swappiness = new File(procFile);
 +          if (swappiness.exists() && swappiness.canRead()) {
 +            InputStream is = new FileInputStream(procFile);
 +            try {
 +              byte[] buffer = new byte[10];
 +              int bytes = is.read(buffer);
 +              String setting = new String(buffer, 0, bytes, Constants.UTF8);
 +              setting = setting.trim();
 +              if (bytes > 0 && Integer.parseInt(setting) > 10) {
 +                log.warn("System swappiness setting is greater than ten (" + setting + ") which can cause time-sensitive operations to be delayed. "
 +                    + " Accumulo is time sensitive because it needs to maintain distributed lock agreement.");
 +              }
 +            } finally {
 +              is.close();
 +            }
 +          }
 +        } catch (Throwable t) {
 +          log.error(t, t);
 +        }
 +      }
 +    }, 1000, 10 * 60 * 1000);
 +  }
 +  
 +  public static void waitForZookeeperAndHdfs(VolumeManager fs) {
 +    log.info("Attempting to talk to zookeeper");
 +    while (true) {
 +      try {
 +        ZooReaderWriter.getInstance().getChildren(Constants.ZROOT);
 +        break;
 +      } catch (InterruptedException e) {
 +        // ignored
 +      } catch (KeeperException ex) {
 +        log.info("Waiting for accumulo to be initialized");
 +        UtilWaitThread.sleep(1000);
 +      }
 +    }
 +    log.info("Zookeeper connected and initialized, attemping to talk to HDFS");
 +    long sleep = 1000;
 +    int unknownHostTries = 3;
 +    while (true) {
 +      try {
 +        if (fs.isReady())
 +          break;
 +        log.warn("Waiting for the NameNode to leave safemode");
 +      } catch (IOException ex) {
 +        log.warn("Unable to connect to HDFS", ex);
 +      } catch (IllegalArgumentException exception) {
 +        /* Unwrap the UnknownHostException so we can deal with it directly */
 +        if (exception.getCause() instanceof UnknownHostException) {
 +          if (unknownHostTries > 0) {
 +            log.warn("Unable to connect to HDFS, will retry. cause: " + exception.getCause());
 +            /* We need to make sure our sleep period is long enough to avoid getting a cached failure of the host lookup. */
 +            sleep = Math.max(sleep, (AddressUtil.getAddressCacheNegativeTtl((UnknownHostException)(exception.getCause()))+1)*1000);
 +          } else {
 +            log.error("Unable to connect to HDFS and have exceeded max number of retries.", exception);
 +            throw exception;
 +          }
 +          unknownHostTries--;
 +        } else {
 +          throw exception;
 +        }
 +      }
 +      log.info("Backing off due to failure; current sleep period is " + sleep / 1000. + " seconds");
 +      UtilWaitThread.sleep(sleep);
 +      /* Back off to give transient failures more time to clear. */
 +      sleep = Math.min(60 * 1000, sleep * 2);
 +    }
 +    log.info("Connected to HDFS");
 +  }
-   
++
++  /**
++   * Exit loudly if there are outstanding Fate operations.
++   * Since Fate serializes class names, we need to make sure there are no queued
++   * transactions from a previous version before continuing an upgrade. The status of the operations is
++   * irrelevant; those in SUCCESSFUL status cause the same problem as those just queued.
++   *
++   * Note that the Master should not allow write access to Fate until after all upgrade steps are complete.
++   *
++   * Should be called as a guard before performing any upgrade steps, after determining that an upgrade is needed.
++   *
++   * see ACCUMULO-2519
++   */
++  public static void abortIfFateTransactions() {
++    try {
++      final ReadOnlyTStore<Accumulo> fate = new ReadOnlyStore<Accumulo>(new ZooStore<Accumulo>(ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZFATE,
++          ZooReaderWriter.getRetryingInstance()));
++      if (!(fate.list().isEmpty())) {
++        throw new AccumuloException("Aborting upgrade because there are outstanding FATE transactions from a previous Accumulo version. Please see the README document for instructions on what to do under your previous version.");
++      }
++    } catch (Exception exception) {
++      log.fatal("Problem verifying Fate readiness", exception);
++      System.exit(1);
++    }
++  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/957c9d1b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
----------------------------------------------------------------------
diff --cc server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
index 760d57f,0000000..374017d
mode 100644,000000..100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
@@@ -1,1018 -1,0 +1,1021 @@@
 +/*
 + * 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.server.util;
 +
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.Collections;
 +import java.util.Comparator;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +import java.util.SortedMap;
 +import java.util.TreeMap;
 +import java.util.concurrent.TimeUnit;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.BatchWriter;
 +import org.apache.accumulo.core.client.BatchWriterConfig;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.IsolatedScanner;
 +import org.apache.accumulo.core.client.MutationsRejectedException;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.client.impl.BatchWriterImpl;
 +import org.apache.accumulo.core.client.impl.ScannerImpl;
 +import org.apache.accumulo.core.client.impl.Writer;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.KeyExtent;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.PartialKey;
 +import org.apache.accumulo.core.data.Range;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.metadata.MetadataTable;
 +import org.apache.accumulo.core.metadata.RootTable;
 +import org.apache.accumulo.core.metadata.schema.DataFileValue;
 +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.ChoppedColumnFamily;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ClonedColumnFamily;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.core.security.Credentials;
 +import org.apache.accumulo.core.tabletserver.log.LogEntry;
 +import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 +import org.apache.accumulo.core.util.ColumnFQ;
 +import org.apache.accumulo.core.util.FastFormat;
 +import org.apache.accumulo.core.util.Pair;
 +import org.apache.accumulo.core.util.UtilWaitThread;
 +import org.apache.accumulo.core.zookeeper.ZooUtil;
 +import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 +import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 +import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 +import org.apache.accumulo.server.ServerConstants;
 +import org.apache.accumulo.server.client.HdfsZooInstance;
 +import org.apache.accumulo.server.fs.FileRef;
 +import org.apache.accumulo.server.fs.VolumeManager;
 +import org.apache.accumulo.server.fs.VolumeManagerImpl;
 +import org.apache.accumulo.server.security.SystemCredentials;
 +import org.apache.accumulo.server.zookeeper.ZooLock;
 +import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 +import org.apache.hadoop.fs.FileStatus;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.io.Text;
 +import org.apache.log4j.Logger;
 +import org.apache.zookeeper.KeeperException;
 +
 +/**
 + * provides a reference to the metadata table for updates by tablet servers
 + */
 +public class MetadataTableUtil {
 +
 +  private static final Text EMPTY_TEXT = new Text();
 +  private static Map<Credentials,Writer> root_tables = new HashMap<Credentials,Writer>();
 +  private static Map<Credentials,Writer> metadata_tables = new HashMap<Credentials,Writer>();
 +  private static final Logger log = Logger.getLogger(MetadataTableUtil.class);
 +
 +  private MetadataTableUtil() {}
 +
 +  public synchronized static Writer getMetadataTable(Credentials credentials) {
 +    Writer metadataTable = metadata_tables.get(credentials);
 +    if (metadataTable == null) {
 +      metadataTable = new Writer(HdfsZooInstance.getInstance(), credentials, MetadataTable.ID);
 +      metadata_tables.put(credentials, metadataTable);
 +    }
 +    return metadataTable;
 +  }
 +
 +  private synchronized static Writer getRootTable(Credentials credentials) {
 +    Writer rootTable = root_tables.get(credentials);
 +    if (rootTable == null) {
 +      rootTable = new Writer(HdfsZooInstance.getInstance(), credentials, RootTable.ID);
 +      root_tables.put(credentials, rootTable);
 +    }
 +    return rootTable;
 +  }
 +
 +  private static void putLockID(ZooLock zooLock, Mutation m) {
 +    TabletsSection.ServerColumnFamily.LOCK_COLUMN.put(m, new Value(zooLock.getLockID().serialize(ZooUtil.getRoot(HdfsZooInstance.getInstance()) + "/")
 +        .getBytes(Constants.UTF8)));
 +  }
 +
 +  private static void update(Credentials credentials, Mutation m, KeyExtent extent) {
 +    update(credentials, null, m, extent);
 +  }
 +
 +  public static void update(Credentials credentials, ZooLock zooLock, Mutation m, KeyExtent extent) {
 +    Writer t = extent.isMeta() ? getRootTable(credentials) : getMetadataTable(credentials);
 +    if (zooLock != null)
 +      putLockID(zooLock, m);
 +    while (true) {
 +      try {
 +        t.update(m);
 +        return;
 +      } catch (AccumuloException e) {
 +        log.error(e, e);
 +      } catch (AccumuloSecurityException e) {
 +        log.error(e, e);
 +      } catch (ConstraintViolationException e) {
 +        log.error(e, e);
 +      } catch (TableNotFoundException e) {
 +        log.error(e, e);
 +      }
 +      UtilWaitThread.sleep(1000);
 +    }
 +
 +  }
 +
 +  public static void updateTabletFlushID(KeyExtent extent, long flushID, Credentials credentials, ZooLock zooLock) {
 +    if (!extent.isRootTablet()) {
 +      Mutation m = new Mutation(extent.getMetadataEntry());
 +      TabletsSection.ServerColumnFamily.FLUSH_COLUMN.put(m, new Value((flushID + "").getBytes(Constants.UTF8)));
 +      update(credentials, zooLock, m, extent);
 +    }
 +  }
 +
 +  public static void updateTabletCompactID(KeyExtent extent, long compactID, Credentials credentials, ZooLock zooLock) {
 +    if (!extent.isRootTablet()) {
 +      Mutation m = new Mutation(extent.getMetadataEntry());
 +      TabletsSection.ServerColumnFamily.COMPACT_COLUMN.put(m, new Value((compactID + "").getBytes(Constants.UTF8)));
 +      update(credentials, zooLock, m, extent);
 +    }
 +  }
 +
 +  public static void updateTabletDataFile(long tid, KeyExtent extent, Map<FileRef,DataFileValue> estSizes, String time, Credentials credentials, ZooLock zooLock) {
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +    byte[] tidBytes = Long.toString(tid).getBytes(Constants.UTF8);
 +
 +    for (Entry<FileRef,DataFileValue> entry : estSizes.entrySet()) {
 +      Text file = entry.getKey().meta();
 +      m.put(DataFileColumnFamily.NAME, file, new Value(entry.getValue().encode()));
 +      m.put(TabletsSection.BulkFileColumnFamily.NAME, file, new Value(tidBytes));
 +    }
 +    TabletsSection.ServerColumnFamily.TIME_COLUMN.put(m, new Value(time.getBytes(Constants.UTF8)));
 +    update(credentials, zooLock, m, extent);
 +  }
 +
 +  public static void updateTabletDir(KeyExtent extent, String newDir, Credentials creds, ZooLock lock) {
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +    TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(m, new Value(newDir.getBytes(Constants.UTF8)));
 +    update(creds, lock, m, extent);
 +  }
 +
 +  public static void addTablet(KeyExtent extent, String path, Credentials credentials, char timeType, ZooLock lock) {
 +    Mutation m = extent.getPrevRowUpdateMutation();
 +
 +    TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(m, new Value(path.getBytes(Constants.UTF8)));
 +    TabletsSection.ServerColumnFamily.TIME_COLUMN.put(m, new Value((timeType + "0").getBytes(Constants.UTF8)));
 +
 +    update(credentials, lock, m, extent);
 +  }
 +
 +  public static void updateTabletPrevEndRow(KeyExtent extent, Credentials credentials) {
 +    Mutation m = extent.getPrevRowUpdateMutation(); //
 +    update(credentials, m, extent);
 +  }
 +
 +  public static void updateTabletVolumes(KeyExtent extent, List<LogEntry> logsToRemove, List<LogEntry> logsToAdd, List<FileRef> filesToRemove,
 +      SortedMap<FileRef,DataFileValue> filesToAdd, String newDir, ZooLock zooLock, Credentials credentials) {
 +
 +    if (extent.isRootTablet()) {
 +      if (newDir != null)
 +        throw new IllegalArgumentException("newDir not expected for " + extent);
 +
 +      if (filesToRemove.size() != 0 || filesToAdd.size() != 0)
 +        throw new IllegalArgumentException("files not expected for " + extent);
 +
 +      // add before removing in case of process death
 +      for (LogEntry logEntry : logsToAdd)
 +        addLogEntry(credentials, logEntry, zooLock);
 +
 +      removeUnusedWALEntries(extent, logsToRemove, zooLock);
 +    } else {
 +      Mutation m = new Mutation(extent.getMetadataEntry());
 +
 +      for (LogEntry logEntry : logsToRemove)
 +        m.putDelete(logEntry.getColumnFamily(), logEntry.getColumnQualifier());
 +
 +      for (LogEntry logEntry : logsToAdd)
 +        m.put(logEntry.getColumnFamily(), logEntry.getColumnQualifier(), logEntry.getValue());
 +
 +      for (FileRef fileRef : filesToRemove)
 +        m.putDelete(DataFileColumnFamily.NAME, fileRef.meta());
 +
 +      for (Entry<FileRef,DataFileValue> entry : filesToAdd.entrySet())
 +        m.put(DataFileColumnFamily.NAME, entry.getKey().meta(), new Value(entry.getValue().encode()));
 +
 +      if (newDir != null)
 +        ServerColumnFamily.DIRECTORY_COLUMN.put(m, new Value(newDir.getBytes(Constants.UTF8)));
 +
 +      update(credentials, m, extent);
 +    }
 +  }
 +
 +  public static SortedMap<FileRef,DataFileValue> getDataFileSizes(KeyExtent extent, Credentials credentials) throws IOException {
 +    TreeMap<FileRef,DataFileValue> sizes = new TreeMap<FileRef,DataFileValue>();
 +
 +    Scanner mdScanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, MetadataTable.ID, Authorizations.EMPTY);
 +    mdScanner.fetchColumnFamily(DataFileColumnFamily.NAME);
 +    Text row = extent.getMetadataEntry();
 +    VolumeManager fs = VolumeManagerImpl.get();
 +
 +    Key endKey = new Key(row, DataFileColumnFamily.NAME, new Text(""));
 +    endKey = endKey.followingKey(PartialKey.ROW_COLFAM);
 +
 +    mdScanner.setRange(new Range(new Key(row), endKey));
 +    for (Entry<Key,Value> entry : mdScanner) {
 +
 +      if (!entry.getKey().getRow().equals(row))
 +        break;
 +      DataFileValue dfv = new DataFileValue(entry.getValue().get());
 +      sizes.put(new FileRef(fs, entry.getKey()), dfv);
 +    }
 +
 +    return sizes;
 +  }
 +
 +  public static void rollBackSplit(Text metadataEntry, Text oldPrevEndRow, Credentials credentials, ZooLock zooLock) {
 +    KeyExtent ke = new KeyExtent(metadataEntry, oldPrevEndRow);
 +    Mutation m = ke.getPrevRowUpdateMutation();
 +    TabletsSection.TabletColumnFamily.SPLIT_RATIO_COLUMN.putDelete(m);
 +    TabletsSection.TabletColumnFamily.OLD_PREV_ROW_COLUMN.putDelete(m);
 +    update(credentials, zooLock, m, new KeyExtent(metadataEntry, (Text) null));
 +  }
 +
 +  public static void splitTablet(KeyExtent extent, Text oldPrevEndRow, double splitRatio, Credentials credentials, ZooLock zooLock) {
 +    Mutation m = extent.getPrevRowUpdateMutation(); //
 +
 +    TabletsSection.TabletColumnFamily.SPLIT_RATIO_COLUMN.put(m, new Value(Double.toString(splitRatio).getBytes(Constants.UTF8)));
 +
 +    TabletsSection.TabletColumnFamily.OLD_PREV_ROW_COLUMN.put(m, KeyExtent.encodePrevEndRow(oldPrevEndRow));
 +    ChoppedColumnFamily.CHOPPED_COLUMN.putDelete(m);
 +    update(credentials, zooLock, m, extent);
 +  }
 +
 +  public static void finishSplit(Text metadataEntry, Map<FileRef,DataFileValue> datafileSizes, List<FileRef> highDatafilesToRemove, Credentials credentials,
 +      ZooLock zooLock) {
 +    Mutation m = new Mutation(metadataEntry);
 +    TabletsSection.TabletColumnFamily.SPLIT_RATIO_COLUMN.putDelete(m);
 +    TabletsSection.TabletColumnFamily.OLD_PREV_ROW_COLUMN.putDelete(m);
 +    ChoppedColumnFamily.CHOPPED_COLUMN.putDelete(m);
 +
 +    for (Entry<FileRef,DataFileValue> entry : datafileSizes.entrySet()) {
 +      m.put(DataFileColumnFamily.NAME, entry.getKey().meta(), new Value(entry.getValue().encode()));
 +    }
 +
 +    for (FileRef pathToRemove : highDatafilesToRemove) {
 +      m.putDelete(DataFileColumnFamily.NAME, pathToRemove.meta());
 +    }
 +
 +    update(credentials, zooLock, m, new KeyExtent(metadataEntry, (Text) null));
 +  }
 +
 +  public static void finishSplit(KeyExtent extent, Map<FileRef,DataFileValue> datafileSizes, List<FileRef> highDatafilesToRemove, Credentials credentials,
 +      ZooLock zooLock) {
 +    finishSplit(extent.getMetadataEntry(), datafileSizes, highDatafilesToRemove, credentials, zooLock);
 +  }
 +
 +  public static void addDeleteEntries(KeyExtent extent, Set<FileRef> datafilesToDelete, Credentials credentials) throws IOException {
 +
 +    String tableId = extent.getTableId().toString();
 +
 +    // TODO could use batch writer,would need to handle failure and retry like update does - ACCUMULO-1294
 +    for (FileRef pathToRemove : datafilesToDelete) {
 +      update(credentials, createDeleteMutation(tableId, pathToRemove.path().toString()), extent);
 +    }
 +  }
 +
 +  public static void addDeleteEntry(String tableId, String path) throws IOException {
 +    update(SystemCredentials.get(), createDeleteMutation(tableId, path), new KeyExtent(new Text(tableId), null, null));
 +  }
 +
 +  public static Mutation createDeleteMutation(String tableId, String pathToRemove) throws IOException {
 +    Path path = VolumeManagerImpl.get().getFullPath(tableId, pathToRemove);
 +    Mutation delFlag = new Mutation(new Text(MetadataSchema.DeletesSection.getRowPrefix() + path.toString()));
 +    delFlag.put(EMPTY_TEXT, EMPTY_TEXT, new Value(new byte[] {}));
 +    return delFlag;
 +  }
 +
 +  public static void removeScanFiles(KeyExtent extent, Set<FileRef> scanFiles, Credentials credentials, ZooLock zooLock) {
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +
 +    for (FileRef pathToRemove : scanFiles)
 +      m.putDelete(ScanFileColumnFamily.NAME, pathToRemove.meta());
 +
 +    update(credentials, zooLock, m, extent);
 +  }
 +
 +  public static void splitDatafiles(Text table, Text midRow, double splitRatio, Map<FileRef,FileUtil.FileInfo> firstAndLastRows,
 +      SortedMap<FileRef,DataFileValue> datafiles, SortedMap<FileRef,DataFileValue> lowDatafileSizes, SortedMap<FileRef,DataFileValue> highDatafileSizes,
 +      List<FileRef> highDatafilesToRemove) {
 +
 +    for (Entry<FileRef,DataFileValue> entry : datafiles.entrySet()) {
 +
 +      Text firstRow = null;
 +      Text lastRow = null;
 +
 +      boolean rowsKnown = false;
 +
 +      FileUtil.FileInfo mfi = firstAndLastRows.get(entry.getKey());
 +
 +      if (mfi != null) {
 +        firstRow = mfi.getFirstRow();
 +        lastRow = mfi.getLastRow();
 +        rowsKnown = true;
 +      }
 +
 +      if (rowsKnown && firstRow.compareTo(midRow) > 0) {
 +        // only in high
 +        long highSize = entry.getValue().getSize();
 +        long highEntries = entry.getValue().getNumEntries();
 +        highDatafileSizes.put(entry.getKey(), new DataFileValue(highSize, highEntries, entry.getValue().getTime()));
 +      } else if (rowsKnown && lastRow.compareTo(midRow) <= 0) {
 +        // only in low
 +        long lowSize = entry.getValue().getSize();
 +        long lowEntries = entry.getValue().getNumEntries();
 +        lowDatafileSizes.put(entry.getKey(), new DataFileValue(lowSize, lowEntries, entry.getValue().getTime()));
 +
 +        highDatafilesToRemove.add(entry.getKey());
 +      } else {
 +        long lowSize = (long) Math.floor((entry.getValue().getSize() * splitRatio));
 +        long lowEntries = (long) Math.floor((entry.getValue().getNumEntries() * splitRatio));
 +        lowDatafileSizes.put(entry.getKey(), new DataFileValue(lowSize, lowEntries, entry.getValue().getTime()));
 +
 +        long highSize = (long) Math.ceil((entry.getValue().getSize() * (1.0 - splitRatio)));
 +        long highEntries = (long) Math.ceil((entry.getValue().getNumEntries() * (1.0 - splitRatio)));
 +        highDatafileSizes.put(entry.getKey(), new DataFileValue(highSize, highEntries, entry.getValue().getTime()));
 +      }
 +    }
 +  }
 +
 +  public static void deleteTable(String tableId, boolean insertDeletes, Credentials credentials, ZooLock lock) throws AccumuloException, IOException {
 +    Scanner ms = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, MetadataTable.ID, Authorizations.EMPTY);
 +    Text tableIdText = new Text(tableId);
 +    BatchWriter bw = new BatchWriterImpl(HdfsZooInstance.getInstance(), credentials, MetadataTable.ID, new BatchWriterConfig().setMaxMemory(1000000)
 +        .setMaxLatency(120000l, TimeUnit.MILLISECONDS).setMaxWriteThreads(2));
 +
 +    // scan metadata for our table and delete everything we find
 +    Mutation m = null;
 +    ms.setRange(new KeyExtent(tableIdText, null, null).toMetadataRange());
 +
 +    // insert deletes before deleting data from metadata... this makes the code fault tolerant
 +    if (insertDeletes) {
 +
 +      ms.fetchColumnFamily(DataFileColumnFamily.NAME);
 +      TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.fetch(ms);
 +
 +      for (Entry<Key,Value> cell : ms) {
 +        Key key = cell.getKey();
 +
 +        if (key.getColumnFamily().equals(DataFileColumnFamily.NAME)) {
 +          FileRef ref = new FileRef(VolumeManagerImpl.get(), key);
 +          bw.addMutation(createDeleteMutation(tableId, ref.meta().toString()));
 +        }
 +
 +        if (TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.hasColumns(key)) {
 +          bw.addMutation(createDeleteMutation(tableId, cell.getValue().toString()));
 +        }
 +      }
 +
 +      bw.flush();
 +
 +      ms.clearColumns();
 +    }
 +
 +    for (Entry<Key,Value> cell : ms) {
 +      Key key = cell.getKey();
 +
 +      if (m == null) {
 +        m = new Mutation(key.getRow());
 +        if (lock != null)
 +          putLockID(lock, m);
 +      }
 +
 +      if (key.getRow().compareTo(m.getRow(), 0, m.getRow().length) != 0) {
 +        bw.addMutation(m);
 +        m = new Mutation(key.getRow());
 +        if (lock != null)
 +          putLockID(lock, m);
 +      }
 +      m.putDelete(key.getColumnFamily(), key.getColumnQualifier());
 +    }
 +
 +    if (m != null)
 +      bw.addMutation(m);
 +
 +    bw.close();
 +  }
 +
 +  static String getZookeeperLogLocation() {
 +    return ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_WALOGS;
 +  }
 +
 +  public static void addLogEntry(Credentials credentials, LogEntry entry, ZooLock zooLock) {
 +    if (entry.extent.isRootTablet()) {
 +      String root = getZookeeperLogLocation();
 +      while (true) {
 +        try {
 +          IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +          if (zoo.isLockHeld(zooLock.getLockID())) {
 +            String[] parts = entry.filename.split("/");
 +            String uniqueId = parts[parts.length - 1];
 +            zoo.putPersistentData(root + "/" + uniqueId, entry.toBytes(), NodeExistsPolicy.OVERWRITE);
 +          }
 +          break;
 +        } catch (KeeperException e) {
 +          log.error(e, e);
 +        } catch (InterruptedException e) {
 +          log.error(e, e);
 +        } catch (IOException e) {
 +          log.error(e, e);
 +        }
 +        UtilWaitThread.sleep(1000);
 +      }
 +    } else {
 +      Mutation m = new Mutation(entry.getRow());
 +      m.put(entry.getColumnFamily(), entry.getColumnQualifier(), entry.getValue());
 +      update(credentials, zooLock, m, entry.extent);
 +    }
 +  }
 +
 +  public static void setRootTabletDir(String dir) throws IOException {
 +    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +    String zpath = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_PATH;
 +    try {
 +      zoo.putPersistentData(zpath, dir.getBytes(Constants.UTF8), -1, NodeExistsPolicy.OVERWRITE);
 +    } catch (KeeperException e) {
 +      throw new IOException(e);
 +    } catch (InterruptedException e) {
 +      Thread.currentThread().interrupt();
 +      throw new IOException(e);
 +    }
 +  }
 +
 +  public static String getRootTabletDir() throws IOException {
 +    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +    String zpath = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_PATH;
 +    try {
 +      return new String(zoo.getData(zpath, null), Constants.UTF8);
 +    } catch (KeeperException e) {
 +      throw new IOException(e);
 +    } catch (InterruptedException e) {
 +      Thread.currentThread().interrupt();
 +      throw new IOException(e);
 +    }
 +  }
 +
 +  public static Pair<List<LogEntry>,SortedMap<FileRef,DataFileValue>> getFileAndLogEntries(Credentials credentials, KeyExtent extent) throws KeeperException,
 +      InterruptedException, IOException {
 +    ArrayList<LogEntry> result = new ArrayList<LogEntry>();
 +    TreeMap<FileRef,DataFileValue> sizes = new TreeMap<FileRef,DataFileValue>();
 +
 +    VolumeManager fs = VolumeManagerImpl.get();
 +    if (extent.isRootTablet()) {
 +      getRootLogEntries(result);
 +      Path rootDir = new Path(getRootTabletDir());
 +      FileStatus[] files = fs.listStatus(rootDir);
 +      for (FileStatus fileStatus : files) {
 +        if (fileStatus.getPath().toString().endsWith("_tmp")) {
 +          continue;
 +        }
 +        DataFileValue dfv = new DataFileValue(0, 0);
 +        sizes.put(new FileRef(fileStatus.getPath().toString(), fileStatus.getPath()), dfv);
 +      }
 +
 +    } else {
 +      String systemTableToCheck = extent.isMeta() ? RootTable.ID : MetadataTable.ID;
 +      Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, systemTableToCheck, Authorizations.EMPTY);
 +      scanner.fetchColumnFamily(LogColumnFamily.NAME);
 +      scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
 +      scanner.setRange(extent.toMetadataRange());
 +
 +      for (Entry<Key,Value> entry : scanner) {
 +        if (!entry.getKey().getRow().equals(extent.getMetadataEntry())) {
 +          throw new RuntimeException("Unexpected row " + entry.getKey().getRow() + " expected " + extent.getMetadataEntry());
 +        }
 +
 +        if (entry.getKey().getColumnFamily().equals(LogColumnFamily.NAME)) {
 +          result.add(LogEntry.fromKeyValue(entry.getKey(), entry.getValue()));
 +        } else if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
 +          DataFileValue dfv = new DataFileValue(entry.getValue().get());
 +          sizes.put(new FileRef(fs, entry.getKey()), dfv);
 +        } else {
 +          throw new RuntimeException("Unexpected col fam " + entry.getKey().getColumnFamily());
 +        }
 +      }
 +    }
 +
 +    return new Pair<List<LogEntry>,SortedMap<FileRef,DataFileValue>>(result, sizes);
 +  }
 +
 +  public static List<LogEntry> getLogEntries(Credentials credentials, KeyExtent extent) throws IOException, KeeperException, InterruptedException {
 +    log.info("Scanning logging entries for " + extent);
 +    ArrayList<LogEntry> result = new ArrayList<LogEntry>();
 +    if (extent.equals(RootTable.EXTENT)) {
 +      log.info("Getting logs for root tablet from zookeeper");
 +      getRootLogEntries(result);
 +    } else {
 +      log.info("Scanning metadata for logs used for tablet " + extent);
 +      Scanner scanner = getTabletLogScanner(credentials, extent);
 +      Text pattern = extent.getMetadataEntry();
 +      for (Entry<Key,Value> entry : scanner) {
 +        Text row = entry.getKey().getRow();
 +        if (entry.getKey().getColumnFamily().equals(LogColumnFamily.NAME)) {
 +          if (row.equals(pattern)) {
 +            result.add(LogEntry.fromKeyValue(entry.getKey(), entry.getValue()));
 +          }
 +        }
 +      }
 +    }
 +
 +    Collections.sort(result, new Comparator<LogEntry>() {
 +      @Override
 +      public int compare(LogEntry o1, LogEntry o2) {
 +        long diff = o1.timestamp - o2.timestamp;
 +        if (diff < 0)
 +          return -1;
 +        if (diff > 0)
 +          return 1;
 +        return 0;
 +      }
 +    });
 +    log.info("Returning logs " + result + " for extent " + extent);
 +    return result;
 +  }
 +
 +  static void getRootLogEntries(ArrayList<LogEntry> result) throws KeeperException, InterruptedException, IOException {
 +    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +    String root = getZookeeperLogLocation();
 +    // there's a little race between getting the children and fetching
 +    // the data. The log can be removed in between.
 +    while (true) {
 +      result.clear();
 +      for (String child : zoo.getChildren(root)) {
 +        LogEntry e = new LogEntry();
 +        try {
 +          e.fromBytes(zoo.getData(root + "/" + child, null));
 +          // upgrade from !0;!0<< -> +r<<
 +          e.extent = RootTable.EXTENT;
 +          result.add(e);
 +        } catch (KeeperException.NoNodeException ex) {
 +          continue;
 +        }
 +      }
 +      break;
 +    }
 +  }
 +
 +  private static Scanner getTabletLogScanner(Credentials credentials, KeyExtent extent) {
 +    String tableId = MetadataTable.ID;
 +    if (extent.isMeta())
 +      tableId = RootTable.ID;
 +    Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, tableId, Authorizations.EMPTY);
 +    scanner.fetchColumnFamily(LogColumnFamily.NAME);
 +    Text start = extent.getMetadataEntry();
 +    Key endKey = new Key(start, LogColumnFamily.NAME);
 +    endKey = endKey.followingKey(PartialKey.ROW_COLFAM);
 +    scanner.setRange(new Range(new Key(start), endKey));
 +    return scanner;
 +  }
 +
 +  private static class LogEntryIterator implements Iterator<LogEntry> {
 +
 +    Iterator<LogEntry> zookeeperEntries = null;
 +    Iterator<LogEntry> rootTableEntries = null;
 +    Iterator<Entry<Key,Value>> metadataEntries = null;
 +
 +    LogEntryIterator(Credentials creds) throws IOException, KeeperException, InterruptedException {
 +      zookeeperEntries = getLogEntries(creds, RootTable.EXTENT).iterator();
 +      rootTableEntries = getLogEntries(creds, new KeyExtent(new Text(MetadataTable.ID), null, null)).iterator();
 +      try {
 +        Scanner scanner = HdfsZooInstance.getInstance().getConnector(creds.getPrincipal(), creds.getToken())
 +            .createScanner(MetadataTable.NAME, Authorizations.EMPTY);
 +        log.info("Setting range to " + MetadataSchema.TabletsSection.getRange());
 +        scanner.setRange(MetadataSchema.TabletsSection.getRange());
 +        scanner.fetchColumnFamily(LogColumnFamily.NAME);
 +        metadataEntries = scanner.iterator();
 +      } catch (Exception ex) {
 +        throw new IOException(ex);
 +      }
 +    }
 +
 +    @Override
 +    public boolean hasNext() {
 +      return zookeeperEntries.hasNext() || rootTableEntries.hasNext() || metadataEntries.hasNext();
 +    }
 +
 +    @Override
 +    public LogEntry next() {
 +      if (zookeeperEntries.hasNext()) {
 +        return zookeeperEntries.next();
 +      }
 +      if (rootTableEntries.hasNext()) {
 +        return rootTableEntries.next();
 +      }
 +      Entry<Key,Value> entry = metadataEntries.next();
 +      return LogEntry.fromKeyValue(entry.getKey(), entry.getValue());
 +    }
 +
 +    @Override
 +    public void remove() {
 +      throw new UnsupportedOperationException();
 +    }
 +  }
 +
 +  public static Iterator<LogEntry> getLogEntries(Credentials creds) throws IOException, KeeperException, InterruptedException {
 +    return new LogEntryIterator(creds);
 +  }
 +
 +  public static void removeUnusedWALEntries(KeyExtent extent, List<LogEntry> logEntries, ZooLock zooLock) {
 +    if (extent.isRootTablet()) {
 +      for (LogEntry entry : logEntries) {
 +        String root = getZookeeperLogLocation();
 +        while (true) {
 +          try {
 +            IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +            if (zoo.isLockHeld(zooLock.getLockID())) {
 +              String parts[] = entry.filename.split("/");
 +              zoo.recursiveDelete(root + "/" + parts[parts.length - 1], NodeMissingPolicy.SKIP);
 +            }
 +            break;
 +          } catch (Exception e) {
 +            log.error(e, e);
 +          }
 +          UtilWaitThread.sleep(1000);
 +        }
 +      }
 +    } else {
 +      Mutation m = new Mutation(extent.getMetadataEntry());
 +      for (LogEntry entry : logEntries) {
 +        m.putDelete(LogColumnFamily.NAME, new Text(entry.getName()));
 +      }
 +      update(SystemCredentials.get(), zooLock, m, extent);
 +    }
 +  }
 +
 +  private static void getFiles(Set<String> files, Map<Key,Value> tablet, String srcTableId) {
 +    for (Entry<Key,Value> entry : tablet.entrySet()) {
 +      if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
 +        String cf = entry.getKey().getColumnQualifier().toString();
 +        if (srcTableId != null && !cf.startsWith("../") && !cf.contains(":")) {
 +          cf = "../" + srcTableId + entry.getKey().getColumnQualifier();
 +        }
 +        files.add(cf);
 +      }
 +    }
 +  }
 +
 +  private static Mutation createCloneMutation(String srcTableId, String tableId, Map<Key,Value> tablet) {
 +
 +    KeyExtent ke = new KeyExtent(tablet.keySet().iterator().next().getRow(), (Text) null);
 +    Mutation m = new Mutation(KeyExtent.getMetadataEntry(new Text(tableId), ke.getEndRow()));
 +
 +    for (Entry<Key,Value> entry : tablet.entrySet()) {
 +      if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
 +        String cf = entry.getKey().getColumnQualifier().toString();
 +        if (!cf.startsWith("../") && !cf.contains(":"))
 +          cf = "../" + srcTableId + entry.getKey().getColumnQualifier();
 +        m.put(entry.getKey().getColumnFamily(), new Text(cf), entry.getValue());
 +      } else if (entry.getKey().getColumnFamily().equals(TabletsSection.CurrentLocationColumnFamily.NAME)) {
 +        m.put(TabletsSection.LastLocationColumnFamily.NAME, entry.getKey().getColumnQualifier(), entry.getValue());
 +      } else if (entry.getKey().getColumnFamily().equals(TabletsSection.LastLocationColumnFamily.NAME)) {
 +        // skip
 +      } else {
 +        m.put(entry.getKey().getColumnFamily(), entry.getKey().getColumnQualifier(), entry.getValue());
 +      }
 +    }
 +    return m;
 +  }
 +
 +  private static Scanner createCloneScanner(String tableId, Connector conn) throws TableNotFoundException {
 +    String tableName = MetadataTable.NAME;
 +    if (tableId.equals(MetadataTable.ID))
 +      tableName = RootTable.NAME;
 +    Scanner mscanner = new IsolatedScanner(conn.createScanner(tableName, Authorizations.EMPTY));
 +    mscanner.setRange(new KeyExtent(new Text(tableId), null, null).toMetadataRange());
 +    mscanner.fetchColumnFamily(DataFileColumnFamily.NAME);
 +    mscanner.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);
 +    mscanner.fetchColumnFamily(TabletsSection.LastLocationColumnFamily.NAME);
 +    mscanner.fetchColumnFamily(ClonedColumnFamily.NAME);
 +    TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(mscanner);
 +    TabletsSection.ServerColumnFamily.TIME_COLUMN.fetch(mscanner);
 +    return mscanner;
 +  }
 +
 +  static void initializeClone(String srcTableId, String tableId, Connector conn, BatchWriter bw) throws TableNotFoundException, MutationsRejectedException {
 +    TabletIterator ti;
 +    if (srcTableId.equals(MetadataTable.ID))
 +      ti = new TabletIterator(createCloneScanner(srcTableId, conn), new Range(), true, true);
 +    else
 +      ti = new TabletIterator(createCloneScanner(srcTableId, conn), new KeyExtent(new Text(srcTableId), null, null).toMetadataRange(), true, true);
 +
 +    if (!ti.hasNext())
 +      throw new RuntimeException(" table deleted during clone?  srcTableId = " + srcTableId);
 +
 +    while (ti.hasNext())
 +      bw.addMutation(createCloneMutation(srcTableId, tableId, ti.next()));
 +
 +    bw.flush();
 +  }
 +
 +  private static int compareEndRows(Text endRow1, Text endRow2) {
 +    return new KeyExtent(new Text("0"), endRow1, null).compareTo(new KeyExtent(new Text("0"), endRow2, null));
 +  }
 +
 +  static int checkClone(String srcTableId, String tableId, Connector conn, BatchWriter bw) throws TableNotFoundException, MutationsRejectedException {
 +    TabletIterator srcIter = new TabletIterator(createCloneScanner(srcTableId, conn), new KeyExtent(new Text(srcTableId), null, null).toMetadataRange(), true,
 +        true);
 +    TabletIterator cloneIter = new TabletIterator(createCloneScanner(tableId, conn), new KeyExtent(new Text(tableId), null, null).toMetadataRange(), true, true);
 +
 +    if (!cloneIter.hasNext() || !srcIter.hasNext())
 +      throw new RuntimeException(" table deleted during clone?  srcTableId = " + srcTableId + " tableId=" + tableId);
 +
 +    int rewrites = 0;
 +
 +    while (cloneIter.hasNext()) {
 +      Map<Key,Value> cloneTablet = cloneIter.next();
 +      Text cloneEndRow = new KeyExtent(cloneTablet.keySet().iterator().next().getRow(), (Text) null).getEndRow();
 +      HashSet<String> cloneFiles = new HashSet<String>();
 +
 +      boolean cloneSuccessful = false;
 +      for (Entry<Key,Value> entry : cloneTablet.entrySet()) {
 +        if (entry.getKey().getColumnFamily().equals(ClonedColumnFamily.NAME)) {
 +          cloneSuccessful = true;
 +          break;
 +        }
 +      }
 +
 +      if (!cloneSuccessful)
 +        getFiles(cloneFiles, cloneTablet, null);
 +
 +      List<Map<Key,Value>> srcTablets = new ArrayList<Map<Key,Value>>();
 +      Map<Key,Value> srcTablet = srcIter.next();
 +      srcTablets.add(srcTablet);
 +
 +      Text srcEndRow = new KeyExtent(srcTablet.keySet().iterator().next().getRow(), (Text) null).getEndRow();
 +
 +      int cmp = compareEndRows(cloneEndRow, srcEndRow);
 +      if (cmp < 0)
 +        throw new TabletIterator.TabletDeletedException("Tablets deleted from src during clone : " + cloneEndRow + " " + srcEndRow);
 +
 +      HashSet<String> srcFiles = new HashSet<String>();
 +      if (!cloneSuccessful)
 +        getFiles(srcFiles, srcTablet, srcTableId);
 +
 +      while (cmp > 0) {
 +        srcTablet = srcIter.next();
 +        srcTablets.add(srcTablet);
 +        srcEndRow = new KeyExtent(srcTablet.keySet().iterator().next().getRow(), (Text) null).getEndRow();
 +        cmp = compareEndRows(cloneEndRow, srcEndRow);
 +        if (cmp < 0)
 +          throw new TabletIterator.TabletDeletedException("Tablets deleted from src during clone : " + cloneEndRow + " " + srcEndRow);
 +
 +        if (!cloneSuccessful)
 +          getFiles(srcFiles, srcTablet, srcTableId);
 +      }
 +
 +      if (cloneSuccessful)
 +        continue;
 +
 +      if (!srcFiles.containsAll(cloneFiles)) {
 +        // delete existing cloned tablet entry
 +        Mutation m = new Mutation(cloneTablet.keySet().iterator().next().getRow());
 +
 +        for (Entry<Key,Value> entry : cloneTablet.entrySet()) {
 +          Key k = entry.getKey();
 +          m.putDelete(k.getColumnFamily(), k.getColumnQualifier(), k.getTimestamp());
 +        }
 +
 +        bw.addMutation(m);
 +
 +        for (Map<Key,Value> st : srcTablets)
 +          bw.addMutation(createCloneMutation(srcTableId, tableId, st));
 +
 +        rewrites++;
 +      } else {
 +        // write out marker that this tablet was successfully cloned
 +        Mutation m = new Mutation(cloneTablet.keySet().iterator().next().getRow());
 +        m.put(ClonedColumnFamily.NAME, new Text(""), new Value("OK".getBytes(Constants.UTF8)));
 +        bw.addMutation(m);
 +      }
 +    }
 +
 +    bw.flush();
 +    return rewrites;
 +  }
 +
 +  public static void cloneTable(Instance instance, String srcTableId, String tableId, VolumeManager volumeManager) throws Exception {
 +
 +    Connector conn = instance.getConnector(SystemCredentials.get().getPrincipal(), SystemCredentials.get().getToken());
 +    BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
 +
 +    while (true) {
 +
 +      try {
 +        initializeClone(srcTableId, tableId, conn, bw);
 +
 +        // the following loop looks changes in the file that occurred during the copy.. if files were dereferenced then they could have been GCed
 +
 +        while (true) {
 +          int rewrites = checkClone(srcTableId, tableId, conn, bw);
 +
 +          if (rewrites == 0)
 +            break;
 +        }
 +
 +        bw.flush();
 +        break;
 +
 +      } catch (TabletIterator.TabletDeletedException tde) {
 +        // tablets were merged in the src table
 +        bw.flush();
 +
 +        // delete what we have cloned and try again
 +        deleteTable(tableId, false, SystemCredentials.get(), null);
 +
 +        log.debug("Tablets merged in table " + srcTableId + " while attempting to clone, trying again");
 +
 +        UtilWaitThread.sleep(100);
 +      }
 +    }
 +
 +    // delete the clone markers and create directory entries
 +    Scanner mscanner = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
 +    mscanner.setRange(new KeyExtent(new Text(tableId), null, null).toMetadataRange());
 +    mscanner.fetchColumnFamily(ClonedColumnFamily.NAME);
 +
 +    int dirCount = 0;
 +
 +    for (Entry<Key,Value> entry : mscanner) {
 +      Key k = entry.getKey();
 +      Mutation m = new Mutation(k.getRow());
 +      m.putDelete(k.getColumnFamily(), k.getColumnQualifier());
 +      String dir = volumeManager.choose(ServerConstants.getTablesDirs()) + "/" + tableId
 +          + new String(FastFormat.toZeroPaddedString(dirCount++, 8, 16, "/c-".getBytes(Constants.UTF8)));
 +      TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(m, new Value(dir.getBytes(Constants.UTF8)));
 +      bw.addMutation(m);
 +    }
 +
 +    bw.close();
 +
 +  }
 +
 +  public static void chopped(KeyExtent extent, ZooLock zooLock) {
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +    ChoppedColumnFamily.CHOPPED_COLUMN.put(m, new Value("chopped".getBytes(Constants.UTF8)));
 +    update(SystemCredentials.get(), zooLock, m, extent);
 +  }
 +
 +  public static void removeBulkLoadEntries(Connector conn, String tableId, long tid) throws Exception {
 +    Scanner mscanner = new IsolatedScanner(conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY));
 +    mscanner.setRange(new KeyExtent(new Text(tableId), null, null).toMetadataRange());
 +    mscanner.fetchColumnFamily(TabletsSection.BulkFileColumnFamily.NAME);
 +    BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
 +    for (Entry<Key,Value> entry : mscanner) {
 +      log.debug("Looking at entry " + entry + " with tid " + tid);
 +      if (Long.parseLong(entry.getValue().toString()) == tid) {
 +        log.debug("deleting entry " + entry);
 +        Mutation m = new Mutation(entry.getKey().getRow());
 +        m.putDelete(entry.getKey().getColumnFamily(), entry.getKey().getColumnQualifier());
 +        bw.addMutation(m);
 +      }
 +    }
 +    bw.close();
 +  }
 +
 +  public static List<FileRef> getBulkFilesLoaded(Connector conn, KeyExtent extent, long tid) throws IOException {
 +    List<FileRef> result = new ArrayList<FileRef>();
 +    try {
 +      VolumeManager fs = VolumeManagerImpl.get();
 +      Scanner mscanner = new IsolatedScanner(conn.createScanner(extent.isMeta() ? RootTable.NAME : MetadataTable.NAME, Authorizations.EMPTY));
 +      mscanner.setRange(extent.toMetadataRange());
 +      mscanner.fetchColumnFamily(TabletsSection.BulkFileColumnFamily.NAME);
 +      for (Entry<Key,Value> entry : mscanner) {
 +        if (Long.parseLong(entry.getValue().toString()) == tid) {
 +          result.add(new FileRef(fs, entry.getKey()));
 +        }
 +      }
 +      return result;
 +    } catch (TableNotFoundException ex) {
 +      // unlikely
 +      throw new RuntimeException("Onos! teh metadata table has vanished!!");
 +    }
 +  }
 +
 +  public static Map<FileRef,Long> getBulkFilesLoaded(Credentials credentials, KeyExtent extent) throws IOException {
 +    Text metadataRow = extent.getMetadataEntry();
 +    Map<FileRef,Long> ret = new HashMap<FileRef,Long>();
 +
 +    VolumeManager fs = VolumeManagerImpl.get();
 +    Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, extent.isMeta() ? RootTable.ID : MetadataTable.ID, Authorizations.EMPTY);
 +    scanner.setRange(new Range(metadataRow));
 +    scanner.fetchColumnFamily(TabletsSection.BulkFileColumnFamily.NAME);
 +    for (Entry<Key,Value> entry : scanner) {
 +      Long tid = Long.parseLong(entry.getValue().toString());
 +      ret.put(new FileRef(fs, entry.getKey()), tid);
 +    }
 +    return ret;
 +  }
 +
 +  public static void addBulkLoadInProgressFlag(String path) {
 +
 +    Mutation m = new Mutation(MetadataSchema.BlipSection.getRowPrefix() + path);
 +    m.put(EMPTY_TEXT, EMPTY_TEXT, new Value(new byte[] {}));
 +
 +    // new KeyExtent is only added to force update to write to the metadata table, not the root table
 +    // because bulk loads aren't supported to the metadata table
 +    update(SystemCredentials.get(), m, new KeyExtent(new Text("anythingNotMetadata"), null, null));
 +  }
 +
 +  public static void removeBulkLoadInProgressFlag(String path) {
 +
 +    Mutation m = new Mutation(MetadataSchema.BlipSection.getRowPrefix() + path);
 +    m.putDelete(EMPTY_TEXT, EMPTY_TEXT);
 +
 +    // new KeyExtent is only added to force update to write to the metadata table, not the root table
 +    // because bulk loads aren't supported to the metadata table
 +    update(SystemCredentials.get(), m, new KeyExtent(new Text("anythingNotMetadata"), null, null));
 +  }
 +
++  /**
++   * During an upgrade we need to move deletion requests for files under the !METADATA table to the root tablet.
++   */
 +  public static void moveMetaDeleteMarkers(Instance instance, Credentials creds) {
 +    // move old delete markers to new location, to standardize table schema between all metadata tables
 +    byte[] EMPTY_BYTES = new byte[0];
 +    Scanner scanner = new ScannerImpl(instance, creds, RootTable.ID, Authorizations.EMPTY);
 +    String oldDeletesPrefix = "!!~del";
 +    Range oldDeletesRange = new Range(oldDeletesPrefix, true, "!!~dem", false);
 +    scanner.setRange(oldDeletesRange);
 +    for (Entry<Key,Value> entry : scanner) {
 +      String row = entry.getKey().getRow().toString();
 +      if (row.startsWith(oldDeletesPrefix)) {
 +        String filename = row.substring(oldDeletesPrefix.length());
 +        // add the new entry first
 +        log.info("Moving " + filename + " marker in " + RootTable.NAME);
 +        Mutation m = new Mutation(MetadataSchema.DeletesSection.getRowPrefix() + filename);
 +        m.put(EMPTY_BYTES, EMPTY_BYTES, EMPTY_BYTES);
 +        update(creds, m, RootTable.EXTENT);
 +        // remove the old entry
 +        m = new Mutation(entry.getKey().getRow());
 +        m.putDelete(EMPTY_BYTES, EMPTY_BYTES);
 +        update(creds, m, RootTable.OLD_EXTENT);
 +      } else {
 +        break;
 +      }
 +    }
 +
 +  }
 +
 +  public static SortedMap<Text,SortedMap<ColumnFQ,Value>> getTabletEntries(SortedMap<Key,Value> tabletKeyValues, List<ColumnFQ> columns) {
 +    TreeMap<Text,SortedMap<ColumnFQ,Value>> tabletEntries = new TreeMap<Text,SortedMap<ColumnFQ,Value>>();
 +
 +    HashSet<ColumnFQ> colSet = null;
 +    if (columns != null) {
 +      colSet = new HashSet<ColumnFQ>(columns);
 +    }
 +
 +    for (Entry<Key,Value> entry : tabletKeyValues.entrySet()) {
 +
 +      if (columns != null && !colSet.contains(new ColumnFQ(entry.getKey()))) {
 +        continue;
 +      }
 +
 +      Text row = entry.getKey().getRow();
 +
 +      SortedMap<ColumnFQ,Value> colVals = tabletEntries.get(row);
 +      if (colVals == null) {
 +        colVals = new TreeMap<ColumnFQ,Value>();
 +        tabletEntries.put(row, colVals);
 +      }
 +
 +      colVals.put(new ColumnFQ(entry.getKey()), entry.getValue());
 +    }
 +
 +    return tabletEntries;
 +  }
 +}


[07/15] Merge branch '1.5.2-SNAPSHOT' into 1.6.0-SNAPSHOT

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/957c9d1b/server/master/src/main/java/org/apache/accumulo/master/Master.java
----------------------------------------------------------------------
diff --cc server/master/src/main/java/org/apache/accumulo/master/Master.java
index e123b49,0000000..2440ee4
mode 100644,000000..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
@@@ -1,1227 -1,0 +1,1252 @@@
 +/*
 + * 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.io.IOException;
 +import java.util.ArrayList;
 +import java.util.Collection;
 +import java.util.Collections;
 +import java.util.HashSet;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +import java.util.SortedMap;
 +import java.util.TreeMap;
++import java.util.concurrent.CountDownLatch;
 +import java.util.concurrent.atomic.AtomicBoolean;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +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.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.TableOperation;
 +import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
 +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.master.state.tables.TableState;
 +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.MasterState;
 +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 +import org.apache.accumulo.core.metadata.MetadataTable;
 +import org.apache.accumulo.core.metadata.RootTable;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 +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.util.Daemon;
 +import org.apache.accumulo.core.util.Pair;
 +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.zookeeper.IZooReaderWriter;
 +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.server.Accumulo;
 +import org.apache.accumulo.server.ServerConstants;
 +import org.apache.accumulo.server.ServerOpts;
 +import org.apache.accumulo.server.client.HdfsZooInstance;
 +import org.apache.accumulo.server.conf.ServerConfiguration;
 +import org.apache.accumulo.server.fs.VolumeManager;
 +import org.apache.accumulo.server.fs.VolumeManager.FileType;
 +import org.apache.accumulo.server.fs.VolumeManagerImpl;
 +import org.apache.accumulo.server.init.Initialize;
 +import org.apache.accumulo.server.master.LiveTServerSet;
 +import org.apache.accumulo.server.master.LiveTServerSet.TServerConnection;
 +import org.apache.accumulo.server.master.balancer.DefaultLoadBalancer;
 +import org.apache.accumulo.server.master.balancer.TabletBalancer;
 +import org.apache.accumulo.server.master.state.CurrentState;
 +import org.apache.accumulo.server.master.state.DeadServerList;
 +import org.apache.accumulo.server.master.state.MergeInfo;
 +import org.apache.accumulo.server.master.state.MergeState;
 +import org.apache.accumulo.server.master.state.MetaDataStateStore;
 +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.TabletState;
 +import org.apache.accumulo.server.master.state.ZooStore;
 +import org.apache.accumulo.server.master.state.ZooTabletStateStore;
 +import org.apache.accumulo.server.security.AuditedSecurityOperation;
 +import org.apache.accumulo.server.security.SecurityOperation;
 +import org.apache.accumulo.server.security.SystemCredentials;
 +import org.apache.accumulo.server.security.handler.ZKPermHandler;
 +import org.apache.accumulo.server.tables.TableManager;
 +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.TServerUtils;
 +import org.apache.accumulo.server.util.TServerUtils.ServerAddress;
 +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.hadoop.fs.Path;
 +import org.apache.hadoop.io.DataInputBuffer;
 +import org.apache.hadoop.io.DataOutputBuffer;
 +import org.apache.hadoop.io.Text;
 +import org.apache.log4j.Logger;
 +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.WatchedEvent;
 +import org.apache.zookeeper.Watcher;
 +import org.apache.zookeeper.data.Stat;
 +
 +import com.google.common.collect.Iterables;
 +
 +/**
 + * The Master is responsible for assigning and balancing tablets to tablet servers.
 + * 
 + * The master will also coordinate log recoveries and reports general status.
 + */
 +public class Master implements LiveTServerSet.Listener, TableObserver, CurrentState {
 +
 +  final static Logger log = Logger.getLogger(Master.class);
 +
 +  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;
 +  final private static long TIME_BETWEEN_MIGRATION_CLEANUPS = 5 * 60 * ONE_SECOND;
 +  final static long WAIT_BETWEEN_ERRORS = ONE_SECOND;
 +  final private static long DEFAULT_WAIT_FOR_WATCHER = 10 * ONE_SECOND;
 +  final private static int MAX_CLEANUP_WAIT_TIME = ONE_SECOND;
 +  final private static int TIME_TO_WAIT_BETWEEN_LOCK_CHECKS = ONE_SECOND;
 +  final static int MAX_TSERVER_WORK_CHUNK = 5000;
 +  final private static int MAX_BAD_STATUS_COUNT = 3;
 +
 +  final VolumeManager fs;
 +  final private Instance instance;
 +  final private String hostname;
 +  final LiveTServerSet tserverSet;
 +  final private List<TabletGroupWatcher> watchers = new ArrayList<TabletGroupWatcher>();
 +  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();
 +  final private Object mergeLock = new Object();
 +  RecoveryManager recoveryManager = null;
 +
 +  ZooLock masterLock = null;
 +  private TServer clientService = null;
 +  TabletBalancer tabletBalancer;
 +
 +  private MasterState state = MasterState.INITIAL;
 +
 +  Fate<Master> fate;
 +
 +  volatile SortedMap<TServerInstance,TabletServerStatus> tserverStatus = Collections.unmodifiableSortedMap(new TreeMap<TServerInstance,TabletServerStatus>());
 +
 +  synchronized MasterState getMasterState() {
 +    return state;
 +  }
 +
 +  public boolean stillMaster() {
 +    return getMasterState() != MasterState.STOP;
 +  }
 +
 +  static final boolean X = true;
 +  static final boolean _ = false;
 +  // @formatter:off
 +  static final boolean transitionOK[][] = {
 +      //                              INITIAL HAVE_LOCK SAFE_MODE NORMAL UNLOAD_META UNLOAD_ROOT STOP
 +      /* INITIAL */                   {X,     X,        _,        _,      _,         _,          X},
 +      /* HAVE_LOCK */                 {_,     X,        X,        X,      _,         _,          X},
 +      /* SAFE_MODE */                 {_,     _,        X,        X,      X,         _,          X},
 +      /* NORMAL */                    {_,     _,        X,        X,      X,         _,          X},
 +      /* UNLOAD_METADATA_TABLETS */   {_,     _,        X,        X,      X,         X,          X},
 +      /* UNLOAD_ROOT_TABLET */        {_,     _,        _,        X,      X,         X,          X},
 +      /* STOP */                      {_,     _,        _,        _,      _,         X,          X}};
 +  //@formatter:on
 +  synchronized void setMasterState(MasterState newState) {
 +    if (state.equals(newState))
 +      return;
 +    if (!transitionOK[state.ordinal()][newState.ordinal()]) {
 +      log.error("Programmer error: master should not transition from " + state + " to " + newState);
 +    }
 +    MasterState oldState = state;
 +    state = newState;
 +    nextEvent.event("State changed from %s to %s", oldState, newState);
 +    if (newState == MasterState.STOP) {
 +      // Give the server a little time before shutdown so the client
 +      // thread requesting the stop can return
 +      SimpleTimer.getInstance().schedule(new Runnable() {
 +        @Override
 +        public void run() {
 +          // This frees the main thread and will cause the master to exit
 +          clientService.stop();
 +          Master.this.nextEvent.event("stopped event loop");
 +        }
 +
 +      }, 100l, 1000l);
 +    }
 +
 +    if (oldState != newState && (newState == MasterState.HAVE_LOCK)) {
 +      upgradeZookeeper();
 +    }
 +
 +    if (oldState != newState && (newState == MasterState.NORMAL)) {
 +      upgradeMetadata();
 +    }
 +  }
 +
 +  private void moveRootTabletToRootTable(IZooReaderWriter zoo) throws Exception {
 +    String dirZPath = ZooUtil.getRoot(instance) + RootTable.ZROOT_TABLET_PATH;
 +
 +    if (!zoo.exists(dirZPath)) {
 +      Path oldPath = fs.getFullPath(FileType.TABLE, "/" + MetadataTable.ID + "/root_tablet");
 +      if (fs.exists(oldPath)) {
 +        String newPath = fs.choose(ServerConstants.getTablesDirs()) + "/" + RootTable.ID;
 +        fs.mkdirs(new Path(newPath));
 +        if (!fs.rename(oldPath, new Path(newPath))) {
 +          throw new IOException("Failed to move root tablet from " + oldPath + " to " + newPath);
 +        }
 +
 +        log.info("Upgrade renamed " + oldPath + " to " + newPath);
 +      }
 +
 +      Path location = null;
 +
 +      for (String basePath : ServerConstants.getTablesDirs()) {
 +        Path path = new Path(basePath + "/" + RootTable.ID + RootTable.ROOT_TABLET_LOCATION);
 +        if (fs.exists(path)) {
 +          if (location != null) {
 +            throw new IllegalStateException("Root table at multiple locations " + location + " " + path);
 +          }
 +
 +          location = path;
 +        }
 +      }
 +
 +      if (location == null)
 +        throw new IllegalStateException("Failed to find root tablet");
 +
 +      log.info("Upgrade setting root table location in zookeeper " + location);
 +      zoo.putPersistentData(dirZPath, location.toString().getBytes(), NodeExistsPolicy.FAIL);
 +    }
 +  }
 +
++  private boolean haveUpgradedZooKeeper = false;
++
 +  private void upgradeZookeeper() {
 +    // 1.5.1 and 1.6.0 both do some state checking after obtaining the zoolock for the
 +    // monitor and before starting up. It's not tied to the data version at all (and would
 +    // introduce unnecessary complexity to try to make the master do it), but be aware
 +    // that the master is not the only thing that may alter zookeeper before starting.
 +
 +    if (Accumulo.getAccumuloPersistentVersion(fs) == ServerConstants.PREV_DATA_VERSION) {
++      // This Master hasn't started Fate yet, so any outstanding transactions must be from before the upgrade.
++      // Change to Guava's Verify once we use Guava 17.
++      if (null != fate) {
++        throw new IllegalStateException("Access to Fate should not have been initialized prior to the Master transitioning to active. Please save all logs and file a bug.");
++      }
++      Accumulo.abortIfFateTransactions();
 +      try {
 +        log.info("Upgrading zookeeper");
 +
 +        IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +
 +        // create initial namespaces
 +        String namespaces = ZooUtil.getRoot(instance) + Constants.ZNAMESPACES;
 +        zoo.putPersistentData(namespaces, new byte[0], NodeExistsPolicy.SKIP);
 +        for (Pair<String,String> namespace : Iterables.concat(
 +            Collections.singleton(new Pair<String,String>(Namespaces.ACCUMULO_NAMESPACE, Namespaces.ACCUMULO_NAMESPACE_ID)),
 +            Collections.singleton(new Pair<String,String>(Namespaces.DEFAULT_NAMESPACE, Namespaces.DEFAULT_NAMESPACE_ID)))) {
 +          String ns = namespace.getFirst();
 +          String id = namespace.getSecond();
 +          log.debug("Upgrade creating namespace \"" + ns + "\" (ID: " + id + ")");
 +          if (!Namespaces.exists(instance, id))
 +            TableManager.prepareNewNamespaceState(instance.getInstanceID(), id, ns, NodeExistsPolicy.SKIP);
 +        }
 +
 +        // create root table
 +        log.debug("Upgrade creating table " + RootTable.NAME + " (ID: " + RootTable.ID + ")");
 +        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
 +        security.grantTablePermission(SystemCredentials.get().toThrift(instance), security.getRootUsername(), RootTable.ID, TablePermission.ALTER_TABLE, Namespaces.ACCUMULO_NAMESPACE_ID);
 +
 +        // put existing tables in the correct namespaces
 +        String tables = ZooUtil.getRoot(instance) + Constants.ZTABLES;
 +        for (String tableId : zoo.getChildren(tables)) {
 +          String targetNamespace = (MetadataTable.ID.equals(tableId) || RootTable.ID.equals(tableId)) ? Namespaces.ACCUMULO_NAMESPACE_ID
 +              : Namespaces.DEFAULT_NAMESPACE_ID;
 +          log.debug("Upgrade moving table " + new String(zoo.getData(tables + "/" + tableId + Constants.ZTABLE_NAME, null), Constants.UTF8) + " (ID: "
 +              + tableId + ") into namespace with ID " + targetNamespace);
 +          zoo.putPersistentData(tables + "/" + tableId + Constants.ZTABLE_NAMESPACE, targetNamespace.getBytes(Constants.UTF8), NodeExistsPolicy.SKIP);
 +        }
 +
 +        // rename metadata table
 +        log.debug("Upgrade renaming table " + MetadataTable.OLD_NAME + " (ID: " + MetadataTable.ID + ") to " + MetadataTable.NAME);
 +        zoo.putPersistentData(tables + "/" + MetadataTable.ID + Constants.ZTABLE_NAME, Tables.qualify(MetadataTable.NAME).getSecond().getBytes(Constants.UTF8),
 +            NodeExistsPolicy.OVERWRITE);
 +
 +        moveRootTabletToRootTable(zoo);
 +
 +        // add system namespace permissions to existing users
 +        ZKPermHandler perm = new ZKPermHandler();
 +        perm.initialize(instance.getInstanceID(), true);
 +        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, Namespaces.ACCUMULO_NAMESPACE_ID, NamespacePermission.READ);
 +        }
 +        perm.grantNamespacePermission("root", Namespaces.ACCUMULO_NAMESPACE_ID, NamespacePermission.ALTER_TABLE);
- 
++        haveUpgradedZooKeeper = true;
 +      } catch (Exception ex) {
 +        log.fatal("Error performing upgrade", ex);
 +        System.exit(1);
 +      }
 +    }
 +  }
 +
 +  private final AtomicBoolean upgradeMetadataRunning = new AtomicBoolean(false);
++  private final CountDownLatch waitForMetadataUpgrade = new CountDownLatch(1);
 +
 +  private final ServerConfiguration serverConfig;
 +
 +  private void upgradeMetadata() {
-     if (Accumulo.getAccumuloPersistentVersion(fs) == ServerConstants.PREV_DATA_VERSION) {
-       if (upgradeMetadataRunning.compareAndSet(false, true)) {
++    // we make sure we're only doing the rest of this method once so that we can signal to other threads that an upgrade wasn't needed.
++    if (upgradeMetadataRunning.compareAndSet(false, true)) {
++      if (Accumulo.getAccumuloPersistentVersion(fs) == ServerConstants.PREV_DATA_VERSION) {
++        // sanity check that we passed the Fate verification prior to ZooKeeper upgrade, and that Fate still hasn't been started.
++        // Change both to use Guava's Verify once we use Guava 17.
++        if (!haveUpgradedZooKeeper) {
++          throw new IllegalStateException("We should only attempt to upgrade Accumulo's !METADATA table if we've already upgraded ZooKeeper. Please save all logs and file a bug.");
++        }
++        if (null != fate) {
++          throw new IllegalStateException("Access to Fate should not have been initialized prior to the Master finishing upgrades. Please save all logs and file a bug.");
++        }
 +        Runnable upgradeTask = new Runnable() {
 +          @Override
 +          public void run() {
 +            try {
++              log.info("Starting to upgrade !METADATA table.");
 +              MetadataTableUtil.moveMetaDeleteMarkers(instance, SystemCredentials.get());
++              log.info("Updating persistent data version.");
 +              Accumulo.updateAccumuloVersion(fs);
- 
 +              log.info("Upgrade complete");
- 
++              waitForMetadataUpgrade.countDown();
 +            } catch (Exception ex) {
 +              log.fatal("Error performing upgrade", ex);
 +              System.exit(1);
 +            }
 +
 +          }
 +        };
 +
 +        // need to run this in a separate thread because a lock is held that prevents metadata tablets from being assigned and this task writes to the
 +        // metadata table
 +        new Thread(upgradeTask).start();
++      } else {
++        waitForMetadataUpgrade.countDown();
 +      }
 +    }
 +  }
 +
 +  private int assignedOrHosted(Text tableId) {
 +    int result = 0;
 +    for (TabletGroupWatcher watcher : watchers) {
 +      TableCounts count = watcher.getStats(tableId);
 +      result += count.hosted() + count.assigned();
 +    }
 +    return result;
 +  }
 +
 +  private int totalAssignedOrHosted() {
 +    int result = 0;
 +    for (TabletGroupWatcher watcher : watchers) {
 +      for (TableCounts counts : watcher.getStats().values()) {
 +        result += counts.assigned() + counts.hosted();
 +      }
 +    }
 +    return result;
 +  }
 +
 +  private int nonMetaDataTabletsAssignedOrHosted() {
 +    return totalAssignedOrHosted() - assignedOrHosted(new Text(MetadataTable.ID)) - assignedOrHosted(new Text(RootTable.ID));
 +  }
 +
 +  private int notHosted() {
 +    int result = 0;
 +    for (TabletGroupWatcher watcher : watchers) {
 +      for (TableCounts counts : watcher.getStats().values()) {
 +        result += counts.assigned() + counts.assignedToDeadServers();
 +      }
 +    }
 +    return result;
 +  }
 +
 +  // The number of unassigned tablets that should be assigned: displayed on the monitor page
 +  int displayUnassigned() {
 +    int result = 0;
 +    switch (getMasterState()) {
 +      case NORMAL:
 +        // Count offline tablets for online tables
 +        for (TabletGroupWatcher watcher : watchers) {
 +          TableManager manager = TableManager.getInstance();
 +          for (Entry<Text,TableCounts> entry : watcher.getStats().entrySet()) {
 +            Text tableId = entry.getKey();
 +            TableCounts counts = entry.getValue();
 +            TableState tableState = manager.getTableState(tableId.toString());
 +            if (tableState != null && tableState.equals(TableState.ONLINE)) {
 +              result += counts.unassigned() + counts.assignedToDeadServers() + counts.assigned();
 +            }
 +          }
 +        }
 +        break;
 +      case SAFE_MODE:
 +        // Count offline tablets for the metadata table
 +        for (TabletGroupWatcher watcher : watchers) {
 +          result += watcher.getStats(METADATA_TABLE_ID).unassigned();
 +        }
 +        break;
 +      case UNLOAD_METADATA_TABLETS:
 +      case UNLOAD_ROOT_TABLET:
 +        for (TabletGroupWatcher watcher : watchers) {
 +          result += watcher.getStats(METADATA_TABLE_ID).unassigned();
 +        }
 +        break;
 +      default:
 +        break;
 +    }
 +    return result;
 +  }
 +
 +  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");
 +  }
 +
 +  public Connector getConnector() throws AccumuloException, AccumuloSecurityException {
 +    return instance.getConnector(SystemCredentials.get().getPrincipal(), SystemCredentials.get().getToken());
 +  }
 +
 +  private Master(ServerConfiguration config, VolumeManager fs, String hostname) throws IOException {
 +    this.serverConfig = config;
 +    this.instance = config.getInstance();
 +    this.fs = fs;
 +    this.hostname = hostname;
 +
 +    AccumuloConfiguration aconf = serverConfig.getConfiguration();
 +
 +    log.info("Version " + Constants.VERSION);
 +    log.info("Instance " + instance.getInstanceID());
 +    ThriftTransportPool.getInstance().setIdleTime(aconf.getTimeInMillis(Property.GENERAL_RPC_TIMEOUT));
 +    security = AuditedSecurityOperation.getInstance();
 +    tserverSet = new LiveTServerSet(instance, config.getConfiguration(), this);
 +    this.tabletBalancer = aconf.instantiateClassProperty(Property.MASTER_TABLET_BALANCER, TabletBalancer.class, new DefaultLoadBalancer());
 +    this.tabletBalancer.init(serverConfig);
 +  }
 +
 +  public TServerConnection getConnection(TServerInstance server) {
 +    return tserverSet.getConnection(server);
 +  }
 +
 +  public MergeInfo getMergeInfo(Text tableId) {
 +    synchronized (mergeLock) {
 +      try {
 +        String path = ZooUtil.getRoot(instance.getInstanceID()) + Constants.ZTABLES + "/" + tableId.toString() + "/merge";
 +        if (!ZooReaderWriter.getInstance().exists(path))
 +          return new MergeInfo();
 +        byte[] data = ZooReaderWriter.getInstance().getData(path, new Stat());
 +        DataInputBuffer in = new DataInputBuffer();
 +        in.reset(data, data.length);
 +        MergeInfo info = new MergeInfo();
 +        info.readFields(in);
 +        return info;
 +      } catch (KeeperException.NoNodeException ex) {
 +        log.info("Error reading merge state, it probably just finished");
 +        return new MergeInfo();
 +      } catch (Exception ex) {
 +        log.warn("Unexpected error reading merge state", ex);
 +        return new MergeInfo();
 +      }
 +    }
 +  }
 +
 +  public void setMergeState(MergeInfo info, MergeState state) throws IOException, KeeperException, InterruptedException {
 +    synchronized (mergeLock) {
 +      String path = ZooUtil.getRoot(instance.getInstanceID()) + Constants.ZTABLES + "/" + info.getExtent().getTableId().toString() + "/merge";
 +      info.setState(state);
 +      if (state.equals(MergeState.NONE)) {
 +        ZooReaderWriter.getInstance().recursiveDelete(path, NodeMissingPolicy.SKIP);
 +      } else {
 +        DataOutputBuffer out = new DataOutputBuffer();
 +        try {
 +          info.write(out);
 +        } catch (IOException ex) {
 +          throw new RuntimeException("Unlikely", ex);
 +        }
 +        ZooReaderWriter.getInstance().putPersistentData(path, out.getData(),
 +            state.equals(MergeState.STARTED) ? ZooUtil.NodeExistsPolicy.FAIL : ZooUtil.NodeExistsPolicy.OVERWRITE);
 +      }
 +      mergeLock.notifyAll();
 +    }
 +    nextEvent.event("Merge state of %s set to %s", info.getExtent(), state);
 +  }
 +
 +  public void clearMergeState(Text tableId) throws IOException, KeeperException, InterruptedException {
 +    synchronized (mergeLock) {
 +      String path = ZooUtil.getRoot(instance.getInstanceID()) + Constants.ZTABLES + "/" + tableId.toString() + "/merge";
 +      ZooReaderWriter.getInstance().recursiveDelete(path, NodeMissingPolicy.SKIP);
 +      mergeLock.notifyAll();
 +    }
 +    nextEvent.event("Merge state of %s cleared", tableId);
 +  }
 +
 +  void setMasterGoalState(MasterGoalState state) {
 +    try {
 +      ZooReaderWriter.getInstance().putPersistentData(ZooUtil.getRoot(instance) + Constants.ZMASTER_GOAL_STATE, state.name().getBytes(),
 +          NodeExistsPolicy.OVERWRITE);
 +    } catch (Exception ex) {
 +      log.error("Unable to set master goal state in zookeeper");
 +    }
 +  }
 +
 +  MasterGoalState getMasterGoalState() {
 +    while (true)
 +      try {
 +        byte[] data = ZooReaderWriter.getInstance().getData(ZooUtil.getRoot(instance) + Constants.ZMASTER_GOAL_STATE, null);
 +        return MasterGoalState.valueOf(new String(data));
 +      } catch (Exception e) {
 +        log.error("Problem getting real goal state: " + e);
 +        UtilWaitThread.sleep(1000);
 +      }
 +  }
 +
 +  public boolean hasCycled(long time) {
 +    for (TabletGroupWatcher watcher : watchers) {
 +      if (watcher.stats.lastScanFinished() < time)
 +        return false;
 +    }
 +
 +    return true;
 +  }
 +
 +  public void clearMigrations(String tableId) {
 +    synchronized (migrations) {
 +      Iterator<KeyExtent> iterator = migrations.keySet().iterator();
 +      while (iterator.hasNext()) {
 +        KeyExtent extent = iterator.next();
 +        if (extent.getTableId().toString().equals(tableId)) {
 +          iterator.remove();
 +        }
 +      }
 +    }
 +  }
 +
 +  static enum TabletGoalState {
 +    HOSTED, UNASSIGNED, DELETED
 +  };
 +
 +  TabletGoalState getSystemGoalState(TabletLocationState tls) {
 +    switch (getMasterState()) {
 +      case NORMAL:
 +        return TabletGoalState.HOSTED;
 +      case HAVE_LOCK: // fall-through intended
 +      case INITIAL: // fall-through intended
 +      case SAFE_MODE:
 +        if (tls.extent.isMeta())
 +          return TabletGoalState.HOSTED;
 +        return TabletGoalState.UNASSIGNED;
 +      case UNLOAD_METADATA_TABLETS:
 +        if (tls.extent.isRootTablet())
 +          return TabletGoalState.HOSTED;
 +        return TabletGoalState.UNASSIGNED;
 +      case UNLOAD_ROOT_TABLET:
 +        return TabletGoalState.UNASSIGNED;
 +      case STOP:
 +        return TabletGoalState.UNASSIGNED;
 +      default:
 +        throw new IllegalStateException("Unknown Master State");
 +    }
 +  }
 +
 +  TabletGoalState getTableGoalState(KeyExtent extent) {
 +    TableState tableState = TableManager.getInstance().getTableState(extent.getTableId().toString());
 +    if (tableState == null)
 +      return TabletGoalState.DELETED;
 +    switch (tableState) {
 +      case DELETING:
 +        return TabletGoalState.DELETED;
 +      case OFFLINE:
 +      case NEW:
 +        return TabletGoalState.UNASSIGNED;
 +      default:
 +        return TabletGoalState.HOSTED;
 +    }
 +  }
 +
 +  TabletGoalState getGoalState(TabletLocationState tls, MergeInfo mergeInfo) {
 +    KeyExtent extent = tls.extent;
 +    // Shutting down?
 +    TabletGoalState state = getSystemGoalState(tls);
 +    if (state == TabletGoalState.HOSTED) {
 +      if (tls.current != null && serversToShutdown.contains(tls.current)) {
 +        return TabletGoalState.UNASSIGNED;
 +      }
 +      // Handle merge transitions
 +      if (mergeInfo.getExtent() != null) {
 +        log.debug("mergeInfo overlaps: " + extent + " " + mergeInfo.overlaps(extent));
 +        if (mergeInfo.overlaps(extent)) {
 +          switch (mergeInfo.getState()) {
 +            case NONE:
 +            case COMPLETE:
 +              break;
 +            case STARTED:
 +            case SPLITTING:
 +              return TabletGoalState.HOSTED;
 +            case WAITING_FOR_CHOPPED:
 +              if (tls.getState(onlineTabletServers()).equals(TabletState.HOSTED)) {
 +                if (tls.chopped)
 +                  return TabletGoalState.UNASSIGNED;
 +              } else {
 +                if (tls.chopped && tls.walogs.isEmpty())
 +                  return TabletGoalState.UNASSIGNED;
 +              }
 +
 +              return TabletGoalState.HOSTED;
 +            case WAITING_FOR_OFFLINE:
 +            case MERGING:
 +              return TabletGoalState.UNASSIGNED;
 +          }
 +        }
 +      }
 +
 +      // taking table offline?
 +      state = getTableGoalState(extent);
 +      if (state == TabletGoalState.HOSTED) {
 +        // Maybe this tablet needs to be migrated
 +        TServerInstance dest = migrations.get(extent);
 +        if (dest != null && tls.current != null && !dest.equals(tls.current)) {
 +          return TabletGoalState.UNASSIGNED;
 +        }
 +      }
 +    }
 +    return state;
 +  }
 +
 +  private class MigrationCleanupThread extends Daemon {
 +
 +    @Override
 +    public void run() {
 +      setName("Migration Cleanup Thread");
 +      while (stillMaster()) {
 +        if (!migrations.isEmpty()) {
 +          try {
 +            cleanupMutations();
 +          } catch (Exception ex) {
 +            log.error("Error cleaning up migrations", ex);
 +          }
 +        }
 +        UtilWaitThread.sleep(TIME_BETWEEN_MIGRATION_CLEANUPS);
 +      }
 +    }
 +
 +    // If a migrating tablet splits, and the tablet dies before sending the
 +    // master a message, the migration will refer to a non-existing tablet,
 +    // so it can never complete. Periodically scan the metadata table and
 +    // remove any migrating tablets that no longer exist.
 +    private void cleanupMutations() throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
 +      Connector connector = getConnector();
 +      Scanner scanner = connector.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
 +      TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
 +      Set<KeyExtent> found = new HashSet<KeyExtent>();
 +      for (Entry<Key,Value> entry : scanner) {
 +        KeyExtent extent = new KeyExtent(entry.getKey().getRow(), entry.getValue());
 +        if (migrations.containsKey(extent)) {
 +          found.add(extent);
 +        }
 +      }
 +      migrations.keySet().retainAll(found);
 +    }
 +  }
 +
 +  private class StatusThread extends Daemon {
 +
 +    @Override
 +    public void run() {
 +      setName("Status Thread");
 +      EventCoordinator.Listener eventListener = nextEvent.getListener();
 +      while (stillMaster()) {
 +        long wait = DEFAULT_WAIT_FOR_WATCHER;
 +        try {
 +          switch (getMasterGoalState()) {
 +            case NORMAL:
 +              setMasterState(MasterState.NORMAL);
 +              break;
 +            case SAFE_MODE:
 +              if (getMasterState() == MasterState.NORMAL) {
 +                setMasterState(MasterState.SAFE_MODE);
 +              }
 +              if (getMasterState() == MasterState.HAVE_LOCK) {
 +                setMasterState(MasterState.SAFE_MODE);
 +              }
 +              break;
 +            case CLEAN_STOP:
 +              switch (getMasterState()) {
 +                case NORMAL:
 +                  setMasterState(MasterState.SAFE_MODE);
 +                  break;
 +                case SAFE_MODE: {
 +                  int count = nonMetaDataTabletsAssignedOrHosted();
 +                  log.debug(String.format("There are %d non-metadata tablets assigned or hosted", count));
 +                  if (count == 0)
 +                    setMasterState(MasterState.UNLOAD_METADATA_TABLETS);
 +                }
 +                  break;
 +                case UNLOAD_METADATA_TABLETS: {
 +                  int count = assignedOrHosted(METADATA_TABLE_ID);
 +                  log.debug(String.format("There are %d metadata tablets assigned or hosted", count));
 +                  if (count == 0)
 +                    setMasterState(MasterState.UNLOAD_ROOT_TABLET);
 +                }
 +                  break;
 +                case UNLOAD_ROOT_TABLET: {
 +                  int count = assignedOrHosted(METADATA_TABLE_ID);
 +                  if (count > 0) {
 +                    log.debug(String.format("%d metadata tablets online", count));
 +                    setMasterState(MasterState.UNLOAD_ROOT_TABLET);
 +                  }
 +                  int root_count = assignedOrHosted(ROOT_TABLE_ID);
 +                  if (root_count > 0)
 +                    log.debug("The root tablet is still assigned or hosted");
 +                  if (count + root_count == 0) {
 +                    Set<TServerInstance> currentServers = tserverSet.getCurrentServers();
 +                    log.debug("stopping " + currentServers.size() + " tablet servers");
 +                    for (TServerInstance server : currentServers) {
 +                      try {
 +                        serversToShutdown.add(server);
 +                        tserverSet.getConnection(server).fastHalt(masterLock);
 +                      } catch (TException e) {
 +                        // its probably down, and we don't care
 +                      } finally {
 +                        tserverSet.remove(server);
 +                      }
 +                    }
 +                    if (currentServers.size() == 0)
 +                      setMasterState(MasterState.STOP);
 +                  }
 +                }
 +                  break;
 +                default:
 +                  break;
 +              }
 +          }
 +          wait = updateStatus();
 +          eventListener.waitForEvents(wait);
 +        } catch (Throwable t) {
 +          log.error("Error balancing tablets", t);
 +          UtilWaitThread.sleep(WAIT_BETWEEN_ERRORS);
 +        }
 +      }
 +    }
 +
 +    private long updateStatus() throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
 +      tserverStatus = Collections.synchronizedSortedMap(gatherTableInformation());
 +      checkForHeldServer(tserverStatus);
 +
 +      if (!badServers.isEmpty()) {
 +        log.debug("not balancing because the balance information is out-of-date " + badServers.keySet());
 +      } else if (notHosted() > 0) {
 +        log.debug("not balancing because there are unhosted tablets");
 +      } else if (getMasterGoalState() == MasterGoalState.CLEAN_STOP) {
 +        log.debug("not balancing because the master is attempting to stop cleanly");
 +      } else if (!serversToShutdown.isEmpty()) {
 +        log.debug("not balancing while shutting down servers " + serversToShutdown);
 +      } else {
 +        return balanceTablets();
 +      }
 +      return DEFAULT_WAIT_FOR_WATCHER;
 +    }
 +
 +    private void checkForHeldServer(SortedMap<TServerInstance,TabletServerStatus> tserverStatus) {
 +      TServerInstance instance = null;
 +      int crazyHoldTime = 0;
 +      int someHoldTime = 0;
 +      final long maxWait = getSystemConfiguration().getTimeInMillis(Property.TSERV_HOLD_TIME_SUICIDE);
 +      for (Entry<TServerInstance,TabletServerStatus> entry : tserverStatus.entrySet()) {
 +        if (entry.getValue().getHoldTime() > 0) {
 +          someHoldTime++;
 +          if (entry.getValue().getHoldTime() > maxWait) {
 +            instance = entry.getKey();
 +            crazyHoldTime++;
 +          }
 +        }
 +      }
 +      if (crazyHoldTime == 1 && someHoldTime == 1 && tserverStatus.size() > 1) {
 +        log.warn("Tablet server " + instance + " exceeded maximum hold time: attempting to kill it");
 +        try {
 +          TServerConnection connection = tserverSet.getConnection(instance);
 +          if (connection != null)
 +            connection.fastHalt(masterLock);
 +        } catch (TException e) {
 +          log.error(e, e);
 +        }
 +        tserverSet.remove(instance);
 +      }
 +    }
 +
 +    private long balanceTablets() {
 +      List<TabletMigration> migrationsOut = new ArrayList<TabletMigration>();
 +      Set<KeyExtent> migrationsCopy = new HashSet<KeyExtent>();
 +      synchronized (migrations) {
 +        migrationsCopy.addAll(migrations.keySet());
 +      }
 +      long wait = tabletBalancer.balance(Collections.unmodifiableSortedMap(tserverStatus), Collections.unmodifiableSet(migrationsCopy), migrationsOut);
 +
 +      for (TabletMigration m : TabletBalancer.checkMigrationSanity(tserverStatus.keySet(), migrationsOut)) {
 +        if (migrations.containsKey(m.tablet)) {
 +          log.warn("balancer requested migration more than once, skipping " + m);
 +          continue;
 +        }
 +        migrations.put(m.tablet, m.newServer);
 +        log.debug("migration " + m);
 +      }
 +      if (migrationsOut.size() > 0) {
 +        nextEvent.event("Migrating %d more tablets, %d total", migrationsOut.size(), migrations.size());
 +      }
 +      return wait;
 +    }
 +
 +  }
 +
 +  private SortedMap<TServerInstance,TabletServerStatus> gatherTableInformation() {
 +    long start = System.currentTimeMillis();
 +    SortedMap<TServerInstance,TabletServerStatus> result = new TreeMap<TServerInstance,TabletServerStatus>();
 +    Set<TServerInstance> currentServers = tserverSet.getCurrentServers();
 +    for (TServerInstance server : currentServers) {
 +      try {
 +        Thread t = Thread.currentThread();
 +        String oldName = t.getName();
 +        try {
 +          t.setName("Getting status from " + server);
 +          TServerConnection connection = tserverSet.getConnection(server);
 +          if (connection == null)
 +            throw new IOException("No connection to " + server);
 +          TabletServerStatus status = connection.getTableMap(false);
 +          result.put(server, status);
 +        } finally {
 +          t.setName(oldName);
 +        }
 +      } catch (Exception ex) {
 +        log.error("unable to get tablet server status " + server + " " + ex.toString());
 +        log.debug("unable to get tablet server status " + server, ex);
 +        if (badServers.get(server).incrementAndGet() > MAX_BAD_STATUS_COUNT) {
 +          log.warn("attempting to stop " + server);
 +          try {
 +            TServerConnection connection = tserverSet.getConnection(server);
 +            if (connection != null)
 +              connection.halt(masterLock);
 +          } catch (TTransportException e) {
 +            // ignore: it's probably down
 +          } catch (Exception e) {
 +            log.info("error talking to troublesome tablet server ", e);
 +          }
 +          badServers.remove(server);
 +          tserverSet.remove(server);
 +        }
 +      }
 +    }
 +    synchronized (badServers) {
 +      badServers.keySet().retainAll(currentServers);
 +      badServers.keySet().removeAll(result.keySet());
 +    }
 +    log.debug(String.format("Finished gathering information from %d servers in %.2f seconds", result.size(), (System.currentTimeMillis() - start) / 1000.));
 +    return result;
 +  }
 +
 +  public void run() throws IOException, InterruptedException, KeeperException {
 +    final String zroot = ZooUtil.getRoot(instance);
 +
 +    getMasterLock(zroot + Constants.ZMASTER_LOCK);
 +
 +    recoveryManager = new RecoveryManager(this);
 +
 +    TableManager.getInstance().addObserver(this);
 +
 +    StatusThread statusThread = new StatusThread();
 +    statusThread.start();
 +
 +    MigrationCleanupThread migrationCleanupThread = new MigrationCleanupThread();
 +    migrationCleanupThread.start();
 +
 +    tserverSet.startListeningForTabletServerChanges();
 +
-     try {
-       final AgeOffStore<Master> store = new AgeOffStore<Master>(new org.apache.accumulo.fate.ZooStore<Master>(ZooUtil.getRoot(instance) + Constants.ZFATE,
-           ZooReaderWriter.getRetryingInstance()), 1000 * 60 * 60 * 8);
- 
-       int threads = this.getConfiguration().getConfiguration().getCount(Property.MASTER_FATE_THREADPOOL_SIZE);
- 
-       fate = new Fate<Master>(this, store, threads);
- 
-       SimpleTimer.getInstance().schedule(new Runnable() {
- 
-         @Override
-         public void run() {
-           store.ageOff();
-         }
-       }, 63000, 63000);
-     } catch (KeeperException e) {
-       throw new IOException(e);
-     } catch (InterruptedException e) {
-       throw new IOException(e);
-     }
- 
 +    ZooReaderWriter.getInstance().getChildren(zroot + Constants.ZRECOVERY, new Watcher() {
 +      @Override
 +      public void process(WatchedEvent event) {
 +        nextEvent.event("Noticed recovery changes", event.getType());
 +        try {
 +          // watcher only fires once, add it back
 +          ZooReaderWriter.getInstance().getChildren(zroot + Constants.ZRECOVERY, this);
 +        } catch (Exception e) {
 +          log.error("Failed to add log recovery watcher back", e);
 +        }
 +      }
 +    });
 +
 +    Credentials systemCreds = SystemCredentials.get();
 +    watchers.add(new TabletGroupWatcher(this, new MetaDataStateStore(instance, systemCreds, this), null));
 +    watchers.add(new TabletGroupWatcher(this, new RootTabletStateStore(instance, systemCreds, this), watchers.get(0)));
 +    watchers.add(new TabletGroupWatcher(this, new ZooTabletStateStore(new ZooStore(zroot)), watchers.get(1)));
 +    for (TabletGroupWatcher watcher : watchers) {
 +      watcher.start();
 +    }
++
++    // Once we are sure the upgrade is complete, we can safely allow fate use.
++    waitForMetadataUpgrade.await();
++
++    try {
++      final AgeOffStore<Master> store = new AgeOffStore<Master>(new org.apache.accumulo.fate.ZooStore<Master>(ZooUtil.getRoot(instance) + Constants.ZFATE,
++          ZooReaderWriter.getRetryingInstance()), 1000 * 60 * 60 * 8);
++
++      int threads = this.getConfiguration().getConfiguration().getCount(Property.MASTER_FATE_THREADPOOL_SIZE);
++
++      fate = new Fate<Master>(this, store, threads);
++
++      SimpleTimer.getInstance().schedule(new Runnable() {
++
++        @Override
++        public void run() {
++          store.ageOff();
++        }
++      }, 63000, 63000);
++    } catch (KeeperException e) {
++      throw new IOException(e);
++    } catch (InterruptedException e) {
++      throw new IOException(e);
++    }
 +
 +    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;
 +    String address = sa.address.toString();
 +    log.info("Setting master lock data to " + address);
 +    masterLock.replaceLockData(address.getBytes());
 +
 +    while (!clientService.isServing()) {
 +      UtilWaitThread.sleep(100);
 +    }
 +    while (clientService.isServing()) {
 +      UtilWaitThread.sleep(500);
 +    }
 +    log.info("Shutting down fate.");
 +    fate.shutdown();
 +
 +    final long deadline = System.currentTimeMillis() + MAX_CLEANUP_WAIT_TIME;
 +    statusThread.join(remaining(deadline));
 +
 +    // quit, even if the tablet servers somehow jam up and the watchers
 +    // don't stop
 +    for (TabletGroupWatcher watcher : watchers) {
 +      watcher.join(remaining(deadline));
 +    }
 +    log.info("exiting");
 +  }
 +
 +  private long remaining(long deadline) {
 +    return Math.max(1, deadline - System.currentTimeMillis());
 +  }
 +
 +  public ZooLock getMasterLock() {
 +    return masterLock;
 +  }
 +
 +  private static class MasterLockWatcher implements ZooLock.AsyncLockWatcher {
 +
 +    boolean acquiredLock = false;
 +    boolean failedToAcquireLock = false;
 +
 +    @Override
 +    public void lostLock(LockLossReason reason) {
 +      Halt.halt("Master lock in zookeeper lost (reason = " + reason + "), exiting!", -1);
 +    }
 +
 +    @Override
 +    public void unableToMonitorLockNode(final Throwable e) {
 +      Halt.halt(-1, new Runnable() {
 +        @Override
 +        public void run() {
 +          log.fatal("No longer able to monitor master lock node", e);
 +        }
 +      });
 +
 +    }
 +
 +    @Override
 +    public synchronized void acquiredLock() {
 +      log.debug("Acquired master lock");
 +
 +      if (acquiredLock || failedToAcquireLock) {
 +        Halt.halt("Zoolock in unexpected state AL " + acquiredLock + " " + failedToAcquireLock, -1);
 +      }
 +
 +      acquiredLock = true;
 +      notifyAll();
 +    }
 +
 +    @Override
 +    public synchronized void failedToAcquireLock(Exception e) {
 +      log.warn("Failed to get master lock " + e);
 +
 +      if (acquiredLock) {
 +        Halt.halt("Zoolock in unexpected state FAL " + acquiredLock + " " + failedToAcquireLock, -1);
 +      }
 +
 +      failedToAcquireLock = true;
 +      notifyAll();
 +    }
 +
 +    public synchronized void waitForChange() {
 +      while (!acquiredLock && !failedToAcquireLock) {
 +        try {
 +          wait();
 +        } catch (InterruptedException e) {}
 +      }
 +    }
 +  }
 +
 +  private void getMasterLock(final String zMasterLoc) throws KeeperException, InterruptedException {
 +    log.info("trying to get master lock");
 +
 +    final String masterClientAddress = hostname + ":" + getSystemConfiguration().getPort(Property.MASTER_CLIENTPORT);
 +
 +    while (true) {
 +
 +      MasterLockWatcher masterLockWatcher = new MasterLockWatcher();
 +      masterLock = new ZooLock(zMasterLoc);
 +      masterLock.lockAsync(masterLockWatcher, masterClientAddress.getBytes());
 +
 +      masterLockWatcher.waitForChange();
 +
 +      if (masterLockWatcher.acquiredLock) {
 +        break;
 +      }
 +
 +      if (!masterLockWatcher.failedToAcquireLock) {
 +        throw new IllegalStateException("master lock in unknown state");
 +      }
 +
 +      masterLock.tryToCancelAsyncLockOrUnlock();
 +
 +      UtilWaitThread.sleep(TIME_TO_WAIT_BETWEEN_LOCK_CHECKS);
 +    }
 +
 +    setMasterState(MasterState.HAVE_LOCK);
 +  }
 +
 +  public static void main(String[] args) throws Exception {
 +    try {
 +      SecurityUtil.serverLogin(ServerConfiguration.getSiteConfiguration());
 +
 +      VolumeManager fs = VolumeManagerImpl.get();
 +      ServerOpts opts = new ServerOpts();
 +      opts.parseArgs("master", args);
 +      String hostname = opts.getAddress();
 +      Instance instance = HdfsZooInstance.getInstance();
 +      ServerConfiguration conf = new ServerConfiguration(instance);
 +      Accumulo.init(fs, conf, "master");
 +      Master master = new Master(conf, fs, hostname);
 +      Accumulo.enableTracing(hostname, "master");
 +      master.run();
 +    } catch (Exception ex) {
 +      log.error("Unexpected exception, exiting", ex);
 +      System.exit(1);
 +    }
 +  }
 +
 +  @Override
 +  public void update(LiveTServerSet current, Set<TServerInstance> deleted, Set<TServerInstance> added) {
 +    DeadServerList obit = new DeadServerList(ZooUtil.getRoot(instance) + Constants.ZDEADTSERVERS);
 +    if (added.size() > 0) {
 +      log.info("New servers: " + added);
 +      for (TServerInstance up : added)
 +        obit.delete(up.hostPort());
 +    }
 +    for (TServerInstance dead : deleted) {
 +      String cause = "unexpected failure";
 +      if (serversToShutdown.contains(dead))
 +        cause = "clean shutdown"; // maybe an incorrect assumption
 +      if (!getMasterGoalState().equals(MasterGoalState.CLEAN_STOP))
 +        obit.post(dead.hostPort(), cause);
 +    }
 +
 +    Set<TServerInstance> unexpected = new HashSet<TServerInstance>(deleted);
 +    unexpected.removeAll(this.serversToShutdown);
 +    if (unexpected.size() > 0) {
 +      if (stillMaster() && !getMasterGoalState().equals(MasterGoalState.CLEAN_STOP)) {
 +        log.warn("Lost servers " + unexpected);
 +      }
 +    }
 +    serversToShutdown.removeAll(deleted);
 +    badServers.keySet().removeAll(deleted);
 +    // clear out any bad server with the same host/port as a new server
 +    synchronized (badServers) {
 +      cleanListByHostAndPort(badServers.keySet(), deleted, added);
 +    }
 +    synchronized (serversToShutdown) {
 +      cleanListByHostAndPort(serversToShutdown, deleted, added);
 +    }
 +
 +    synchronized (migrations) {
 +      Iterator<Entry<KeyExtent,TServerInstance>> iter = migrations.entrySet().iterator();
 +      while (iter.hasNext()) {
 +        Entry<KeyExtent,TServerInstance> entry = iter.next();
 +        if (deleted.contains(entry.getValue())) {
 +          log.info("Canceling migration of " + entry.getKey() + " to " + entry.getValue());
 +          iter.remove();
 +        }
 +      }
 +    }
 +    nextEvent.event("There are now %d tablet servers", current.size());
 +  }
 +
 +  private static void cleanListByHostAndPort(Collection<TServerInstance> badServers, Set<TServerInstance> deleted, Set<TServerInstance> added) {
 +    Iterator<TServerInstance> badIter = badServers.iterator();
 +    while (badIter.hasNext()) {
 +      TServerInstance bad = badIter.next();
 +      for (TServerInstance add : added) {
 +        if (bad.hostPort().equals(add.hostPort())) {
 +          badIter.remove();
 +          break;
 +        }
 +      }
 +      for (TServerInstance del : deleted) {
 +        if (bad.hostPort().equals(del.hostPort())) {
 +          badIter.remove();
 +          break;
 +        }
 +      }
 +    }
 +  }
 +
 +  @Override
 +  public void stateChanged(String tableId, TableState state) {
 +    nextEvent.event("Table state in zookeeper changed for %s to %s", tableId, state);
 +  }
 +
 +  @Override
 +  public void initialize(Map<String,TableState> tableIdToStateMap) {}
 +
 +  @Override
 +  public void sessionExpired() {}
 +
 +  @Override
 +  public Set<String> onlineTables() {
 +    Set<String> result = new HashSet<String>();
 +    if (getMasterState() != MasterState.NORMAL) {
 +      if (getMasterState() != MasterState.UNLOAD_METADATA_TABLETS)
 +        result.add(MetadataTable.ID);
 +      if (getMasterState() != MasterState.UNLOAD_ROOT_TABLET)
 +        result.add(RootTable.ID);
 +      return result;
 +    }
 +    TableManager manager = TableManager.getInstance();
 +
 +    for (String tableId : Tables.getIdToNameMap(instance).keySet()) {
 +      TableState state = manager.getTableState(tableId);
 +      if (state != null) {
 +        if (state == TableState.ONLINE)
 +          result.add(tableId);
 +      }
 +    }
 +    return result;
 +  }
 +
 +  @Override
 +  public Set<TServerInstance> onlineTabletServers() {
 +    return tserverSet.getCurrentServers();
 +  }
 +
 +  @Override
 +  public Collection<MergeInfo> merges() {
 +    List<MergeInfo> result = new ArrayList<MergeInfo>();
 +    for (String tableId : Tables.getIdToNameMap(instance).keySet()) {
 +      result.add(getMergeInfo(new Text(tableId)));
 +    }
 +    return result;
 +  }
 +
 +  // recovers state from the persistent transaction to shutdown a server
 +  public void shutdownTServer(TServerInstance server) {
 +    nextEvent.event("Tablet Server shutdown requested for %s", server);
 +    serversToShutdown.add(server);
 +  }
 +
 +  public EventCoordinator getEventCoordinator() {
 +    return nextEvent;
 +  }
 +
 +  public Instance getInstance() {
 +    return this.instance;
 +  }
 +
 +  public AccumuloConfiguration getSystemConfiguration() {
 +    return serverConfig.getConfiguration();
 +  }
 +
 +  public ServerConfiguration getConfiguration() {
 +    return serverConfig;
 +  }
 +
 +  public VolumeManager getFileSystem() {
 +    return this.fs;
 +  }
 +
 +  public void assignedTablet(KeyExtent extent) {
 +    if (extent.isMeta()) {
 +      if (getMasterState().equals(MasterState.UNLOAD_ROOT_TABLET)) {
 +        setMasterState(MasterState.UNLOAD_METADATA_TABLETS);
 +      }
 +    }
 +    if (extent.isRootTablet()) {
 +      // probably too late, but try anyhow
 +      if (getMasterState().equals(MasterState.STOP)) {
 +        setMasterState(MasterState.UNLOAD_ROOT_TABLET);
 +      }
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/957c9d1b/server/master/src/main/java/org/apache/accumulo/master/util/FateAdmin.java
----------------------------------------------------------------------
diff --cc server/master/src/main/java/org/apache/accumulo/master/util/FateAdmin.java
index 8fcb689,0000000..4e72832
mode 100644,000000..100644
--- a/server/master/src/main/java/org/apache/accumulo/master/util/FateAdmin.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/util/FateAdmin.java
@@@ -1,92 -1,0 +1,93 @@@
 +/*
 + * 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.util;
 +
 +import java.util.ArrayList;
 +import java.util.List;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.cli.Help;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.zookeeper.ZooUtil;
 +import org.apache.accumulo.fate.AdminUtil;
 +import org.apache.accumulo.fate.ZooStore;
++import org.apache.accumulo.fate.ReadOnlyStore;
 +import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 +import org.apache.accumulo.master.Master;
 +import org.apache.accumulo.server.client.HdfsZooInstance;
 +import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 +
 +import com.beust.jcommander.JCommander;
 +import com.beust.jcommander.Parameter;
 +import com.beust.jcommander.Parameters;
 +
 +/**
 + * A utility to administer FATE operations
 + */
 +public class FateAdmin {
 +  
 +  static class TxOpts {
 +    @Parameter(description = "<txid>", required = true)
 +    List<String> args = new ArrayList<String>();
 +  }
 +  
 +  @Parameters(commandDescription = "Stop an existing FATE by transaction id")
 +  static class FailOpts extends TxOpts {}
 +  
 +  @Parameters(commandDescription = "Delete an existing FATE by transaction id")
 +  static class DeleteOpts extends TxOpts {}
 +  
 +  @Parameters(commandDescription = "List the existing FATE transactions")
 +  static class PrintOpts {}
 +  
 +  public static void main(String[] args) throws Exception {
 +    Help opts = new Help();
 +    JCommander jc = new JCommander(opts);
 +    jc.setProgramName(FateAdmin.class.getName());
 +    jc.addCommand("fail", new FailOpts());
 +    jc.addCommand("delete", new DeleteOpts());
 +    jc.addCommand("print", new PrintOpts());
 +    jc.parse(args);
 +    if (opts.help || jc.getParsedCommand() == null) {
 +      jc.usage();
 +      System.exit(1);
 +    }
 +    
 +    System.err.printf("This tool has been deprecated%nFATE administration now available within 'accumulo shell'%n$ fate fail <txid>... | delete <txid>... | print [<txid>...]%n%n");
 +    
 +    AdminUtil<Master> admin = new AdminUtil<Master>();
 +    
 +    Instance instance = HdfsZooInstance.getInstance();
 +    String path = ZooUtil.getRoot(instance) + Constants.ZFATE;
 +    String masterPath = ZooUtil.getRoot(instance) + Constants.ZMASTER_LOCK;
 +    IZooReaderWriter zk = ZooReaderWriter.getRetryingInstance();
 +    ZooStore<Master> zs = new ZooStore<Master>(path, zk);
 +    
 +    if (jc.getParsedCommand().equals("fail")) {
 +      if (!admin.prepFail(zs, zk, masterPath, args[1])) {
 +        System.exit(1);
 +      }
 +    } else if (jc.getParsedCommand().equals("delete")) {
 +      if (!admin.prepDelete(zs, zk, masterPath, args[1])) {
 +        System.exit(1);
 +      }
 +      admin.deleteLocks(zs, zk, ZooUtil.getRoot(instance) + Constants.ZTABLE_LOCKS, args[1]);
 +    } else if (jc.getParsedCommand().equals("print")) {
-       admin.print(zs, zk, ZooUtil.getRoot(instance) + Constants.ZTABLE_LOCKS);
++      admin.print(new ReadOnlyStore(zs), zk, ZooUtil.getRoot(instance) + Constants.ZTABLE_LOCKS);
 +    }
 +  }
 +}


[12/15] git commit: ACCUMULO-2519 Updates Classes added in 1.6.0 for read only fate changes.

Posted by bu...@apache.org.
ACCUMULO-2519 Updates Classes added in 1.6.0 for read only fate changes.


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

Branch: refs/heads/master
Commit: e4aa11e1b1a046dec9116273eb57f053aa68fd3f
Parents: 957c9d1
Author: Sean Busbey <bu...@cloudera.com>
Authored: Fri Apr 4 01:35:01 2014 -0700
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Apr 4 17:28:54 2014 -0700

----------------------------------------------------------------------
 .../org/apache/accumulo/core/util/shell/commands/FateCommand.java  | 2 +-
 .../main/java/org/apache/accumulo/master/FateServiceHandler.java   | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/e4aa11e1/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FateCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FateCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FateCommand.java
index 6bf4f30..0196baf 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FateCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FateCommand.java
@@ -31,7 +31,7 @@ import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.AdminUtil;
-import org.apache.accumulo.fate.TStore.TStatus;
+import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
 import org.apache.accumulo.fate.ZooStore;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e4aa11e1/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
index 555e3e4..d63a63e 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
@@ -41,7 +41,7 @@ import org.apache.accumulo.core.master.thrift.FateService;
 import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.util.ArgumentChecker.Validator;
 import org.apache.accumulo.core.util.ByteBufferUtil;
-import org.apache.accumulo.fate.TStore.TStatus;
+import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
 import org.apache.accumulo.master.tableOps.BulkImport;
 import org.apache.accumulo.master.tableOps.CancelCompactions;
 import org.apache.accumulo.master.tableOps.ChangeTableState;


[02/15] git commit: ACCUMULO-2519 Aborts upgrade if there are Fate transactions from an old version.

Posted by bu...@apache.org.
ACCUMULO-2519 Aborts upgrade if there are Fate transactions from an old version.


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

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: 5a504b311c0e5f59ff5b14221c6bf61f43b4d093
Parents: a904f69
Author: Sean Busbey <bu...@cloudera.com>
Authored: Fri Mar 28 01:46:09 2014 -0500
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Apr 4 17:27:05 2014 -0700

----------------------------------------------------------------------
 README                                          |  14 +++
 .../org/apache/accumulo/server/Accumulo.java    |  31 ++++++
 .../apache/accumulo/server/master/Master.java   | 100 ++++++++++++-------
 .../server/tabletserver/TabletServer.java       |   5 +
 .../accumulo/server/util/MetadataTable.java     |   3 +
 5 files changed, 116 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/5a504b31/README
----------------------------------------------------------------------
diff --git a/README b/README
index 115a9b7..0bb1030 100644
--- a/README
+++ b/README
@@ -54,12 +54,26 @@ accumulo.
 
  This happens automatically the first time Accumulo 1.5 is started.  
 
+  * Verify that there are no outstanding FATE operations
+    - Under 1.4 you can list what's in FATE by running
+      $ACCUMULO_HOME/bin/accumulo org.apache.accumulo.server.fate.Admin print
+    - Note that operations in any state will prevent an upgrade. It is safe
+      to delete operations with status SUCCESSFUL. For others, you should restart
+      your 1.4 cluster and allow them to finish.
   * Stop the 1.4 instance.  
   * Configure 1.5 to use the hdfs directory, walog directories, and zookeepers
     that 1.4 was using.
   * Copy other 1.4 configuration options as needed.
   * Start Accumulo 1.5. 
 
+  The upgrade process must make changes to Accumulo's internal state in both ZooKeeper and
+  the table metadata. This process may take some time as Tablet Servers move write-ahead
+  logs to HDFS and then do recovery. During this time, the Monitor will claim that the
+  Master is down and some services may send the Monitor log messages about failure to
+  communicate with each other. These messages are safe to ignore. If you need detail on
+  the upgrade's progress you should view the local logs on the Tablet Servers and active
+  Master.
+
 ******************************************************************************
 4. Configuring
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5a504b31/server/src/main/java/org/apache/accumulo/server/Accumulo.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/Accumulo.java b/server/src/main/java/org/apache/accumulo/server/Accumulo.java
index 99ec7e4..420b6cc 100644
--- a/server/src/main/java/org/apache/accumulo/server/Accumulo.java
+++ b/server/src/main/java/org/apache/accumulo/server/Accumulo.java
@@ -27,11 +27,16 @@ import java.util.Map.Entry;
 import java.util.TreeMap;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.trace.DistributedTrace;
 import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.util.Version;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.ReadOnlyTStore;
+import org.apache.accumulo.fate.ReadOnlyStore;
+import org.apache.accumulo.fate.ZooStore;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.util.time.SimpleTimer;
@@ -53,6 +58,7 @@ public class Accumulo {
     try {
       if (getAccumuloPersistentVersion(fs) == Constants.PREV_DATA_VERSION) {
         fs.create(new Path(ServerConstants.getDataVersionLocation() + "/" + Constants.DATA_VERSION));
+        // TODO document failure mode & recovery if FS permissions cause above to work and below to fail ACCUMULO-2596
         fs.delete(new Path(ServerConstants.getDataVersionLocation() + "/" + Constants.PREV_DATA_VERSION), false);
       }
     } catch (IOException e) {
@@ -263,4 +269,29 @@ public class Accumulo {
       throw new RuntimeException("cannot find method setSafeMode", ex);
     }
   }
+
+  /**
+   * Exit loudly if there are outstanding Fate operations.
+   * Since Fate serializes class names, we need to make sure there are no queued
+   * transactions from a previous version before continuing an upgrade. The status of the operations is
+   * irrelevant; those in SUCCESSFUL status cause the same problem as those just queued.
+   *
+   * Note that the Master should not allow write access to Fate until after all upgrade steps are complete.
+   *
+   * Should be called as a guard before performing any upgrade steps, after determining that an upgrade is needed.
+   *
+   * see ACCUMULO-2519
+   */
+  public static void abortIfFateTransactions() {
+    try {
+      final ReadOnlyTStore<Accumulo> fate = new ReadOnlyStore<Accumulo>(new ZooStore<Accumulo>(ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZFATE,
+          ZooReaderWriter.getRetryingInstance()));
+      if (!(fate.list().isEmpty())) {
+        throw new AccumuloException("Aborting upgrade because there are outstanding FATE transactions from a previous Accumulo version. Please see the README document for instructions on what to do under your previous version.");
+      }
+    } catch (Exception exception) {
+      log.fatal("Problem verifying Fate readiness", exception);
+      System.exit(1);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5a504b31/server/src/main/java/org/apache/accumulo/server/master/Master.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/Master.java b/server/src/main/java/org/apache/accumulo/server/master/Master.java
index 270eb18..a2ad2e6 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/Master.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/Master.java
@@ -34,6 +34,7 @@ import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -271,7 +272,9 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       upgradeMetadata();
     }
   }
-  
+
+  private boolean haveUpgradedZooKeeper = false;
+
   private void upgradeZookeeper() {
     // 1.5.1 and 1.6.0 both do some state checking after obtaining the zoolock for the
     // monitor and before starting up. It's not tied to the data version at all (and would
@@ -279,59 +282,79 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
     // that the master is not the only thing that may alter zookeeper before starting.
 
     if (Accumulo.getAccumuloPersistentVersion(fs) == Constants.PREV_DATA_VERSION) {
+      // This Master hasn't started Fate yet, so any outstanding transactions must be from before the upgrade.
+      // Change to Guava's Verify once we use Guava 17.
+      if (null != fate) {
+        throw new IllegalStateException("Access to Fate should not have been initialized prior to the Master transitioning to active. Please save all logs and file a bug.");
+      }
+      Accumulo.abortIfFateTransactions();
       try {
         log.info("Upgrading zookeeper");
-        
+
         IZooReaderWriter zoo = ZooReaderWriter.getInstance();
-        
+
         zoo.recursiveDelete(ZooUtil.getRoot(instance) + "/loggers", NodeMissingPolicy.SKIP);
         zoo.recursiveDelete(ZooUtil.getRoot(instance) + "/dead/loggers", NodeMissingPolicy.SKIP);
 
         zoo.putPersistentData(ZooUtil.getRoot(instance) + Constants.ZRECOVERY, new byte[] {'0'}, NodeExistsPolicy.SKIP);
-        
+
         for (String id : Tables.getIdToNameMap(instance).keySet()) {
-          
+
           zoo.putPersistentData(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + id + Constants.ZTABLE_COMPACT_CANCEL_ID, "0".getBytes(Constants.UTF8),
               NodeExistsPolicy.SKIP);
         }
+        haveUpgradedZooKeeper = true;
       } catch (Exception ex) {
         log.fatal("Error performing upgrade", ex);
         System.exit(1);
       }
     }
   }
-  
+
   private final AtomicBoolean upgradeMetadataRunning = new AtomicBoolean(false);
-  
+  private final CountDownLatch waitForMetadataUpgrade = new CountDownLatch(1);
+
   private final ServerConfiguration serverConfig;
   
   private void upgradeMetadata() {
-    if (Accumulo.getAccumuloPersistentVersion(fs) == Constants.PREV_DATA_VERSION) {
-      if (upgradeMetadataRunning.compareAndSet(false, true)) {
+    // we make sure we're only doing the rest of this method once so that we can signal to other threads that an upgrade wasn't needed.
+    if (upgradeMetadataRunning.compareAndSet(false, true)) {
+      if (Accumulo.getAccumuloPersistentVersion(fs) == Constants.PREV_DATA_VERSION) {
+        // sanity check that we passed the Fate verification prior to ZooKeeper upgrade, and that Fate still hasn't been started.
+        // Change both to use Guava's Verify once we use Guava 17.
+        if (!haveUpgradedZooKeeper) {
+          throw new IllegalStateException("We should only attempt to upgrade Accumulo's !METADATA table if we've already upgraded ZooKeeper. Please save all logs and file a bug.");
+        }
+        if (null != fate) {
+          throw new IllegalStateException("Access to Fate should not have been initialized prior to the Master finishing upgrades. Please save all logs and file a bug.");
+        }
         Runnable upgradeTask = new Runnable() {
           @Override
           public void run() {
             try {
+              log.info("Starting to upgrade !METADATA table.");
               MetadataTable.moveMetaDeleteMarkers(instance, SecurityConstants.getSystemCredentials());
+              log.info("Updating persistent data version.");
               Accumulo.updateAccumuloVersion(fs);
-              
               log.info("Upgrade complete");
-              
+              waitForMetadataUpgrade.countDown();
             } catch (Exception ex) {
               log.fatal("Error performing upgrade", ex);
               System.exit(1);
             }
-            
+
           }
         };
-        
+
         // need to run this in a separate thread because a lock is held that prevents !METADATA tablets from being assigned and this task writes to the
         // !METADATA table
         new Thread(upgradeTask).start();
+      } else {
+        waitForMetadataUpgrade.countDown();
       }
     }
   }
-  
+
   private int assignedOrHosted(Text tableId) {
     int result = 0;
     for (TabletGroupWatcher watcher : watchers) {
@@ -2136,28 +2159,6 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
     
     tserverSet.startListeningForTabletServerChanges();
     
-    // TODO: add shutdown for fate object - ACCUMULO-1307
-    try {
-      final AgeOffStore<Master> store = new AgeOffStore<Master>(new org.apache.accumulo.fate.ZooStore<Master>(ZooUtil.getRoot(instance) + Constants.ZFATE,
-          ZooReaderWriter.getRetryingInstance()), 1000 * 60 * 60 * 8);
-      
-      int threads = this.getConfiguration().getConfiguration().getCount(Property.MASTER_FATE_THREADPOOL_SIZE);
-      
-      fate = new Fate<Master>(this, store, threads);
-      
-      SimpleTimer.getInstance().schedule(new Runnable() {
-        
-        @Override
-        public void run() {
-          store.ageOff();
-        }
-      }, 63000, 63000);
-    } catch (KeeperException e) {
-      throw new IOException(e);
-    } catch (InterruptedException e) {
-      throw new IOException(e);
-    }
-    
     ZooReaderWriter.getInstance().getChildren(zroot + Constants.ZRECOVERY, new Watcher() {
       @Override
       public void process(WatchedEvent event) {
@@ -2183,7 +2184,32 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
     for (TabletGroupWatcher watcher : watchers) {
       watcher.start();
     }
-    
+
+    // Once we are sure tablet servers are no longer checking for an empty Fate transaction queue before doing WAL upgrades, we can safely start using Fate ourselves.
+    waitForMetadataUpgrade.await();
+
+    // TODO: add shutdown for fate object - ACCUMULO-1307
+    try {
+      final AgeOffStore<Master> store = new AgeOffStore<Master>(new org.apache.accumulo.fate.ZooStore<Master>(ZooUtil.getRoot(instance) + Constants.ZFATE,
+          ZooReaderWriter.getRetryingInstance()), 1000 * 60 * 60 * 8);
+
+      int threads = this.getConfiguration().getConfiguration().getCount(Property.MASTER_FATE_THREADPOOL_SIZE);
+
+      fate = new Fate<Master>(this, store, threads);
+
+      SimpleTimer.getInstance().schedule(new Runnable() {
+
+        @Override
+        public void run() {
+          store.ageOff();
+        }
+      }, 63000, 63000);
+    } catch (KeeperException e) {
+      throw new IOException(e);
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
+
     Processor<Iface> processor = new Processor<Iface>(TraceWrap.service(new MasterClientServiceHandler()));
     ServerPort serverPort = TServerUtils.startServer(getSystemConfiguration(), Property.MASTER_CLIENTPORT, processor, "Master",
         "Master Client Service Handler", null, Property.MASTER_MINTHREADS, Property.MASTER_THREADCHECK, Property.GENERAL_MAX_MESSAGE_SIZE);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5a504b31/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java b/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
index d76946d..ad3d615 100644
--- a/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
+++ b/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
@@ -3322,6 +3322,11 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
    * 
    */
   public static void recoverLocalWriteAheadLogs(FileSystem fs, ServerConfiguration serverConf) throws IOException {
+    if (Accumulo.getAccumuloPersistentVersion(fs) == Constants.PREV_DATA_VERSION) {
+      // If the Master has not yet signaled a finish to upgrading, we need to make sure we can rollback in the
+      // event of outstanding transactions in Fate from the previous version.
+      Accumulo.abortIfFateTransactions();
+    }
     FileSystem localfs = FileSystem.getLocal(fs.getConf()).getRawFileSystem();
     AccumuloConfiguration conf = serverConf.getConfiguration();
     String localWalDirectories = conf.get(Property.LOGGER_DIR);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5a504b31/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java b/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
index 7328a55..d6e0a3c 100644
--- a/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
+++ b/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
@@ -1233,6 +1233,9 @@ public class MetadataTable extends org.apache.accumulo.core.util.MetadataTable {
     update(SecurityConstants.getSystemCredentials(), m);
   }
 
+  /**
+   * During an upgrade from Accumulo 1.4 -> 1.5, we need to move deletion requests for files under the !METADATA table to the root tablet.
+   */
   public static void moveMetaDeleteMarkers(Instance instance, TCredentials creds) {
     // move delete markers from the normal delete keyspace to the root tablet delete keyspace if the files are for the !METADATA table
     Scanner scanner = new ScannerImpl(instance, creds, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);


[09/15] Merge branch '1.5.2-SNAPSHOT' into 1.6.0-SNAPSHOT

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/957c9d1b/server/master/src/main/java/org/apache/accumulo/master/Master.java
----------------------------------------------------------------------
diff --cc server/master/src/main/java/org/apache/accumulo/master/Master.java
index e123b49,0000000..2440ee4
mode 100644,000000..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
@@@ -1,1227 -1,0 +1,1252 @@@
 +/*
 + * 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.io.IOException;
 +import java.util.ArrayList;
 +import java.util.Collection;
 +import java.util.Collections;
 +import java.util.HashSet;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +import java.util.SortedMap;
 +import java.util.TreeMap;
++import java.util.concurrent.CountDownLatch;
 +import java.util.concurrent.atomic.AtomicBoolean;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +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.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.TableOperation;
 +import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
 +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.master.state.tables.TableState;
 +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.MasterState;
 +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 +import org.apache.accumulo.core.metadata.MetadataTable;
 +import org.apache.accumulo.core.metadata.RootTable;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 +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.util.Daemon;
 +import org.apache.accumulo.core.util.Pair;
 +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.zookeeper.IZooReaderWriter;
 +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.server.Accumulo;
 +import org.apache.accumulo.server.ServerConstants;
 +import org.apache.accumulo.server.ServerOpts;
 +import org.apache.accumulo.server.client.HdfsZooInstance;
 +import org.apache.accumulo.server.conf.ServerConfiguration;
 +import org.apache.accumulo.server.fs.VolumeManager;
 +import org.apache.accumulo.server.fs.VolumeManager.FileType;
 +import org.apache.accumulo.server.fs.VolumeManagerImpl;
 +import org.apache.accumulo.server.init.Initialize;
 +import org.apache.accumulo.server.master.LiveTServerSet;
 +import org.apache.accumulo.server.master.LiveTServerSet.TServerConnection;
 +import org.apache.accumulo.server.master.balancer.DefaultLoadBalancer;
 +import org.apache.accumulo.server.master.balancer.TabletBalancer;
 +import org.apache.accumulo.server.master.state.CurrentState;
 +import org.apache.accumulo.server.master.state.DeadServerList;
 +import org.apache.accumulo.server.master.state.MergeInfo;
 +import org.apache.accumulo.server.master.state.MergeState;
 +import org.apache.accumulo.server.master.state.MetaDataStateStore;
 +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.TabletState;
 +import org.apache.accumulo.server.master.state.ZooStore;
 +import org.apache.accumulo.server.master.state.ZooTabletStateStore;
 +import org.apache.accumulo.server.security.AuditedSecurityOperation;
 +import org.apache.accumulo.server.security.SecurityOperation;
 +import org.apache.accumulo.server.security.SystemCredentials;
 +import org.apache.accumulo.server.security.handler.ZKPermHandler;
 +import org.apache.accumulo.server.tables.TableManager;
 +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.TServerUtils;
 +import org.apache.accumulo.server.util.TServerUtils.ServerAddress;
 +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.hadoop.fs.Path;
 +import org.apache.hadoop.io.DataInputBuffer;
 +import org.apache.hadoop.io.DataOutputBuffer;
 +import org.apache.hadoop.io.Text;
 +import org.apache.log4j.Logger;
 +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.WatchedEvent;
 +import org.apache.zookeeper.Watcher;
 +import org.apache.zookeeper.data.Stat;
 +
 +import com.google.common.collect.Iterables;
 +
 +/**
 + * The Master is responsible for assigning and balancing tablets to tablet servers.
 + * 
 + * The master will also coordinate log recoveries and reports general status.
 + */
 +public class Master implements LiveTServerSet.Listener, TableObserver, CurrentState {
 +
 +  final static Logger log = Logger.getLogger(Master.class);
 +
 +  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;
 +  final private static long TIME_BETWEEN_MIGRATION_CLEANUPS = 5 * 60 * ONE_SECOND;
 +  final static long WAIT_BETWEEN_ERRORS = ONE_SECOND;
 +  final private static long DEFAULT_WAIT_FOR_WATCHER = 10 * ONE_SECOND;
 +  final private static int MAX_CLEANUP_WAIT_TIME = ONE_SECOND;
 +  final private static int TIME_TO_WAIT_BETWEEN_LOCK_CHECKS = ONE_SECOND;
 +  final static int MAX_TSERVER_WORK_CHUNK = 5000;
 +  final private static int MAX_BAD_STATUS_COUNT = 3;
 +
 +  final VolumeManager fs;
 +  final private Instance instance;
 +  final private String hostname;
 +  final LiveTServerSet tserverSet;
 +  final private List<TabletGroupWatcher> watchers = new ArrayList<TabletGroupWatcher>();
 +  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();
 +  final private Object mergeLock = new Object();
 +  RecoveryManager recoveryManager = null;
 +
 +  ZooLock masterLock = null;
 +  private TServer clientService = null;
 +  TabletBalancer tabletBalancer;
 +
 +  private MasterState state = MasterState.INITIAL;
 +
 +  Fate<Master> fate;
 +
 +  volatile SortedMap<TServerInstance,TabletServerStatus> tserverStatus = Collections.unmodifiableSortedMap(new TreeMap<TServerInstance,TabletServerStatus>());
 +
 +  synchronized MasterState getMasterState() {
 +    return state;
 +  }
 +
 +  public boolean stillMaster() {
 +    return getMasterState() != MasterState.STOP;
 +  }
 +
 +  static final boolean X = true;
 +  static final boolean _ = false;
 +  // @formatter:off
 +  static final boolean transitionOK[][] = {
 +      //                              INITIAL HAVE_LOCK SAFE_MODE NORMAL UNLOAD_META UNLOAD_ROOT STOP
 +      /* INITIAL */                   {X,     X,        _,        _,      _,         _,          X},
 +      /* HAVE_LOCK */                 {_,     X,        X,        X,      _,         _,          X},
 +      /* SAFE_MODE */                 {_,     _,        X,        X,      X,         _,          X},
 +      /* NORMAL */                    {_,     _,        X,        X,      X,         _,          X},
 +      /* UNLOAD_METADATA_TABLETS */   {_,     _,        X,        X,      X,         X,          X},
 +      /* UNLOAD_ROOT_TABLET */        {_,     _,        _,        X,      X,         X,          X},
 +      /* STOP */                      {_,     _,        _,        _,      _,         X,          X}};
 +  //@formatter:on
 +  synchronized void setMasterState(MasterState newState) {
 +    if (state.equals(newState))
 +      return;
 +    if (!transitionOK[state.ordinal()][newState.ordinal()]) {
 +      log.error("Programmer error: master should not transition from " + state + " to " + newState);
 +    }
 +    MasterState oldState = state;
 +    state = newState;
 +    nextEvent.event("State changed from %s to %s", oldState, newState);
 +    if (newState == MasterState.STOP) {
 +      // Give the server a little time before shutdown so the client
 +      // thread requesting the stop can return
 +      SimpleTimer.getInstance().schedule(new Runnable() {
 +        @Override
 +        public void run() {
 +          // This frees the main thread and will cause the master to exit
 +          clientService.stop();
 +          Master.this.nextEvent.event("stopped event loop");
 +        }
 +
 +      }, 100l, 1000l);
 +    }
 +
 +    if (oldState != newState && (newState == MasterState.HAVE_LOCK)) {
 +      upgradeZookeeper();
 +    }
 +
 +    if (oldState != newState && (newState == MasterState.NORMAL)) {
 +      upgradeMetadata();
 +    }
 +  }
 +
 +  private void moveRootTabletToRootTable(IZooReaderWriter zoo) throws Exception {
 +    String dirZPath = ZooUtil.getRoot(instance) + RootTable.ZROOT_TABLET_PATH;
 +
 +    if (!zoo.exists(dirZPath)) {
 +      Path oldPath = fs.getFullPath(FileType.TABLE, "/" + MetadataTable.ID + "/root_tablet");
 +      if (fs.exists(oldPath)) {
 +        String newPath = fs.choose(ServerConstants.getTablesDirs()) + "/" + RootTable.ID;
 +        fs.mkdirs(new Path(newPath));
 +        if (!fs.rename(oldPath, new Path(newPath))) {
 +          throw new IOException("Failed to move root tablet from " + oldPath + " to " + newPath);
 +        }
 +
 +        log.info("Upgrade renamed " + oldPath + " to " + newPath);
 +      }
 +
 +      Path location = null;
 +
 +      for (String basePath : ServerConstants.getTablesDirs()) {
 +        Path path = new Path(basePath + "/" + RootTable.ID + RootTable.ROOT_TABLET_LOCATION);
 +        if (fs.exists(path)) {
 +          if (location != null) {
 +            throw new IllegalStateException("Root table at multiple locations " + location + " " + path);
 +          }
 +
 +          location = path;
 +        }
 +      }
 +
 +      if (location == null)
 +        throw new IllegalStateException("Failed to find root tablet");
 +
 +      log.info("Upgrade setting root table location in zookeeper " + location);
 +      zoo.putPersistentData(dirZPath, location.toString().getBytes(), NodeExistsPolicy.FAIL);
 +    }
 +  }
 +
++  private boolean haveUpgradedZooKeeper = false;
++
 +  private void upgradeZookeeper() {
 +    // 1.5.1 and 1.6.0 both do some state checking after obtaining the zoolock for the
 +    // monitor and before starting up. It's not tied to the data version at all (and would
 +    // introduce unnecessary complexity to try to make the master do it), but be aware
 +    // that the master is not the only thing that may alter zookeeper before starting.
 +
 +    if (Accumulo.getAccumuloPersistentVersion(fs) == ServerConstants.PREV_DATA_VERSION) {
++      // This Master hasn't started Fate yet, so any outstanding transactions must be from before the upgrade.
++      // Change to Guava's Verify once we use Guava 17.
++      if (null != fate) {
++        throw new IllegalStateException("Access to Fate should not have been initialized prior to the Master transitioning to active. Please save all logs and file a bug.");
++      }
++      Accumulo.abortIfFateTransactions();
 +      try {
 +        log.info("Upgrading zookeeper");
 +
 +        IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +
 +        // create initial namespaces
 +        String namespaces = ZooUtil.getRoot(instance) + Constants.ZNAMESPACES;
 +        zoo.putPersistentData(namespaces, new byte[0], NodeExistsPolicy.SKIP);
 +        for (Pair<String,String> namespace : Iterables.concat(
 +            Collections.singleton(new Pair<String,String>(Namespaces.ACCUMULO_NAMESPACE, Namespaces.ACCUMULO_NAMESPACE_ID)),
 +            Collections.singleton(new Pair<String,String>(Namespaces.DEFAULT_NAMESPACE, Namespaces.DEFAULT_NAMESPACE_ID)))) {
 +          String ns = namespace.getFirst();
 +          String id = namespace.getSecond();
 +          log.debug("Upgrade creating namespace \"" + ns + "\" (ID: " + id + ")");
 +          if (!Namespaces.exists(instance, id))
 +            TableManager.prepareNewNamespaceState(instance.getInstanceID(), id, ns, NodeExistsPolicy.SKIP);
 +        }
 +
 +        // create root table
 +        log.debug("Upgrade creating table " + RootTable.NAME + " (ID: " + RootTable.ID + ")");
 +        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
 +        security.grantTablePermission(SystemCredentials.get().toThrift(instance), security.getRootUsername(), RootTable.ID, TablePermission.ALTER_TABLE, Namespaces.ACCUMULO_NAMESPACE_ID);
 +
 +        // put existing tables in the correct namespaces
 +        String tables = ZooUtil.getRoot(instance) + Constants.ZTABLES;
 +        for (String tableId : zoo.getChildren(tables)) {
 +          String targetNamespace = (MetadataTable.ID.equals(tableId) || RootTable.ID.equals(tableId)) ? Namespaces.ACCUMULO_NAMESPACE_ID
 +              : Namespaces.DEFAULT_NAMESPACE_ID;
 +          log.debug("Upgrade moving table " + new String(zoo.getData(tables + "/" + tableId + Constants.ZTABLE_NAME, null), Constants.UTF8) + " (ID: "
 +              + tableId + ") into namespace with ID " + targetNamespace);
 +          zoo.putPersistentData(tables + "/" + tableId + Constants.ZTABLE_NAMESPACE, targetNamespace.getBytes(Constants.UTF8), NodeExistsPolicy.SKIP);
 +        }
 +
 +        // rename metadata table
 +        log.debug("Upgrade renaming table " + MetadataTable.OLD_NAME + " (ID: " + MetadataTable.ID + ") to " + MetadataTable.NAME);
 +        zoo.putPersistentData(tables + "/" + MetadataTable.ID + Constants.ZTABLE_NAME, Tables.qualify(MetadataTable.NAME).getSecond().getBytes(Constants.UTF8),
 +            NodeExistsPolicy.OVERWRITE);
 +
 +        moveRootTabletToRootTable(zoo);
 +
 +        // add system namespace permissions to existing users
 +        ZKPermHandler perm = new ZKPermHandler();
 +        perm.initialize(instance.getInstanceID(), true);
 +        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, Namespaces.ACCUMULO_NAMESPACE_ID, NamespacePermission.READ);
 +        }
 +        perm.grantNamespacePermission("root", Namespaces.ACCUMULO_NAMESPACE_ID, NamespacePermission.ALTER_TABLE);
- 
++        haveUpgradedZooKeeper = true;
 +      } catch (Exception ex) {
 +        log.fatal("Error performing upgrade", ex);
 +        System.exit(1);
 +      }
 +    }
 +  }
 +
 +  private final AtomicBoolean upgradeMetadataRunning = new AtomicBoolean(false);
++  private final CountDownLatch waitForMetadataUpgrade = new CountDownLatch(1);
 +
 +  private final ServerConfiguration serverConfig;
 +
 +  private void upgradeMetadata() {
-     if (Accumulo.getAccumuloPersistentVersion(fs) == ServerConstants.PREV_DATA_VERSION) {
-       if (upgradeMetadataRunning.compareAndSet(false, true)) {
++    // we make sure we're only doing the rest of this method once so that we can signal to other threads that an upgrade wasn't needed.
++    if (upgradeMetadataRunning.compareAndSet(false, true)) {
++      if (Accumulo.getAccumuloPersistentVersion(fs) == ServerConstants.PREV_DATA_VERSION) {
++        // sanity check that we passed the Fate verification prior to ZooKeeper upgrade, and that Fate still hasn't been started.
++        // Change both to use Guava's Verify once we use Guava 17.
++        if (!haveUpgradedZooKeeper) {
++          throw new IllegalStateException("We should only attempt to upgrade Accumulo's !METADATA table if we've already upgraded ZooKeeper. Please save all logs and file a bug.");
++        }
++        if (null != fate) {
++          throw new IllegalStateException("Access to Fate should not have been initialized prior to the Master finishing upgrades. Please save all logs and file a bug.");
++        }
 +        Runnable upgradeTask = new Runnable() {
 +          @Override
 +          public void run() {
 +            try {
++              log.info("Starting to upgrade !METADATA table.");
 +              MetadataTableUtil.moveMetaDeleteMarkers(instance, SystemCredentials.get());
++              log.info("Updating persistent data version.");
 +              Accumulo.updateAccumuloVersion(fs);
- 
 +              log.info("Upgrade complete");
- 
++              waitForMetadataUpgrade.countDown();
 +            } catch (Exception ex) {
 +              log.fatal("Error performing upgrade", ex);
 +              System.exit(1);
 +            }
 +
 +          }
 +        };
 +
 +        // need to run this in a separate thread because a lock is held that prevents metadata tablets from being assigned and this task writes to the
 +        // metadata table
 +        new Thread(upgradeTask).start();
++      } else {
++        waitForMetadataUpgrade.countDown();
 +      }
 +    }
 +  }
 +
 +  private int assignedOrHosted(Text tableId) {
 +    int result = 0;
 +    for (TabletGroupWatcher watcher : watchers) {
 +      TableCounts count = watcher.getStats(tableId);
 +      result += count.hosted() + count.assigned();
 +    }
 +    return result;
 +  }
 +
 +  private int totalAssignedOrHosted() {
 +    int result = 0;
 +    for (TabletGroupWatcher watcher : watchers) {
 +      for (TableCounts counts : watcher.getStats().values()) {
 +        result += counts.assigned() + counts.hosted();
 +      }
 +    }
 +    return result;
 +  }
 +
 +  private int nonMetaDataTabletsAssignedOrHosted() {
 +    return totalAssignedOrHosted() - assignedOrHosted(new Text(MetadataTable.ID)) - assignedOrHosted(new Text(RootTable.ID));
 +  }
 +
 +  private int notHosted() {
 +    int result = 0;
 +    for (TabletGroupWatcher watcher : watchers) {
 +      for (TableCounts counts : watcher.getStats().values()) {
 +        result += counts.assigned() + counts.assignedToDeadServers();
 +      }
 +    }
 +    return result;
 +  }
 +
 +  // The number of unassigned tablets that should be assigned: displayed on the monitor page
 +  int displayUnassigned() {
 +    int result = 0;
 +    switch (getMasterState()) {
 +      case NORMAL:
 +        // Count offline tablets for online tables
 +        for (TabletGroupWatcher watcher : watchers) {
 +          TableManager manager = TableManager.getInstance();
 +          for (Entry<Text,TableCounts> entry : watcher.getStats().entrySet()) {
 +            Text tableId = entry.getKey();
 +            TableCounts counts = entry.getValue();
 +            TableState tableState = manager.getTableState(tableId.toString());
 +            if (tableState != null && tableState.equals(TableState.ONLINE)) {
 +              result += counts.unassigned() + counts.assignedToDeadServers() + counts.assigned();
 +            }
 +          }
 +        }
 +        break;
 +      case SAFE_MODE:
 +        // Count offline tablets for the metadata table
 +        for (TabletGroupWatcher watcher : watchers) {
 +          result += watcher.getStats(METADATA_TABLE_ID).unassigned();
 +        }
 +        break;
 +      case UNLOAD_METADATA_TABLETS:
 +      case UNLOAD_ROOT_TABLET:
 +        for (TabletGroupWatcher watcher : watchers) {
 +          result += watcher.getStats(METADATA_TABLE_ID).unassigned();
 +        }
 +        break;
 +      default:
 +        break;
 +    }
 +    return result;
 +  }
 +
 +  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");
 +  }
 +
 +  public Connector getConnector() throws AccumuloException, AccumuloSecurityException {
 +    return instance.getConnector(SystemCredentials.get().getPrincipal(), SystemCredentials.get().getToken());
 +  }
 +
 +  private Master(ServerConfiguration config, VolumeManager fs, String hostname) throws IOException {
 +    this.serverConfig = config;
 +    this.instance = config.getInstance();
 +    this.fs = fs;
 +    this.hostname = hostname;
 +
 +    AccumuloConfiguration aconf = serverConfig.getConfiguration();
 +
 +    log.info("Version " + Constants.VERSION);
 +    log.info("Instance " + instance.getInstanceID());
 +    ThriftTransportPool.getInstance().setIdleTime(aconf.getTimeInMillis(Property.GENERAL_RPC_TIMEOUT));
 +    security = AuditedSecurityOperation.getInstance();
 +    tserverSet = new LiveTServerSet(instance, config.getConfiguration(), this);
 +    this.tabletBalancer = aconf.instantiateClassProperty(Property.MASTER_TABLET_BALANCER, TabletBalancer.class, new DefaultLoadBalancer());
 +    this.tabletBalancer.init(serverConfig);
 +  }
 +
 +  public TServerConnection getConnection(TServerInstance server) {
 +    return tserverSet.getConnection(server);
 +  }
 +
 +  public MergeInfo getMergeInfo(Text tableId) {
 +    synchronized (mergeLock) {
 +      try {
 +        String path = ZooUtil.getRoot(instance.getInstanceID()) + Constants.ZTABLES + "/" + tableId.toString() + "/merge";
 +        if (!ZooReaderWriter.getInstance().exists(path))
 +          return new MergeInfo();
 +        byte[] data = ZooReaderWriter.getInstance().getData(path, new Stat());
 +        DataInputBuffer in = new DataInputBuffer();
 +        in.reset(data, data.length);
 +        MergeInfo info = new MergeInfo();
 +        info.readFields(in);
 +        return info;
 +      } catch (KeeperException.NoNodeException ex) {
 +        log.info("Error reading merge state, it probably just finished");
 +        return new MergeInfo();
 +      } catch (Exception ex) {
 +        log.warn("Unexpected error reading merge state", ex);
 +        return new MergeInfo();
 +      }
 +    }
 +  }
 +
 +  public void setMergeState(MergeInfo info, MergeState state) throws IOException, KeeperException, InterruptedException {
 +    synchronized (mergeLock) {
 +      String path = ZooUtil.getRoot(instance.getInstanceID()) + Constants.ZTABLES + "/" + info.getExtent().getTableId().toString() + "/merge";
 +      info.setState(state);
 +      if (state.equals(MergeState.NONE)) {
 +        ZooReaderWriter.getInstance().recursiveDelete(path, NodeMissingPolicy.SKIP);
 +      } else {
 +        DataOutputBuffer out = new DataOutputBuffer();
 +        try {
 +          info.write(out);
 +        } catch (IOException ex) {
 +          throw new RuntimeException("Unlikely", ex);
 +        }
 +        ZooReaderWriter.getInstance().putPersistentData(path, out.getData(),
 +            state.equals(MergeState.STARTED) ? ZooUtil.NodeExistsPolicy.FAIL : ZooUtil.NodeExistsPolicy.OVERWRITE);
 +      }
 +      mergeLock.notifyAll();
 +    }
 +    nextEvent.event("Merge state of %s set to %s", info.getExtent(), state);
 +  }
 +
 +  public void clearMergeState(Text tableId) throws IOException, KeeperException, InterruptedException {
 +    synchronized (mergeLock) {
 +      String path = ZooUtil.getRoot(instance.getInstanceID()) + Constants.ZTABLES + "/" + tableId.toString() + "/merge";
 +      ZooReaderWriter.getInstance().recursiveDelete(path, NodeMissingPolicy.SKIP);
 +      mergeLock.notifyAll();
 +    }
 +    nextEvent.event("Merge state of %s cleared", tableId);
 +  }
 +
 +  void setMasterGoalState(MasterGoalState state) {
 +    try {
 +      ZooReaderWriter.getInstance().putPersistentData(ZooUtil.getRoot(instance) + Constants.ZMASTER_GOAL_STATE, state.name().getBytes(),
 +          NodeExistsPolicy.OVERWRITE);
 +    } catch (Exception ex) {
 +      log.error("Unable to set master goal state in zookeeper");
 +    }
 +  }
 +
 +  MasterGoalState getMasterGoalState() {
 +    while (true)
 +      try {
 +        byte[] data = ZooReaderWriter.getInstance().getData(ZooUtil.getRoot(instance) + Constants.ZMASTER_GOAL_STATE, null);
 +        return MasterGoalState.valueOf(new String(data));
 +      } catch (Exception e) {
 +        log.error("Problem getting real goal state: " + e);
 +        UtilWaitThread.sleep(1000);
 +      }
 +  }
 +
 +  public boolean hasCycled(long time) {
 +    for (TabletGroupWatcher watcher : watchers) {
 +      if (watcher.stats.lastScanFinished() < time)
 +        return false;
 +    }
 +
 +    return true;
 +  }
 +
 +  public void clearMigrations(String tableId) {
 +    synchronized (migrations) {
 +      Iterator<KeyExtent> iterator = migrations.keySet().iterator();
 +      while (iterator.hasNext()) {
 +        KeyExtent extent = iterator.next();
 +        if (extent.getTableId().toString().equals(tableId)) {
 +          iterator.remove();
 +        }
 +      }
 +    }
 +  }
 +
 +  static enum TabletGoalState {
 +    HOSTED, UNASSIGNED, DELETED
 +  };
 +
 +  TabletGoalState getSystemGoalState(TabletLocationState tls) {
 +    switch (getMasterState()) {
 +      case NORMAL:
 +        return TabletGoalState.HOSTED;
 +      case HAVE_LOCK: // fall-through intended
 +      case INITIAL: // fall-through intended
 +      case SAFE_MODE:
 +        if (tls.extent.isMeta())
 +          return TabletGoalState.HOSTED;
 +        return TabletGoalState.UNASSIGNED;
 +      case UNLOAD_METADATA_TABLETS:
 +        if (tls.extent.isRootTablet())
 +          return TabletGoalState.HOSTED;
 +        return TabletGoalState.UNASSIGNED;
 +      case UNLOAD_ROOT_TABLET:
 +        return TabletGoalState.UNASSIGNED;
 +      case STOP:
 +        return TabletGoalState.UNASSIGNED;
 +      default:
 +        throw new IllegalStateException("Unknown Master State");
 +    }
 +  }
 +
 +  TabletGoalState getTableGoalState(KeyExtent extent) {
 +    TableState tableState = TableManager.getInstance().getTableState(extent.getTableId().toString());
 +    if (tableState == null)
 +      return TabletGoalState.DELETED;
 +    switch (tableState) {
 +      case DELETING:
 +        return TabletGoalState.DELETED;
 +      case OFFLINE:
 +      case NEW:
 +        return TabletGoalState.UNASSIGNED;
 +      default:
 +        return TabletGoalState.HOSTED;
 +    }
 +  }
 +
 +  TabletGoalState getGoalState(TabletLocationState tls, MergeInfo mergeInfo) {
 +    KeyExtent extent = tls.extent;
 +    // Shutting down?
 +    TabletGoalState state = getSystemGoalState(tls);
 +    if (state == TabletGoalState.HOSTED) {
 +      if (tls.current != null && serversToShutdown.contains(tls.current)) {
 +        return TabletGoalState.UNASSIGNED;
 +      }
 +      // Handle merge transitions
 +      if (mergeInfo.getExtent() != null) {
 +        log.debug("mergeInfo overlaps: " + extent + " " + mergeInfo.overlaps(extent));
 +        if (mergeInfo.overlaps(extent)) {
 +          switch (mergeInfo.getState()) {
 +            case NONE:
 +            case COMPLETE:
 +              break;
 +            case STARTED:
 +            case SPLITTING:
 +              return TabletGoalState.HOSTED;
 +            case WAITING_FOR_CHOPPED:
 +              if (tls.getState(onlineTabletServers()).equals(TabletState.HOSTED)) {
 +                if (tls.chopped)
 +                  return TabletGoalState.UNASSIGNED;
 +              } else {
 +                if (tls.chopped && tls.walogs.isEmpty())
 +                  return TabletGoalState.UNASSIGNED;
 +              }
 +
 +              return TabletGoalState.HOSTED;
 +            case WAITING_FOR_OFFLINE:
 +            case MERGING:
 +              return TabletGoalState.UNASSIGNED;
 +          }
 +        }
 +      }
 +
 +      // taking table offline?
 +      state = getTableGoalState(extent);
 +      if (state == TabletGoalState.HOSTED) {
 +        // Maybe this tablet needs to be migrated
 +        TServerInstance dest = migrations.get(extent);
 +        if (dest != null && tls.current != null && !dest.equals(tls.current)) {
 +          return TabletGoalState.UNASSIGNED;
 +        }
 +      }
 +    }
 +    return state;
 +  }
 +
 +  private class MigrationCleanupThread extends Daemon {
 +
 +    @Override
 +    public void run() {
 +      setName("Migration Cleanup Thread");
 +      while (stillMaster()) {
 +        if (!migrations.isEmpty()) {
 +          try {
 +            cleanupMutations();
 +          } catch (Exception ex) {
 +            log.error("Error cleaning up migrations", ex);
 +          }
 +        }
 +        UtilWaitThread.sleep(TIME_BETWEEN_MIGRATION_CLEANUPS);
 +      }
 +    }
 +
 +    // If a migrating tablet splits, and the tablet dies before sending the
 +    // master a message, the migration will refer to a non-existing tablet,
 +    // so it can never complete. Periodically scan the metadata table and
 +    // remove any migrating tablets that no longer exist.
 +    private void cleanupMutations() throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
 +      Connector connector = getConnector();
 +      Scanner scanner = connector.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
 +      TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
 +      Set<KeyExtent> found = new HashSet<KeyExtent>();
 +      for (Entry<Key,Value> entry : scanner) {
 +        KeyExtent extent = new KeyExtent(entry.getKey().getRow(), entry.getValue());
 +        if (migrations.containsKey(extent)) {
 +          found.add(extent);
 +        }
 +      }
 +      migrations.keySet().retainAll(found);
 +    }
 +  }
 +
 +  private class StatusThread extends Daemon {
 +
 +    @Override
 +    public void run() {
 +      setName("Status Thread");
 +      EventCoordinator.Listener eventListener = nextEvent.getListener();
 +      while (stillMaster()) {
 +        long wait = DEFAULT_WAIT_FOR_WATCHER;
 +        try {
 +          switch (getMasterGoalState()) {
 +            case NORMAL:
 +              setMasterState(MasterState.NORMAL);
 +              break;
 +            case SAFE_MODE:
 +              if (getMasterState() == MasterState.NORMAL) {
 +                setMasterState(MasterState.SAFE_MODE);
 +              }
 +              if (getMasterState() == MasterState.HAVE_LOCK) {
 +                setMasterState(MasterState.SAFE_MODE);
 +              }
 +              break;
 +            case CLEAN_STOP:
 +              switch (getMasterState()) {
 +                case NORMAL:
 +                  setMasterState(MasterState.SAFE_MODE);
 +                  break;
 +                case SAFE_MODE: {
 +                  int count = nonMetaDataTabletsAssignedOrHosted();
 +                  log.debug(String.format("There are %d non-metadata tablets assigned or hosted", count));
 +                  if (count == 0)
 +                    setMasterState(MasterState.UNLOAD_METADATA_TABLETS);
 +                }
 +                  break;
 +                case UNLOAD_METADATA_TABLETS: {
 +                  int count = assignedOrHosted(METADATA_TABLE_ID);
 +                  log.debug(String.format("There are %d metadata tablets assigned or hosted", count));
 +                  if (count == 0)
 +                    setMasterState(MasterState.UNLOAD_ROOT_TABLET);
 +                }
 +                  break;
 +                case UNLOAD_ROOT_TABLET: {
 +                  int count = assignedOrHosted(METADATA_TABLE_ID);
 +                  if (count > 0) {
 +                    log.debug(String.format("%d metadata tablets online", count));
 +                    setMasterState(MasterState.UNLOAD_ROOT_TABLET);
 +                  }
 +                  int root_count = assignedOrHosted(ROOT_TABLE_ID);
 +                  if (root_count > 0)
 +                    log.debug("The root tablet is still assigned or hosted");
 +                  if (count + root_count == 0) {
 +                    Set<TServerInstance> currentServers = tserverSet.getCurrentServers();
 +                    log.debug("stopping " + currentServers.size() + " tablet servers");
 +                    for (TServerInstance server : currentServers) {
 +                      try {
 +                        serversToShutdown.add(server);
 +                        tserverSet.getConnection(server).fastHalt(masterLock);
 +                      } catch (TException e) {
 +                        // its probably down, and we don't care
 +                      } finally {
 +                        tserverSet.remove(server);
 +                      }
 +                    }
 +                    if (currentServers.size() == 0)
 +                      setMasterState(MasterState.STOP);
 +                  }
 +                }
 +                  break;
 +                default:
 +                  break;
 +              }
 +          }
 +          wait = updateStatus();
 +          eventListener.waitForEvents(wait);
 +        } catch (Throwable t) {
 +          log.error("Error balancing tablets", t);
 +          UtilWaitThread.sleep(WAIT_BETWEEN_ERRORS);
 +        }
 +      }
 +    }
 +
 +    private long updateStatus() throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
 +      tserverStatus = Collections.synchronizedSortedMap(gatherTableInformation());
 +      checkForHeldServer(tserverStatus);
 +
 +      if (!badServers.isEmpty()) {
 +        log.debug("not balancing because the balance information is out-of-date " + badServers.keySet());
 +      } else if (notHosted() > 0) {
 +        log.debug("not balancing because there are unhosted tablets");
 +      } else if (getMasterGoalState() == MasterGoalState.CLEAN_STOP) {
 +        log.debug("not balancing because the master is attempting to stop cleanly");
 +      } else if (!serversToShutdown.isEmpty()) {
 +        log.debug("not balancing while shutting down servers " + serversToShutdown);
 +      } else {
 +        return balanceTablets();
 +      }
 +      return DEFAULT_WAIT_FOR_WATCHER;
 +    }
 +
 +    private void checkForHeldServer(SortedMap<TServerInstance,TabletServerStatus> tserverStatus) {
 +      TServerInstance instance = null;
 +      int crazyHoldTime = 0;
 +      int someHoldTime = 0;
 +      final long maxWait = getSystemConfiguration().getTimeInMillis(Property.TSERV_HOLD_TIME_SUICIDE);
 +      for (Entry<TServerInstance,TabletServerStatus> entry : tserverStatus.entrySet()) {
 +        if (entry.getValue().getHoldTime() > 0) {
 +          someHoldTime++;
 +          if (entry.getValue().getHoldTime() > maxWait) {
 +            instance = entry.getKey();
 +            crazyHoldTime++;
 +          }
 +        }
 +      }
 +      if (crazyHoldTime == 1 && someHoldTime == 1 && tserverStatus.size() > 1) {
 +        log.warn("Tablet server " + instance + " exceeded maximum hold time: attempting to kill it");
 +        try {
 +          TServerConnection connection = tserverSet.getConnection(instance);
 +          if (connection != null)
 +            connection.fastHalt(masterLock);
 +        } catch (TException e) {
 +          log.error(e, e);
 +        }
 +        tserverSet.remove(instance);
 +      }
 +    }
 +
 +    private long balanceTablets() {
 +      List<TabletMigration> migrationsOut = new ArrayList<TabletMigration>();
 +      Set<KeyExtent> migrationsCopy = new HashSet<KeyExtent>();
 +      synchronized (migrations) {
 +        migrationsCopy.addAll(migrations.keySet());
 +      }
 +      long wait = tabletBalancer.balance(Collections.unmodifiableSortedMap(tserverStatus), Collections.unmodifiableSet(migrationsCopy), migrationsOut);
 +
 +      for (TabletMigration m : TabletBalancer.checkMigrationSanity(tserverStatus.keySet(), migrationsOut)) {
 +        if (migrations.containsKey(m.tablet)) {
 +          log.warn("balancer requested migration more than once, skipping " + m);
 +          continue;
 +        }
 +        migrations.put(m.tablet, m.newServer);
 +        log.debug("migration " + m);
 +      }
 +      if (migrationsOut.size() > 0) {
 +        nextEvent.event("Migrating %d more tablets, %d total", migrationsOut.size(), migrations.size());
 +      }
 +      return wait;
 +    }
 +
 +  }
 +
 +  private SortedMap<TServerInstance,TabletServerStatus> gatherTableInformation() {
 +    long start = System.currentTimeMillis();
 +    SortedMap<TServerInstance,TabletServerStatus> result = new TreeMap<TServerInstance,TabletServerStatus>();
 +    Set<TServerInstance> currentServers = tserverSet.getCurrentServers();
 +    for (TServerInstance server : currentServers) {
 +      try {
 +        Thread t = Thread.currentThread();
 +        String oldName = t.getName();
 +        try {
 +          t.setName("Getting status from " + server);
 +          TServerConnection connection = tserverSet.getConnection(server);
 +          if (connection == null)
 +            throw new IOException("No connection to " + server);
 +          TabletServerStatus status = connection.getTableMap(false);
 +          result.put(server, status);
 +        } finally {
 +          t.setName(oldName);
 +        }
 +      } catch (Exception ex) {
 +        log.error("unable to get tablet server status " + server + " " + ex.toString());
 +        log.debug("unable to get tablet server status " + server, ex);
 +        if (badServers.get(server).incrementAndGet() > MAX_BAD_STATUS_COUNT) {
 +          log.warn("attempting to stop " + server);
 +          try {
 +            TServerConnection connection = tserverSet.getConnection(server);
 +            if (connection != null)
 +              connection.halt(masterLock);
 +          } catch (TTransportException e) {
 +            // ignore: it's probably down
 +          } catch (Exception e) {
 +            log.info("error talking to troublesome tablet server ", e);
 +          }
 +          badServers.remove(server);
 +          tserverSet.remove(server);
 +        }
 +      }
 +    }
 +    synchronized (badServers) {
 +      badServers.keySet().retainAll(currentServers);
 +      badServers.keySet().removeAll(result.keySet());
 +    }
 +    log.debug(String.format("Finished gathering information from %d servers in %.2f seconds", result.size(), (System.currentTimeMillis() - start) / 1000.));
 +    return result;
 +  }
 +
 +  public void run() throws IOException, InterruptedException, KeeperException {
 +    final String zroot = ZooUtil.getRoot(instance);
 +
 +    getMasterLock(zroot + Constants.ZMASTER_LOCK);
 +
 +    recoveryManager = new RecoveryManager(this);
 +
 +    TableManager.getInstance().addObserver(this);
 +
 +    StatusThread statusThread = new StatusThread();
 +    statusThread.start();
 +
 +    MigrationCleanupThread migrationCleanupThread = new MigrationCleanupThread();
 +    migrationCleanupThread.start();
 +
 +    tserverSet.startListeningForTabletServerChanges();
 +
-     try {
-       final AgeOffStore<Master> store = new AgeOffStore<Master>(new org.apache.accumulo.fate.ZooStore<Master>(ZooUtil.getRoot(instance) + Constants.ZFATE,
-           ZooReaderWriter.getRetryingInstance()), 1000 * 60 * 60 * 8);
- 
-       int threads = this.getConfiguration().getConfiguration().getCount(Property.MASTER_FATE_THREADPOOL_SIZE);
- 
-       fate = new Fate<Master>(this, store, threads);
- 
-       SimpleTimer.getInstance().schedule(new Runnable() {
- 
-         @Override
-         public void run() {
-           store.ageOff();
-         }
-       }, 63000, 63000);
-     } catch (KeeperException e) {
-       throw new IOException(e);
-     } catch (InterruptedException e) {
-       throw new IOException(e);
-     }
- 
 +    ZooReaderWriter.getInstance().getChildren(zroot + Constants.ZRECOVERY, new Watcher() {
 +      @Override
 +      public void process(WatchedEvent event) {
 +        nextEvent.event("Noticed recovery changes", event.getType());
 +        try {
 +          // watcher only fires once, add it back
 +          ZooReaderWriter.getInstance().getChildren(zroot + Constants.ZRECOVERY, this);
 +        } catch (Exception e) {
 +          log.error("Failed to add log recovery watcher back", e);
 +        }
 +      }
 +    });
 +
 +    Credentials systemCreds = SystemCredentials.get();
 +    watchers.add(new TabletGroupWatcher(this, new MetaDataStateStore(instance, systemCreds, this), null));
 +    watchers.add(new TabletGroupWatcher(this, new RootTabletStateStore(instance, systemCreds, this), watchers.get(0)));
 +    watchers.add(new TabletGroupWatcher(this, new ZooTabletStateStore(new ZooStore(zroot)), watchers.get(1)));
 +    for (TabletGroupWatcher watcher : watchers) {
 +      watcher.start();
 +    }
++
++    // Once we are sure the upgrade is complete, we can safely allow fate use.
++    waitForMetadataUpgrade.await();
++
++    try {
++      final AgeOffStore<Master> store = new AgeOffStore<Master>(new org.apache.accumulo.fate.ZooStore<Master>(ZooUtil.getRoot(instance) + Constants.ZFATE,
++          ZooReaderWriter.getRetryingInstance()), 1000 * 60 * 60 * 8);
++
++      int threads = this.getConfiguration().getConfiguration().getCount(Property.MASTER_FATE_THREADPOOL_SIZE);
++
++      fate = new Fate<Master>(this, store, threads);
++
++      SimpleTimer.getInstance().schedule(new Runnable() {
++
++        @Override
++        public void run() {
++          store.ageOff();
++        }
++      }, 63000, 63000);
++    } catch (KeeperException e) {
++      throw new IOException(e);
++    } catch (InterruptedException e) {
++      throw new IOException(e);
++    }
 +
 +    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;
 +    String address = sa.address.toString();
 +    log.info("Setting master lock data to " + address);
 +    masterLock.replaceLockData(address.getBytes());
 +
 +    while (!clientService.isServing()) {
 +      UtilWaitThread.sleep(100);
 +    }
 +    while (clientService.isServing()) {
 +      UtilWaitThread.sleep(500);
 +    }
 +    log.info("Shutting down fate.");
 +    fate.shutdown();
 +
 +    final long deadline = System.currentTimeMillis() + MAX_CLEANUP_WAIT_TIME;
 +    statusThread.join(remaining(deadline));
 +
 +    // quit, even if the tablet servers somehow jam up and the watchers
 +    // don't stop
 +    for (TabletGroupWatcher watcher : watchers) {
 +      watcher.join(remaining(deadline));
 +    }
 +    log.info("exiting");
 +  }
 +
 +  private long remaining(long deadline) {
 +    return Math.max(1, deadline - System.currentTimeMillis());
 +  }
 +
 +  public ZooLock getMasterLock() {
 +    return masterLock;
 +  }
 +
 +  private static class MasterLockWatcher implements ZooLock.AsyncLockWatcher {
 +
 +    boolean acquiredLock = false;
 +    boolean failedToAcquireLock = false;
 +
 +    @Override
 +    public void lostLock(LockLossReason reason) {
 +      Halt.halt("Master lock in zookeeper lost (reason = " + reason + "), exiting!", -1);
 +    }
 +
 +    @Override
 +    public void unableToMonitorLockNode(final Throwable e) {
 +      Halt.halt(-1, new Runnable() {
 +        @Override
 +        public void run() {
 +          log.fatal("No longer able to monitor master lock node", e);
 +        }
 +      });
 +
 +    }
 +
 +    @Override
 +    public synchronized void acquiredLock() {
 +      log.debug("Acquired master lock");
 +
 +      if (acquiredLock || failedToAcquireLock) {
 +        Halt.halt("Zoolock in unexpected state AL " + acquiredLock + " " + failedToAcquireLock, -1);
 +      }
 +
 +      acquiredLock = true;
 +      notifyAll();
 +    }
 +
 +    @Override
 +    public synchronized void failedToAcquireLock(Exception e) {
 +      log.warn("Failed to get master lock " + e);
 +
 +      if (acquiredLock) {
 +        Halt.halt("Zoolock in unexpected state FAL " + acquiredLock + " " + failedToAcquireLock, -1);
 +      }
 +
 +      failedToAcquireLock = true;
 +      notifyAll();
 +    }
 +
 +    public synchronized void waitForChange() {
 +      while (!acquiredLock && !failedToAcquireLock) {
 +        try {
 +          wait();
 +        } catch (InterruptedException e) {}
 +      }
 +    }
 +  }
 +
 +  private void getMasterLock(final String zMasterLoc) throws KeeperException, InterruptedException {
 +    log.info("trying to get master lock");
 +
 +    final String masterClientAddress = hostname + ":" + getSystemConfiguration().getPort(Property.MASTER_CLIENTPORT);
 +
 +    while (true) {
 +
 +      MasterLockWatcher masterLockWatcher = new MasterLockWatcher();
 +      masterLock = new ZooLock(zMasterLoc);
 +      masterLock.lockAsync(masterLockWatcher, masterClientAddress.getBytes());
 +
 +      masterLockWatcher.waitForChange();
 +
 +      if (masterLockWatcher.acquiredLock) {
 +        break;
 +      }
 +
 +      if (!masterLockWatcher.failedToAcquireLock) {
 +        throw new IllegalStateException("master lock in unknown state");
 +      }
 +
 +      masterLock.tryToCancelAsyncLockOrUnlock();
 +
 +      UtilWaitThread.sleep(TIME_TO_WAIT_BETWEEN_LOCK_CHECKS);
 +    }
 +
 +    setMasterState(MasterState.HAVE_LOCK);
 +  }
 +
 +  public static void main(String[] args) throws Exception {
 +    try {
 +      SecurityUtil.serverLogin(ServerConfiguration.getSiteConfiguration());
 +
 +      VolumeManager fs = VolumeManagerImpl.get();
 +      ServerOpts opts = new ServerOpts();
 +      opts.parseArgs("master", args);
 +      String hostname = opts.getAddress();
 +      Instance instance = HdfsZooInstance.getInstance();
 +      ServerConfiguration conf = new ServerConfiguration(instance);
 +      Accumulo.init(fs, conf, "master");
 +      Master master = new Master(conf, fs, hostname);
 +      Accumulo.enableTracing(hostname, "master");
 +      master.run();
 +    } catch (Exception ex) {
 +      log.error("Unexpected exception, exiting", ex);
 +      System.exit(1);
 +    }
 +  }
 +
 +  @Override
 +  public void update(LiveTServerSet current, Set<TServerInstance> deleted, Set<TServerInstance> added) {
 +    DeadServerList obit = new DeadServerList(ZooUtil.getRoot(instance) + Constants.ZDEADTSERVERS);
 +    if (added.size() > 0) {
 +      log.info("New servers: " + added);
 +      for (TServerInstance up : added)
 +        obit.delete(up.hostPort());
 +    }
 +    for (TServerInstance dead : deleted) {
 +      String cause = "unexpected failure";
 +      if (serversToShutdown.contains(dead))
 +        cause = "clean shutdown"; // maybe an incorrect assumption
 +      if (!getMasterGoalState().equals(MasterGoalState.CLEAN_STOP))
 +        obit.post(dead.hostPort(), cause);
 +    }
 +
 +    Set<TServerInstance> unexpected = new HashSet<TServerInstance>(deleted);
 +    unexpected.removeAll(this.serversToShutdown);
 +    if (unexpected.size() > 0) {
 +      if (stillMaster() && !getMasterGoalState().equals(MasterGoalState.CLEAN_STOP)) {
 +        log.warn("Lost servers " + unexpected);
 +      }
 +    }
 +    serversToShutdown.removeAll(deleted);
 +    badServers.keySet().removeAll(deleted);
 +    // clear out any bad server with the same host/port as a new server
 +    synchronized (badServers) {
 +      cleanListByHostAndPort(badServers.keySet(), deleted, added);
 +    }
 +    synchronized (serversToShutdown) {
 +      cleanListByHostAndPort(serversToShutdown, deleted, added);
 +    }
 +
 +    synchronized (migrations) {
 +      Iterator<Entry<KeyExtent,TServerInstance>> iter = migrations.entrySet().iterator();
 +      while (iter.hasNext()) {
 +        Entry<KeyExtent,TServerInstance> entry = iter.next();
 +        if (deleted.contains(entry.getValue())) {
 +          log.info("Canceling migration of " + entry.getKey() + " to " + entry.getValue());
 +          iter.remove();
 +        }
 +      }
 +    }
 +    nextEvent.event("There are now %d tablet servers", current.size());
 +  }
 +
 +  private static void cleanListByHostAndPort(Collection<TServerInstance> badServers, Set<TServerInstance> deleted, Set<TServerInstance> added) {
 +    Iterator<TServerInstance> badIter = badServers.iterator();
 +    while (badIter.hasNext()) {
 +      TServerInstance bad = badIter.next();
 +      for (TServerInstance add : added) {
 +        if (bad.hostPort().equals(add.hostPort())) {
 +          badIter.remove();
 +          break;
 +        }
 +      }
 +      for (TServerInstance del : deleted) {
 +        if (bad.hostPort().equals(del.hostPort())) {
 +          badIter.remove();
 +          break;
 +        }
 +      }
 +    }
 +  }
 +
 +  @Override
 +  public void stateChanged(String tableId, TableState state) {
 +    nextEvent.event("Table state in zookeeper changed for %s to %s", tableId, state);
 +  }
 +
 +  @Override
 +  public void initialize(Map<String,TableState> tableIdToStateMap) {}
 +
 +  @Override
 +  public void sessionExpired() {}
 +
 +  @Override
 +  public Set<String> onlineTables() {
 +    Set<String> result = new HashSet<String>();
 +    if (getMasterState() != MasterState.NORMAL) {
 +      if (getMasterState() != MasterState.UNLOAD_METADATA_TABLETS)
 +        result.add(MetadataTable.ID);
 +      if (getMasterState() != MasterState.UNLOAD_ROOT_TABLET)
 +        result.add(RootTable.ID);
 +      return result;
 +    }
 +    TableManager manager = TableManager.getInstance();
 +
 +    for (String tableId : Tables.getIdToNameMap(instance).keySet()) {
 +      TableState state = manager.getTableState(tableId);
 +      if (state != null) {
 +        if (state == TableState.ONLINE)
 +          result.add(tableId);
 +      }
 +    }
 +    return result;
 +  }
 +
 +  @Override
 +  public Set<TServerInstance> onlineTabletServers() {
 +    return tserverSet.getCurrentServers();
 +  }
 +
 +  @Override
 +  public Collection<MergeInfo> merges() {
 +    List<MergeInfo> result = new ArrayList<MergeInfo>();
 +    for (String tableId : Tables.getIdToNameMap(instance).keySet()) {
 +      result.add(getMergeInfo(new Text(tableId)));
 +    }
 +    return result;
 +  }
 +
 +  // recovers state from the persistent transaction to shutdown a server
 +  public void shutdownTServer(TServerInstance server) {
 +    nextEvent.event("Tablet Server shutdown requested for %s", server);
 +    serversToShutdown.add(server);
 +  }
 +
 +  public EventCoordinator getEventCoordinator() {
 +    return nextEvent;
 +  }
 +
 +  public Instance getInstance() {
 +    return this.instance;
 +  }
 +
 +  public AccumuloConfiguration getSystemConfiguration() {
 +    return serverConfig.getConfiguration();
 +  }
 +
 +  public ServerConfiguration getConfiguration() {
 +    return serverConfig;
 +  }
 +
 +  public VolumeManager getFileSystem() {
 +    return this.fs;
 +  }
 +
 +  public void assignedTablet(KeyExtent extent) {
 +    if (extent.isMeta()) {
 +      if (getMasterState().equals(MasterState.UNLOAD_ROOT_TABLET)) {
 +        setMasterState(MasterState.UNLOAD_METADATA_TABLETS);
 +      }
 +    }
 +    if (extent.isRootTablet()) {
 +      // probably too late, but try anyhow
 +      if (getMasterState().equals(MasterState.STOP)) {
 +        setMasterState(MasterState.UNLOAD_ROOT_TABLET);
 +      }
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/957c9d1b/server/master/src/main/java/org/apache/accumulo/master/util/FateAdmin.java
----------------------------------------------------------------------
diff --cc server/master/src/main/java/org/apache/accumulo/master/util/FateAdmin.java
index 8fcb689,0000000..4e72832
mode 100644,000000..100644
--- a/server/master/src/main/java/org/apache/accumulo/master/util/FateAdmin.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/util/FateAdmin.java
@@@ -1,92 -1,0 +1,93 @@@
 +/*
 + * 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.util;
 +
 +import java.util.ArrayList;
 +import java.util.List;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.cli.Help;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.zookeeper.ZooUtil;
 +import org.apache.accumulo.fate.AdminUtil;
 +import org.apache.accumulo.fate.ZooStore;
++import org.apache.accumulo.fate.ReadOnlyStore;
 +import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 +import org.apache.accumulo.master.Master;
 +import org.apache.accumulo.server.client.HdfsZooInstance;
 +import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 +
 +import com.beust.jcommander.JCommander;
 +import com.beust.jcommander.Parameter;
 +import com.beust.jcommander.Parameters;
 +
 +/**
 + * A utility to administer FATE operations
 + */
 +public class FateAdmin {
 +  
 +  static class TxOpts {
 +    @Parameter(description = "<txid>", required = true)
 +    List<String> args = new ArrayList<String>();
 +  }
 +  
 +  @Parameters(commandDescription = "Stop an existing FATE by transaction id")
 +  static class FailOpts extends TxOpts {}
 +  
 +  @Parameters(commandDescription = "Delete an existing FATE by transaction id")
 +  static class DeleteOpts extends TxOpts {}
 +  
 +  @Parameters(commandDescription = "List the existing FATE transactions")
 +  static class PrintOpts {}
 +  
 +  public static void main(String[] args) throws Exception {
 +    Help opts = new Help();
 +    JCommander jc = new JCommander(opts);
 +    jc.setProgramName(FateAdmin.class.getName());
 +    jc.addCommand("fail", new FailOpts());
 +    jc.addCommand("delete", new DeleteOpts());
 +    jc.addCommand("print", new PrintOpts());
 +    jc.parse(args);
 +    if (opts.help || jc.getParsedCommand() == null) {
 +      jc.usage();
 +      System.exit(1);
 +    }
 +    
 +    System.err.printf("This tool has been deprecated%nFATE administration now available within 'accumulo shell'%n$ fate fail <txid>... | delete <txid>... | print [<txid>...]%n%n");
 +    
 +    AdminUtil<Master> admin = new AdminUtil<Master>();
 +    
 +    Instance instance = HdfsZooInstance.getInstance();
 +    String path = ZooUtil.getRoot(instance) + Constants.ZFATE;
 +    String masterPath = ZooUtil.getRoot(instance) + Constants.ZMASTER_LOCK;
 +    IZooReaderWriter zk = ZooReaderWriter.getRetryingInstance();
 +    ZooStore<Master> zs = new ZooStore<Master>(path, zk);
 +    
 +    if (jc.getParsedCommand().equals("fail")) {
 +      if (!admin.prepFail(zs, zk, masterPath, args[1])) {
 +        System.exit(1);
 +      }
 +    } else if (jc.getParsedCommand().equals("delete")) {
 +      if (!admin.prepDelete(zs, zk, masterPath, args[1])) {
 +        System.exit(1);
 +      }
 +      admin.deleteLocks(zs, zk, ZooUtil.getRoot(instance) + Constants.ZTABLE_LOCKS, args[1]);
 +    } else if (jc.getParsedCommand().equals("print")) {
-       admin.print(zs, zk, ZooUtil.getRoot(instance) + Constants.ZTABLE_LOCKS);
++      admin.print(new ReadOnlyStore(zs), zk, ZooUtil.getRoot(instance) + Constants.ZTABLE_LOCKS);
 +    }
 +  }
 +}


[15/15] git commit: Merge branch '1.6.0-SNAPSHOT'

Posted by bu...@apache.org.
Merge branch '1.6.0-SNAPSHOT'

Conflicts:
	server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
	server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
	server/master/src/main/java/org/apache/accumulo/master/Master.java


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

Branch: refs/heads/master
Commit: ebbd5e62b884a2c1d64dfdf1e32a4d097d4ac662
Parents: bdc2a99 f5a94f0
Author: Sean Busbey <bu...@cloudera.com>
Authored: Fri Apr 4 17:35:16 2014 -0700
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Apr 4 17:35:16 2014 -0700

----------------------------------------------------------------------
 README                                          |  40 ++++--
 .../core/util/shell/commands/FateCommand.java   |   2 +-
 fate/pom.xml                                    |   9 ++
 .../org/apache/accumulo/fate/AdminUtil.java     |  14 +--
 .../java/org/apache/accumulo/fate/Fate.java     |   2 +-
 .../org/apache/accumulo/fate/ReadOnlyRepo.java  |  32 +++++
 .../org/apache/accumulo/fate/ReadOnlyStore.java | 111 ++++++++++++++++
 .../apache/accumulo/fate/ReadOnlyTStore.java    | 125 +++++++++++++++++++
 .../java/org/apache/accumulo/fate/Repo.java     |   5 +-
 .../java/org/apache/accumulo/fate/TStore.java   |  62 +--------
 .../apache/accumulo/fate/AgeOffStoreTest.java   |   2 +-
 .../apache/accumulo/fate/ReadOnlyStoreTest.java |  72 +++++++++++
 .../org/apache/accumulo/fate/SimpleStore.java   |   2 +-
 .../org/apache/accumulo/server/Accumulo.java    |  33 ++++-
 .../accumulo/server/util/MetadataTableUtil.java |   3 +
 .../accumulo/master/FateServiceHandler.java     |   2 +-
 .../java/org/apache/accumulo/master/Master.java |  77 ++++++++----
 .../apache/accumulo/master/util/FateAdmin.java  |   3 +-
 18 files changed, 483 insertions(+), 113 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/ebbd5e62/fate/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ebbd5e62/fate/src/main/java/org/apache/accumulo/fate/Fate.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ebbd5e62/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
----------------------------------------------------------------------
diff --cc server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
index 29ed2b7,4e1eb35..c29f3c6
--- a/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
@@@ -27,7 -26,7 +27,8 @@@ import java.util.Map.Entry
  import java.util.TreeMap;
  
  import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
+ import org.apache.accumulo.core.client.AccumuloException;
  import org.apache.accumulo.core.conf.Property;
  import org.apache.accumulo.core.trace.DistributedTrace;
  import org.apache.accumulo.core.util.AddressUtil;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ebbd5e62/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ebbd5e62/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
----------------------------------------------------------------------
diff --cc server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
index ae60da0,d63a63e..ff488b9
--- a/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
@@@ -39,9 -39,9 +39,9 @@@ import org.apache.accumulo.core.iterato
  import org.apache.accumulo.core.master.thrift.FateOperation;
  import org.apache.accumulo.core.master.thrift.FateService;
  import org.apache.accumulo.core.security.thrift.TCredentials;
 -import org.apache.accumulo.core.util.ArgumentChecker.Validator;
  import org.apache.accumulo.core.util.ByteBufferUtil;
 +import org.apache.accumulo.core.util.Validator;
- import org.apache.accumulo.fate.TStore.TStatus;
+ import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
  import org.apache.accumulo.master.tableOps.BulkImport;
  import org.apache.accumulo.master.tableOps.CancelCompactions;
  import org.apache.accumulo.master.tableOps.ChangeTableState;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ebbd5e62/server/master/src/main/java/org/apache/accumulo/master/Master.java
----------------------------------------------------------------------


[06/15] git commit: ACCUMULO-2551 adds read only fate operations.

Posted by bu...@apache.org.
ACCUMULO-2551 adds read only fate operations.


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

Branch: refs/heads/1.5.2-SNAPSHOT
Commit: a904f69110942bd139e701d617d0aa3b647001f8
Parents: f67c386
Author: Sean Busbey <bu...@cloudera.com>
Authored: Tue Mar 25 17:29:59 2014 -0500
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Apr 4 17:27:05 2014 -0700

----------------------------------------------------------------------
 fate/pom.xml                                    |   9 ++
 .../org/apache/accumulo/fate/AdminUtil.java     |  12 +-
 .../java/org/apache/accumulo/fate/Fate.java     |   2 +-
 .../org/apache/accumulo/fate/ReadOnlyRepo.java  |  32 +++++
 .../org/apache/accumulo/fate/ReadOnlyStore.java | 111 ++++++++++++++++
 .../apache/accumulo/fate/ReadOnlyTStore.java    | 125 +++++++++++++++++++
 .../java/org/apache/accumulo/fate/Repo.java     |   5 +-
 .../java/org/apache/accumulo/fate/TStore.java   |  72 ++---------
 .../apache/accumulo/fate/AgeOffStoreTest.java   |   2 +-
 .../apache/accumulo/fate/ReadOnlyStoreTest.java |  72 +++++++++++
 .../org/apache/accumulo/fate/SimpleStore.java   |   2 +-
 .../org/apache/accumulo/server/fate/Admin.java  |   3 +-
 .../apache/accumulo/server/master/Master.java   |   2 +-
 13 files changed, 370 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/pom.xml
----------------------------------------------------------------------
diff --git a/fate/pom.xml b/fate/pom.xml
index 433e61a..0868e4c 100644
--- a/fate/pom.xml
+++ b/fate/pom.xml
@@ -26,6 +26,10 @@
   <name>Fate</name>
   <dependencies>
     <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
       <groupId>commons-lang</groupId>
       <artifactId>commons-lang</artifactId>
       <scope>provided</scope>
@@ -45,5 +49,10 @@
       <artifactId>junit</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.easymock</groupId>
+      <artifactId>easymock</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java b/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
index 0162466..0238fde 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
@@ -24,7 +24,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
-import org.apache.accumulo.fate.TStore.TStatus;
+import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooLock;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
@@ -36,7 +36,7 @@ import org.apache.zookeeper.KeeperException;
 public class AdminUtil<T> {
   private static final Charset UTF8 = Charset.forName("UTF-8");
   
-  public void print(ZooStore<T> zs, IZooReaderWriter zk, String lockPath) throws KeeperException, InterruptedException {
+  public void print(ReadOnlyTStore<T> zs, IZooReaderWriter zk, String lockPath) throws KeeperException, InterruptedException {
     Map<Long,List<String>> heldLocks = new HashMap<Long,List<String>>();
     Map<Long,List<String>> waitingLocks = new HashMap<Long,List<String>>();
     
@@ -108,7 +108,7 @@ public class AdminUtil<T> {
         wlocks = Collections.emptyList();
       
       String top = null;
-      Repo<T> repo = zs.top(tid);
+      ReadOnlyRepo<T> repo = zs.top(tid);
       if (repo != null)
         top = repo.getDescription();
       
@@ -132,7 +132,7 @@ public class AdminUtil<T> {
     }
   }
   
-  public boolean prepDelete(ZooStore<T> zs, IZooReaderWriter zk, String path, String txidStr) {
+  public boolean prepDelete(TStore<T> zs, IZooReaderWriter zk, String path, String txidStr) {
     if (!checkGlobalLock(zk, path)) {
       return false;
     }
@@ -145,7 +145,7 @@ public class AdminUtil<T> {
     return true;
   }
   
-  public boolean prepFail(ZooStore<T> zs, IZooReaderWriter zk, String path, String txidStr) {
+  public boolean prepFail(TStore<T> zs, IZooReaderWriter zk, String path, String txidStr) {
     if (!checkGlobalLock(zk, path)) {
       return false;
     }
@@ -158,7 +158,7 @@ public class AdminUtil<T> {
     return true;
   }
   
-  public void deleteLocks(ZooStore<T> zs, IZooReaderWriter zk, String path, String txidStr) throws KeeperException, InterruptedException {
+  public void deleteLocks(TStore<T> zs, IZooReaderWriter zk, String path, String txidStr) throws KeeperException, InterruptedException {
     // delete any locks assoc w/ fate operation
     List<String> lockedIds = zk.getChildren(path);
     

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/main/java/org/apache/accumulo/fate/Fate.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/Fate.java b/fate/src/main/java/org/apache/accumulo/fate/Fate.java
index 9d24b0b..b2eb681 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/Fate.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/Fate.java
@@ -18,7 +18,7 @@ package org.apache.accumulo.fate;
 
 import java.util.EnumSet;
 
-import org.apache.accumulo.fate.TStore.TStatus;
+import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
 import org.apache.accumulo.fate.util.Daemon;
 import org.apache.accumulo.fate.util.LoggingRunnable;
 import org.apache.log4j.Logger;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyRepo.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyRepo.java b/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyRepo.java
new file mode 100644
index 0000000..24d00d9
--- /dev/null
+++ b/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyRepo.java
@@ -0,0 +1,32 @@
+/*
+ * 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.fate;
+
+/**
+ * Read only access to a repeatable persisted operation.
+ *
+ * By definition, these methods are safe to call without impacting the state of FATE. They should also be
+ * safe to call without impacting the state of system components.
+ *
+ */
+public interface ReadOnlyRepo<T> {
+
+  long isReady(long tid, T environment) throws Exception;
+
+  String getDescription();
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyStore.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyStore.java b/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyStore.java
new file mode 100644
index 0000000..7cb20ff
--- /dev/null
+++ b/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyStore.java
@@ -0,0 +1,111 @@
+/*
+ * 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.fate;
+
+import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
+import com.google.common.base.Preconditions;
+
+import java.io.Serializable;
+import java.util.EnumSet;
+import java.util.List;
+
+/**
+ * This store decorates a TStore to make sure it can not be modified.
+ *
+ * Unlike relying directly on the ReadOnlyTStore interface, this class will not allow subsequent users to cast back to a
+ * mutable TStore successfully.
+ *
+ */
+public class ReadOnlyStore<T> implements ReadOnlyTStore<T> {
+
+  private final TStore<T> store;
+
+  /**
+   * @param store may not be null
+   */
+  public ReadOnlyStore(TStore<T> store) {
+    Preconditions.checkNotNull(store);
+    this.store = store;
+  }
+
+  @Override
+  public long reserve() {
+    return store.reserve();
+  }
+
+  @Override
+  public void reserve(long tid) {
+    store.reserve(tid);
+  }
+
+  @Override
+  public void unreserve(long tid, long deferTime) {
+    store.unreserve(tid, deferTime);
+  }
+
+  /**
+   * Decorates a Repo to make sure it is treated as a ReadOnlyRepo.
+   *
+   * Similar to ReadOnlyStore, won't allow subsequent user to cast a ReadOnlyRepo back to a mutable Repo.
+   */
+  protected static class ReadOnlyRepoWrapper<X> implements ReadOnlyRepo<X> {
+    private final Repo<X> repo;
+
+    /**
+     * @param repo may not be null
+     */
+    public ReadOnlyRepoWrapper(Repo<X> repo) {
+      Preconditions.checkNotNull(repo);
+      this.repo = repo;
+    }
+
+    @Override
+    public long isReady(long tid, X environment) throws Exception {
+      return repo.isReady(tid, environment);
+    }
+
+    @Override
+    public String getDescription() {
+      return repo.getDescription();
+    }
+  }
+
+  @Override
+  public ReadOnlyRepo<T> top(long tid) {
+    return new ReadOnlyRepoWrapper(store.top(tid));
+  }
+
+  @Override
+  public TStatus getStatus(long tid) {
+    return store.getStatus(tid);
+  }
+
+  @Override
+  public TStatus waitForStatusChange(long tid, EnumSet<TStatus> expected) {
+    return store.waitForStatusChange(tid, expected);
+  }
+
+  @Override
+  public Serializable getProperty(long tid, String prop) {
+    return store.getProperty(tid, prop);
+  }
+
+  @Override
+  public List<Long> list() {
+    return store.list();
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyTStore.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyTStore.java b/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyTStore.java
new file mode 100644
index 0000000..d390139
--- /dev/null
+++ b/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyTStore.java
@@ -0,0 +1,125 @@
+/*
+ * 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.fate;
+
+import java.io.Serializable;
+import java.util.EnumSet;
+import java.util.List;
+
+/**
+ * Read only access to a Transaction Store.
+ *
+ * A transaction consists of a number of operations. Instances of this class may check on the queue of outstanding
+ * transactions but may neither modify them nor create new ones.
+ */
+public interface ReadOnlyTStore<T> {
+
+  /**
+   * Possible operational status codes. Serialized by name within stores.
+   */
+  enum TStatus {
+    /** Unseeded transaction */
+    NEW,
+    /** Transaction is eligible to be executing */
+    IN_PROGRESS,
+    /** Transaction has failed, and is in the process of being rolled back */
+    FAILED_IN_PROGRESS,
+    /** Transaction has failed and has been fully rolled back */
+    FAILED,
+    /** Transaction has succeeded */
+    SUCCESSFUL,
+    /** Unrecognized or unknown transaction state */
+    UNKNOWN
+  }
+
+  /**
+   * Reserve a transaction that is IN_PROGRESS or FAILED_IN_PROGRESS.
+   *
+   * Reserving a transaction id ensures that nothing else in-process interacting via the same instance
+   * will be operating on that transaction id.
+   *
+   * @return a transaction id that is safe to interact with, chosen by the store.
+   */
+  long reserve();
+
+  /**
+   * Reserve the specific tid.
+   *
+   * Reserving a transaction id ensures that nothing else in-process interacting via the same instance
+   * will be operating on that transaction id.
+   *
+   */
+  void reserve(long tid);
+
+  /**
+   * Return the given transaction to the store.
+   *
+   * upon successful return the store now controls the referenced transaction id. caller should no longer interact with it.
+   *
+   * @param tid transaction id, previously reserved.
+   * @param deferTime time in millis to keep this transaction out of the pool used in the {@link #reserve() reserve} method. must be non-negative.
+   */
+  void unreserve(long tid, long deferTime);
+
+
+  /**
+   * Get the current operation for the given transaction id.
+   *
+   * Caller must have already reserved tid.
+   *
+   * @param tid transaction id, previously reserved.
+   * @return a read-only view of the operation
+   */
+  ReadOnlyRepo<T> top(long tid);
+
+  /**
+   * Get the state of a given transaction.
+   *
+   * Caller must have already reserved tid.
+   *
+   * @param tid transaction id, previously reserved.
+   * @return execution status
+   */
+  TStatus getStatus(long tid);
+
+  /**
+   * Wait for the satus of a transaction to change
+   *
+   * @param tid transaction id, need not have been reserved.
+   * @param expected a set of possible statuses we are interested in being notified about. may not be null.
+   * @return execution status.
+   */
+  TStatus waitForStatusChange(long tid, EnumSet<TStatus> expected);
+
+  /**
+   * Retrieve a transaction-specific property.
+   *
+   * Caller must have already reserved tid.
+   *
+   * @param tid transaction id, previously reserved.
+   * @param prop name of property to retrieve.
+   */
+  Serializable getProperty(long tid, String prop);
+
+  /**
+   * list all transaction ids in store.
+   *
+   * @return all outstanding transactions, including those reserved by others.
+   */
+  List<Long> list();
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/main/java/org/apache/accumulo/fate/Repo.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/Repo.java b/fate/src/main/java/org/apache/accumulo/fate/Repo.java
index 8bdca10..b0ebd1a 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/Repo.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/Repo.java
@@ -22,15 +22,12 @@ import java.io.Serializable;
  * Repeatable persisted operation
  * 
  */
-public interface Repo<T> extends Serializable {
-  long isReady(long tid, T environment) throws Exception;
+public interface Repo<T> extends ReadOnlyRepo<T>, Serializable {
   
   Repo<T> call(long tid, T environment) throws Exception;
   
   void undo(long tid, T environment) throws Exception;
   
-  String getDescription();
-  
   // this allows the last fate op to return something to the user
   String getReturn();
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/main/java/org/apache/accumulo/fate/TStore.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/TStore.java b/fate/src/main/java/org/apache/accumulo/fate/TStore.java
index 3554064..5ca24fc 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/TStore.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/TStore.java
@@ -27,45 +27,14 @@ import java.util.List;
  * service can then execute the transaction's operation, possibly pushing more operations onto the transaction as each step successfully completes. If a step
  * fails, the stack can be unwound, undoing each operation.
  */
-public interface TStore<T> {
-  
-  public enum TStatus {
-    /** Unseeded transaction */
-    NEW,
-    /** Transaction is eligible to be executing */
-    IN_PROGRESS,
-    /** Transaction has failed, and is in the process of being rolled back */
-    FAILED_IN_PROGRESS,
-    /** Transaction has failed and has been fully rolled back */
-    FAILED,
-    /** Transaction has succeeded */
-    SUCCESSFUL,
-    /** Unrecognized or unknown transaction state */
-    UNKNOWN
-  }
+public interface TStore<T> extends ReadOnlyTStore<T> {
   
   /**
    * Create a new transaction id
    * 
    * @return a transaction id
    */
-  public long create();
-  
-  /**
-   * Reserve a transaction that is IN_PROGRESS or FAILED_IN_PROGRESS.
-   * 
-   */
-  long reserve();
-  
-  public void reserve(long tid);
-  
-  /**
-   * Return the given transaction to the store
-   * 
-   * @param tid
-   * @param deferTime
-   */
-  void unreserve(long tid, long deferTime);
+  long create();
   
   /**
    * Get the current operation for the given transaction id.
@@ -74,6 +43,7 @@ public interface TStore<T> {
    *          transaction id
    * @return the operation
    */
+  @Override
   Repo<T> top(long tid);
   
   /**
@@ -84,7 +54,7 @@ public interface TStore<T> {
    * @param repo
    *          the operation
    */
-  public void push(long tid, Repo<T> repo) throws StackOverflowException;
+  void push(long tid, Repo<T> repo) throws StackOverflowException;
   
   /**
    * Remove the last pushed operation from the given transaction.
@@ -94,15 +64,6 @@ public interface TStore<T> {
   void pop(long tid);
   
   /**
-   * Get the state of a given transaction.
-   * 
-   * @param tid
-   *          transaction id
-   * @return execution status
-   */
-  public TStatus getStatus(long tid);
-  
-  /**
    * Update the state of a given transaction
    * 
    * @param tid
@@ -110,19 +71,9 @@ public interface TStore<T> {
    * @param status
    *          execution status
    */
-  public void setStatus(long tid, TStatus status);
-  
-  /**
-   * Wait for the satus of a transaction to change
-   * 
-   * @param tid
-   *          transaction id
-   */
-  public TStatus waitForStatusChange(long tid, EnumSet<TStatus> expected);
-  
-  public void setProperty(long tid, String prop, Serializable val);
+  void setStatus(long tid, TStatus status);
   
-  public Serializable getProperty(long tid, String prop);
+  void setProperty(long tid, String prop, Serializable val);
   
   /**
    * Remove the transaction from the store.
@@ -130,13 +81,6 @@ public interface TStore<T> {
    * @param tid
    *          the transaction id
    */
-  public void delete(long tid);
-  
-  /**
-   * list all transaction ids in store
-   * 
-   */
-  
-  public List<Long> list();
-  
+  void delete(long tid);
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/test/java/org/apache/accumulo/fate/AgeOffStoreTest.java
----------------------------------------------------------------------
diff --git a/fate/src/test/java/org/apache/accumulo/fate/AgeOffStoreTest.java b/fate/src/test/java/org/apache/accumulo/fate/AgeOffStoreTest.java
index c212649..4f5b112 100644
--- a/fate/src/test/java/org/apache/accumulo/fate/AgeOffStoreTest.java
+++ b/fate/src/test/java/org/apache/accumulo/fate/AgeOffStoreTest.java
@@ -20,7 +20,7 @@ import java.util.Arrays;
 import java.util.HashSet;
 
 import org.apache.accumulo.fate.AgeOffStore.TimeSource;
-import org.apache.accumulo.fate.TStore.TStatus;
+import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
 import org.junit.Assert;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/test/java/org/apache/accumulo/fate/ReadOnlyStoreTest.java
----------------------------------------------------------------------
diff --git a/fate/src/test/java/org/apache/accumulo/fate/ReadOnlyStoreTest.java b/fate/src/test/java/org/apache/accumulo/fate/ReadOnlyStoreTest.java
new file mode 100644
index 0000000..c2d5f92
--- /dev/null
+++ b/fate/src/test/java/org/apache/accumulo/fate/ReadOnlyStoreTest.java
@@ -0,0 +1,72 @@
+/*
+ * 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.fate;
+
+import java.util.Collections;
+import java.util.EnumSet;
+
+import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
+
+import org.easymock.EasyMock;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Make sure read only decorate passes read methods.
+ */
+public class ReadOnlyStoreTest {
+
+  @Test
+  public void everythingPassesThrough() throws Exception {
+    @SuppressWarnings("unchecked")
+    Repo<String> repo = EasyMock.createMock(Repo.class);
+    EasyMock.expect(repo.getDescription()).andReturn("description");
+    EasyMock.expect(repo.isReady(0xdeadbeefl, null)).andReturn(0x0l);
+
+    @SuppressWarnings("unchecked")
+    TStore<String> mock = EasyMock.createNiceMock(TStore.class);
+    EasyMock.expect(mock.reserve()).andReturn(0xdeadbeefl);
+    mock.reserve(0xdeadbeefl);
+    EasyMock.expect(mock.top(0xdeadbeefl)).andReturn(repo);
+    EasyMock.expect(mock.getStatus(0xdeadbeefl)).andReturn(TStatus.UNKNOWN);
+    mock.unreserve(0xdeadbeefl, 30);
+
+    EasyMock.expect(mock.waitForStatusChange(0xdeadbeefl, EnumSet.allOf(TStatus.class))).andReturn(TStatus.UNKNOWN);
+    EasyMock.expect(mock.getProperty(0xdeadbeefl, "com.example.anyproperty")).andReturn("property");
+    EasyMock.expect(mock.list()).andReturn(Collections.<Long>emptyList());
+
+    EasyMock.replay(repo);
+    EasyMock.replay(mock);
+
+    ReadOnlyTStore<String> store = new ReadOnlyStore<String>(mock);
+    Assert.assertEquals(0xdeadbeefl, store.reserve());
+    store.reserve(0xdeadbeefl);
+    ReadOnlyRepo<String> top = store.top(0xdeadbeefl);
+    Assert.assertFalse(top instanceof Repo);
+    Assert.assertEquals("description", top.getDescription());
+    Assert.assertEquals(0x0l, top.isReady(0xdeadbeefl, null));
+    Assert.assertEquals(TStatus.UNKNOWN, store.getStatus(0xdeadbeefl));
+    store.unreserve(0xdeadbeefl, 30);
+
+    Assert.assertEquals(TStatus.UNKNOWN, store.waitForStatusChange(0xdeadbeefl, EnumSet.allOf(TStatus.class)));
+    Assert.assertEquals("property", store.getProperty(0xdeadbeefl, "com.example.anyproperty"));
+    Assert.assertEquals(Collections.<Long>emptyList(), store.list());
+
+    EasyMock.verify(repo);
+    EasyMock.verify(mock);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/test/java/org/apache/accumulo/fate/SimpleStore.java
----------------------------------------------------------------------
diff --git a/fate/src/test/java/org/apache/accumulo/fate/SimpleStore.java b/fate/src/test/java/org/apache/accumulo/fate/SimpleStore.java
index 3b78131..60eabfb 100644
--- a/fate/src/test/java/org/apache/accumulo/fate/SimpleStore.java
+++ b/fate/src/test/java/org/apache/accumulo/fate/SimpleStore.java
@@ -28,7 +28,7 @@ import java.util.Set;
 import org.apache.commons.lang.NotImplementedException;
 
 /**
- * 
+ * Transient in memory store for transactions.
  */
 public class SimpleStore<T> implements TStore<T> {
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/server/src/main/java/org/apache/accumulo/server/fate/Admin.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/fate/Admin.java b/server/src/main/java/org/apache/accumulo/server/fate/Admin.java
index 4a5f0bc..fc9e342 100644
--- a/server/src/main/java/org/apache/accumulo/server/fate/Admin.java
+++ b/server/src/main/java/org/apache/accumulo/server/fate/Admin.java
@@ -25,6 +25,7 @@ import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.AdminUtil;
 import org.apache.accumulo.fate.ZooStore;
+import org.apache.accumulo.fate.ReadOnlyStore;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.master.Master;
@@ -88,7 +89,7 @@ public class Admin {
       }
       admin.deleteLocks(zs, zk, ZooUtil.getRoot(instance) + Constants.ZTABLE_LOCKS, args[1]);
     } else if (jc.getParsedCommand().equals("print")) {
-      admin.print(zs, zk, ZooUtil.getRoot(instance) + Constants.ZTABLE_LOCKS);
+      admin.print(new ReadOnlyStore(zs), zk, ZooUtil.getRoot(instance) + Constants.ZTABLE_LOCKS);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/server/src/main/java/org/apache/accumulo/server/master/Master.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/Master.java b/server/src/main/java/org/apache/accumulo/server/master/Master.java
index 8c4c864..270eb18 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/Master.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/Master.java
@@ -90,7 +90,7 @@ 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.ReadOnlyTStore.TStatus;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooLock.LockLossReason;
 import org.apache.accumulo.fate.zookeeper.ZooReaderWriter.Mutator;


[10/15] git commit: Merge branch '1.5.2-SNAPSHOT' into 1.6.0-SNAPSHOT

Posted by bu...@apache.org.
Merge branch '1.5.2-SNAPSHOT' into 1.6.0-SNAPSHOT

  Conflicts:
      fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
      fate/src/main/java/org/apache/accumulo/fate/TStore.java
      server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
      server/src/main/java/org/apache/accumulo/server/master/Master.java
      server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
      server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java


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

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: 957c9d1b34afb8a5b7347921ba2d5b0b22bbc828
Parents: 2cb526e 5a504b3
Author: Sean Busbey <bu...@cloudera.com>
Authored: Fri Apr 4 17:27:39 2014 -0700
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Apr 4 17:28:32 2014 -0700

----------------------------------------------------------------------
 README                                          |  14 +++
 fate/pom.xml                                    |   9 ++
 .../org/apache/accumulo/fate/AdminUtil.java     |  14 +--
 .../java/org/apache/accumulo/fate/Fate.java     |   2 +-
 .../org/apache/accumulo/fate/ReadOnlyRepo.java  |  32 +++++
 .../org/apache/accumulo/fate/ReadOnlyStore.java | 111 ++++++++++++++++
 .../apache/accumulo/fate/ReadOnlyTStore.java    | 125 +++++++++++++++++++
 .../java/org/apache/accumulo/fate/Repo.java     |   5 +-
 .../java/org/apache/accumulo/fate/TStore.java   |  62 +--------
 .../apache/accumulo/fate/AgeOffStoreTest.java   |   2 +-
 .../apache/accumulo/fate/ReadOnlyStoreTest.java |  72 +++++++++++
 .../org/apache/accumulo/fate/SimpleStore.java   |   2 +-
 .../org/apache/accumulo/server/Accumulo.java    |  33 ++++-
 .../accumulo/server/util/MetadataTableUtil.java |   3 +
 .../java/org/apache/accumulo/master/Master.java |  77 ++++++++----
 .../apache/accumulo/master/util/FateAdmin.java  |   3 +-
 16 files changed, 464 insertions(+), 102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/957c9d1b/README
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/957c9d1b/fate/pom.xml
----------------------------------------------------------------------
diff --cc fate/pom.xml
index b339758,0868e4c..a3a7b42
--- a/fate/pom.xml
+++ b/fate/pom.xml
@@@ -24,11 -24,15 +24,15 @@@
    </parent>
    <artifactId>accumulo-fate</artifactId>
    <name>Fate</name>
 +  <description>A FAult-Tolerant Executor library used by Apache Accumulo.</description>
    <dependencies>
      <dependency>
+       <groupId>com.google.guava</groupId>
+       <artifactId>guava</artifactId>
+     </dependency>
+     <dependency>
        <groupId>commons-lang</groupId>
        <artifactId>commons-lang</artifactId>
 -      <scope>provided</scope>
      </dependency>
      <dependency>
        <groupId>log4j</groupId>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/957c9d1b/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
----------------------------------------------------------------------
diff --cc fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
index 8277c71,0238fde..1f8c32d
--- a/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
@@@ -25,9 -23,8 +25,9 @@@ import java.util.HashMap
  import java.util.List;
  import java.util.Map;
  import java.util.Map.Entry;
 +import java.util.Set;
  
- import org.apache.accumulo.fate.TStore.TStatus;
+ import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
  import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
  import org.apache.accumulo.fate.zookeeper.ZooLock;
  import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
@@@ -39,32 -36,7 +39,32 @@@ import org.apache.zookeeper.KeeperExcep
  public class AdminUtil<T> {
    private static final Charset UTF8 = Charset.forName("UTF-8");
    
 +  private boolean exitOnError = false;
 +  
 +  /**
 +   * Default constructor
 +   */
 +  public AdminUtil() {
 +    this(true);
 +  }
 +  
 +  /**
 +   * Constructor
 +   * 
 +   * @param exitOnError
 +   *          <code>System.exit(1)</code> on error if true
 +   */
 +  public AdminUtil(boolean exitOnError) {
 +    super();
 +    this.exitOnError = exitOnError;
 +  }
 +  
-   public void print(ZooStore<T> zs, IZooReaderWriter zk, String lockPath) throws KeeperException, InterruptedException {
+   public void print(ReadOnlyTStore<T> zs, IZooReaderWriter zk, String lockPath) throws KeeperException, InterruptedException {
 +    print(zs, zk, lockPath, new Formatter(System.out), null, null);
 +  }
 +  
-   public void print(ZooStore<T> zs, IZooReaderWriter zk, String lockPath, Formatter fmt, Set<Long> filterTxid, EnumSet<TStatus> filterStatus)
++  public void print(ReadOnlyTStore<T> zs, IZooReaderWriter zk, String lockPath, Formatter fmt, Set<Long> filterTxid, EnumSet<TStatus> filterStatus)
 +      throws KeeperException, InterruptedException {
      Map<Long,List<String>> heldLocks = new HashMap<Long,List<String>>();
      Map<Long,List<String>> waitingLocks = new HashMap<Long,List<String>>();
      
@@@ -169,79 -137,28 +169,79 @@@
        return false;
      }
      
 -    long txid = Long.parseLong(txidStr, 16);
 +    long txid;
 +    try {
 +      txid = Long.parseLong(txidStr, 16);
 +    } catch (NumberFormatException nfe) {
 +      System.out.printf("Invalid transaction ID format: %s%n", txidStr);
 +      return false;
 +    }
 +    boolean state = false;
      zs.reserve(txid);
 -    zs.delete(txid);
 -    zs.unreserve(txid, 0);
 +    TStatus ts = zs.getStatus(txid);
 +    switch (ts) {
 +      case UNKNOWN:
 +        System.out.printf("Invalid transaction ID: %016x%n", txid);
 +        break;
 +      
 +      case IN_PROGRESS:
 +      case NEW:
 +      case FAILED:
 +      case FAILED_IN_PROGRESS:
 +      case SUCCESSFUL:
 +        System.out.printf("Deleting transaction: %016x (%s)%n", txid, ts);
 +        zs.delete(txid);
 +        state = true;
 +        break;
 +    }
      
 -    return true;
 +    zs.unreserve(txid, 0);
 +    return state;
    }
    
-   public boolean prepFail(ZooStore<T> zs, IZooReaderWriter zk, String path, String txidStr) {
+   public boolean prepFail(TStore<T> zs, IZooReaderWriter zk, String path, String txidStr) {
      if (!checkGlobalLock(zk, path)) {
        return false;
      }
      
 -    long txid = Long.parseLong(txidStr, 16);
 +    long txid;
 +    try {
 +      txid = Long.parseLong(txidStr, 16);
 +    } catch (NumberFormatException nfe) {
 +      System.out.printf("Invalid transaction ID format: %s%n", txidStr);
 +      return false;
 +    }
 +    boolean state = false;
      zs.reserve(txid);
 -    zs.setStatus(txid, TStatus.FAILED_IN_PROGRESS);
 -    zs.unreserve(txid, 0);
 +    TStatus ts = zs.getStatus(txid);
 +    switch (ts) {
 +      case UNKNOWN:
 +        System.out.printf("Invalid transaction ID: %016x%n", txid);
 +        break;
 +      
 +      case IN_PROGRESS:
 +      case NEW:
 +        System.out.printf("Failing transaction: %016x (%s)%n", txid, ts);
 +        zs.setStatus(txid, TStatus.FAILED_IN_PROGRESS);
 +        state = true;
 +        break;
 +      
 +      case SUCCESSFUL:
 +        System.out.printf("Transaction already completed: %016x (%s)%n", txid, ts);
 +        break;
 +      
 +      case FAILED:
 +      case FAILED_IN_PROGRESS:
 +        System.out.printf("Transaction already failed: %016x (%s)%n", txid, ts);
 +        state = true;
 +        break;
 +    }
      
 -    return true;
 +    zs.unreserve(txid, 0);
 +    return state;
    }
    
-   public void deleteLocks(ZooStore<T> zs, IZooReaderWriter zk, String path, String txidStr) throws KeeperException, InterruptedException {
+   public void deleteLocks(TStore<T> zs, IZooReaderWriter zk, String path, String txidStr) throws KeeperException, InterruptedException {
      // delete any locks assoc w/ fate operation
      List<String> lockedIds = zk.getChildren(path);
      

http://git-wip-us.apache.org/repos/asf/accumulo/blob/957c9d1b/fate/src/main/java/org/apache/accumulo/fate/Fate.java
----------------------------------------------------------------------
diff --cc fate/src/main/java/org/apache/accumulo/fate/Fate.java
index 3561fc8,b2eb681..2b232ac
--- a/fate/src/main/java/org/apache/accumulo/fate/Fate.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/Fate.java
@@@ -17,9 -17,8 +17,9 @@@
  package org.apache.accumulo.fate;
  
  import java.util.EnumSet;
 +import java.util.concurrent.atomic.AtomicBoolean;
  
- import org.apache.accumulo.fate.TStore.TStatus;
+ import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
  import org.apache.accumulo.fate.util.Daemon;
  import org.apache.accumulo.fate.util.LoggingRunnable;
  import org.apache.log4j.Logger;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/957c9d1b/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
----------------------------------------------------------------------
diff --cc server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
index a9e4b5f,0000000..4e1eb35
mode 100644,000000..100644
--- a/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
@@@ -1,242 -1,0 +1,271 @@@
 +/*
 + * 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.server;
 +
 +import java.io.File;
 +import java.io.FileInputStream;
 +import java.io.IOException;
 +import java.io.InputStream;
 +import java.net.InetAddress;
 +import java.net.UnknownHostException;
 +import java.util.Map.Entry;
 +import java.util.TreeMap;
 +
 +import org.apache.accumulo.core.Constants;
++import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.trace.DistributedTrace;
 +import org.apache.accumulo.core.util.AddressUtil;
 +import org.apache.accumulo.core.util.UtilWaitThread;
 +import org.apache.accumulo.core.util.Version;
 +import org.apache.accumulo.core.volume.Volume;
++import org.apache.accumulo.core.zookeeper.ZooUtil;
++import org.apache.accumulo.fate.ReadOnlyTStore;
++import org.apache.accumulo.fate.ReadOnlyStore;
++import org.apache.accumulo.fate.ZooStore;
 +import org.apache.accumulo.server.client.HdfsZooInstance;
 +import org.apache.accumulo.server.conf.ServerConfiguration;
 +import org.apache.accumulo.server.fs.VolumeManager;
 +import org.apache.accumulo.server.util.time.SimpleTimer;
 +import org.apache.accumulo.server.watcher.MonitorLog4jWatcher;
 +import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 +import org.apache.hadoop.fs.FileStatus;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.log4j.Logger;
 +import org.apache.log4j.helpers.LogLog;
 +import org.apache.log4j.xml.DOMConfigurator;
 +import org.apache.zookeeper.KeeperException;
 +
 +public class Accumulo {
 +  
 +  private static final Logger log = Logger.getLogger(Accumulo.class);
 +  
 +  public static synchronized void updateAccumuloVersion(VolumeManager fs) {
 +    for (Volume volume : fs.getVolumes()) {
 +      try {
 +        if (getAccumuloPersistentVersion(fs) == ServerConstants.PREV_DATA_VERSION) {
 +          log.debug("Attempting to upgrade " + volume);
 +          Path dataVersionLocation = ServerConstants.getDataVersionLocation(volume);
 +          fs.create(new Path(dataVersionLocation, Integer.toString(ServerConstants.DATA_VERSION))).close();
- 
++          // TODO document failure mode & recovery if FS permissions cause above to work and below to fail ACCUMULO-2596
 +          Path prevDataVersionLoc = new Path(dataVersionLocation, Integer.toString(ServerConstants.PREV_DATA_VERSION));
 +          if (!fs.delete(prevDataVersionLoc)) {
 +            throw new RuntimeException("Could not delete previous data version location (" + prevDataVersionLoc + ") for " + volume);
 +          }
 +        }
 +      } catch (IOException e) {
 +        throw new RuntimeException("Unable to set accumulo version: an error occurred.", e);
 +      }
 +    }
 +  }
 +  
 +  public static synchronized int getAccumuloPersistentVersion(FileSystem fs, Path path) {
 +    int dataVersion;
 +    try {
 +      FileStatus[] files = fs.listStatus(path);
 +      if (files == null || files.length == 0) {
 +        dataVersion = -1; // assume it is 0.5 or earlier
 +      } else {
 +        dataVersion = Integer.parseInt(files[0].getPath().getName());
 +      }
 +      return dataVersion;
 +    } catch (IOException e) {
 +      throw new RuntimeException("Unable to read accumulo version: an error occurred.", e);
 +    }
 +  }
 +  
 +  public static synchronized int getAccumuloPersistentVersion(VolumeManager fs) {
 +    // It doesn't matter which Volume is used as they should all have the data version stored
 +    Volume v = fs.getVolumes().iterator().next();
 +    Path path = ServerConstants.getDataVersionLocation(v);
 +    return getAccumuloPersistentVersion(v.getFileSystem(), path);
 +  }
 +
 +  public static synchronized Path getAccumuloInstanceIdPath(VolumeManager fs) {
 +    // It doesn't matter which Volume is used as they should all have the instance ID stored
 +    Volume v = fs.getVolumes().iterator().next();
 +    return ServerConstants.getInstanceIdLocation(v);
 +  }
 +
 +  public static void enableTracing(String address, String application) {
 +    try {
 +      DistributedTrace.enable(HdfsZooInstance.getInstance(), ZooReaderWriter.getInstance(), application, address);
 +    } catch (Exception ex) {
 +      log.error("creating remote sink for trace spans", ex);
 +    }
 +  }
 +  
 +  public static void init(VolumeManager fs, ServerConfiguration config, String application) throws UnknownHostException {
 +    
 +    System.setProperty("org.apache.accumulo.core.application", application);
 +    
 +    if (System.getenv("ACCUMULO_LOG_DIR") != null)
 +      System.setProperty("org.apache.accumulo.core.dir.log", System.getenv("ACCUMULO_LOG_DIR"));
 +    else
 +      System.setProperty("org.apache.accumulo.core.dir.log", System.getenv("ACCUMULO_HOME") + "/logs/");
 +    
 +    String localhost = InetAddress.getLocalHost().getHostName();
 +    System.setProperty("org.apache.accumulo.core.ip.localhost.hostname", localhost);
 +    
 +    int logPort = config.getConfiguration().getPort(Property.MONITOR_LOG4J_PORT);
 +    System.setProperty("org.apache.accumulo.core.host.log.port", Integer.toString(logPort));
 +    
 +    // Use a specific log config, if it exists
 +    String logConfig = String.format("%s/%s_logger.xml", System.getenv("ACCUMULO_CONF_DIR"), application);
 +    if (!new File(logConfig).exists()) {
 +      // otherwise, use the generic config
 +      logConfig = String.format("%s/generic_logger.xml", System.getenv("ACCUMULO_CONF_DIR"));
 +    }
 +    // Turn off messages about not being able to reach the remote logger... we protect against that.
 +    LogLog.setQuietMode(true);
 +
 +    // Read the auditing config
 +    String auditConfig = String.format("%s/auditLog.xml", System.getenv("ACCUMULO_CONF_DIR"));
 +
 +    DOMConfigurator.configureAndWatch(auditConfig, 5000);
 +
 +    // Configure logging using information advertised in zookeeper by the monitor
 +    new MonitorLog4jWatcher(config.getInstance().getInstanceID(), logConfig, 5000).start();
 +
 +    log.info(application + " starting");
 +    log.info("Instance " + config.getInstance().getInstanceID());
 +    int dataVersion = Accumulo.getAccumuloPersistentVersion(fs);
 +    log.info("Data Version " + dataVersion);
 +    Accumulo.waitForZookeeperAndHdfs(fs);
 +    
 +    Version codeVersion = new Version(Constants.VERSION);
 +    if (dataVersion != ServerConstants.DATA_VERSION && dataVersion != ServerConstants.PREV_DATA_VERSION) {
 +      throw new RuntimeException("This version of accumulo (" + codeVersion + ") is not compatible with files stored using data version " + dataVersion);
 +    }
 +    
 +    TreeMap<String,String> sortedProps = new TreeMap<String,String>();
 +    for (Entry<String,String> entry : config.getConfiguration())
 +      sortedProps.put(entry.getKey(), entry.getValue());
 +    
 +    for (Entry<String,String> entry : sortedProps.entrySet()) {
 +      String key = entry.getKey();
 +      log.info(key + " = " + (Property.isSensitive(key) ? "<hidden>" : entry.getValue()));
 +    }
 +    
 +    monitorSwappiness();
 +  }
 +  
 +  /**
 +   * 
 +   */
 +  public static void monitorSwappiness() {
 +    SimpleTimer.getInstance().schedule(new Runnable() {
 +      @Override
 +      public void run() {
 +        try {
 +          String procFile = "/proc/sys/vm/swappiness";
 +          File swappiness = new File(procFile);
 +          if (swappiness.exists() && swappiness.canRead()) {
 +            InputStream is = new FileInputStream(procFile);
 +            try {
 +              byte[] buffer = new byte[10];
 +              int bytes = is.read(buffer);
 +              String setting = new String(buffer, 0, bytes, Constants.UTF8);
 +              setting = setting.trim();
 +              if (bytes > 0 && Integer.parseInt(setting) > 10) {
 +                log.warn("System swappiness setting is greater than ten (" + setting + ") which can cause time-sensitive operations to be delayed. "
 +                    + " Accumulo is time sensitive because it needs to maintain distributed lock agreement.");
 +              }
 +            } finally {
 +              is.close();
 +            }
 +          }
 +        } catch (Throwable t) {
 +          log.error(t, t);
 +        }
 +      }
 +    }, 1000, 10 * 60 * 1000);
 +  }
 +  
 +  public static void waitForZookeeperAndHdfs(VolumeManager fs) {
 +    log.info("Attempting to talk to zookeeper");
 +    while (true) {
 +      try {
 +        ZooReaderWriter.getInstance().getChildren(Constants.ZROOT);
 +        break;
 +      } catch (InterruptedException e) {
 +        // ignored
 +      } catch (KeeperException ex) {
 +        log.info("Waiting for accumulo to be initialized");
 +        UtilWaitThread.sleep(1000);
 +      }
 +    }
 +    log.info("Zookeeper connected and initialized, attemping to talk to HDFS");
 +    long sleep = 1000;
 +    int unknownHostTries = 3;
 +    while (true) {
 +      try {
 +        if (fs.isReady())
 +          break;
 +        log.warn("Waiting for the NameNode to leave safemode");
 +      } catch (IOException ex) {
 +        log.warn("Unable to connect to HDFS", ex);
 +      } catch (IllegalArgumentException exception) {
 +        /* Unwrap the UnknownHostException so we can deal with it directly */
 +        if (exception.getCause() instanceof UnknownHostException) {
 +          if (unknownHostTries > 0) {
 +            log.warn("Unable to connect to HDFS, will retry. cause: " + exception.getCause());
 +            /* We need to make sure our sleep period is long enough to avoid getting a cached failure of the host lookup. */
 +            sleep = Math.max(sleep, (AddressUtil.getAddressCacheNegativeTtl((UnknownHostException)(exception.getCause()))+1)*1000);
 +          } else {
 +            log.error("Unable to connect to HDFS and have exceeded max number of retries.", exception);
 +            throw exception;
 +          }
 +          unknownHostTries--;
 +        } else {
 +          throw exception;
 +        }
 +      }
 +      log.info("Backing off due to failure; current sleep period is " + sleep / 1000. + " seconds");
 +      UtilWaitThread.sleep(sleep);
 +      /* Back off to give transient failures more time to clear. */
 +      sleep = Math.min(60 * 1000, sleep * 2);
 +    }
 +    log.info("Connected to HDFS");
 +  }
-   
++
++  /**
++   * Exit loudly if there are outstanding Fate operations.
++   * Since Fate serializes class names, we need to make sure there are no queued
++   * transactions from a previous version before continuing an upgrade. The status of the operations is
++   * irrelevant; those in SUCCESSFUL status cause the same problem as those just queued.
++   *
++   * Note that the Master should not allow write access to Fate until after all upgrade steps are complete.
++   *
++   * Should be called as a guard before performing any upgrade steps, after determining that an upgrade is needed.
++   *
++   * see ACCUMULO-2519
++   */
++  public static void abortIfFateTransactions() {
++    try {
++      final ReadOnlyTStore<Accumulo> fate = new ReadOnlyStore<Accumulo>(new ZooStore<Accumulo>(ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZFATE,
++          ZooReaderWriter.getRetryingInstance()));
++      if (!(fate.list().isEmpty())) {
++        throw new AccumuloException("Aborting upgrade because there are outstanding FATE transactions from a previous Accumulo version. Please see the README document for instructions on what to do under your previous version.");
++      }
++    } catch (Exception exception) {
++      log.fatal("Problem verifying Fate readiness", exception);
++      System.exit(1);
++    }
++  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/957c9d1b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
----------------------------------------------------------------------
diff --cc server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
index 760d57f,0000000..374017d
mode 100644,000000..100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
@@@ -1,1018 -1,0 +1,1021 @@@
 +/*
 + * 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.server.util;
 +
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.Collections;
 +import java.util.Comparator;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +import java.util.SortedMap;
 +import java.util.TreeMap;
 +import java.util.concurrent.TimeUnit;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.BatchWriter;
 +import org.apache.accumulo.core.client.BatchWriterConfig;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.IsolatedScanner;
 +import org.apache.accumulo.core.client.MutationsRejectedException;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.client.impl.BatchWriterImpl;
 +import org.apache.accumulo.core.client.impl.ScannerImpl;
 +import org.apache.accumulo.core.client.impl.Writer;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.KeyExtent;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.PartialKey;
 +import org.apache.accumulo.core.data.Range;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.metadata.MetadataTable;
 +import org.apache.accumulo.core.metadata.RootTable;
 +import org.apache.accumulo.core.metadata.schema.DataFileValue;
 +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.ChoppedColumnFamily;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ClonedColumnFamily;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.core.security.Credentials;
 +import org.apache.accumulo.core.tabletserver.log.LogEntry;
 +import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 +import org.apache.accumulo.core.util.ColumnFQ;
 +import org.apache.accumulo.core.util.FastFormat;
 +import org.apache.accumulo.core.util.Pair;
 +import org.apache.accumulo.core.util.UtilWaitThread;
 +import org.apache.accumulo.core.zookeeper.ZooUtil;
 +import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 +import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 +import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 +import org.apache.accumulo.server.ServerConstants;
 +import org.apache.accumulo.server.client.HdfsZooInstance;
 +import org.apache.accumulo.server.fs.FileRef;
 +import org.apache.accumulo.server.fs.VolumeManager;
 +import org.apache.accumulo.server.fs.VolumeManagerImpl;
 +import org.apache.accumulo.server.security.SystemCredentials;
 +import org.apache.accumulo.server.zookeeper.ZooLock;
 +import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 +import org.apache.hadoop.fs.FileStatus;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.io.Text;
 +import org.apache.log4j.Logger;
 +import org.apache.zookeeper.KeeperException;
 +
 +/**
 + * provides a reference to the metadata table for updates by tablet servers
 + */
 +public class MetadataTableUtil {
 +
 +  private static final Text EMPTY_TEXT = new Text();
 +  private static Map<Credentials,Writer> root_tables = new HashMap<Credentials,Writer>();
 +  private static Map<Credentials,Writer> metadata_tables = new HashMap<Credentials,Writer>();
 +  private static final Logger log = Logger.getLogger(MetadataTableUtil.class);
 +
 +  private MetadataTableUtil() {}
 +
 +  public synchronized static Writer getMetadataTable(Credentials credentials) {
 +    Writer metadataTable = metadata_tables.get(credentials);
 +    if (metadataTable == null) {
 +      metadataTable = new Writer(HdfsZooInstance.getInstance(), credentials, MetadataTable.ID);
 +      metadata_tables.put(credentials, metadataTable);
 +    }
 +    return metadataTable;
 +  }
 +
 +  private synchronized static Writer getRootTable(Credentials credentials) {
 +    Writer rootTable = root_tables.get(credentials);
 +    if (rootTable == null) {
 +      rootTable = new Writer(HdfsZooInstance.getInstance(), credentials, RootTable.ID);
 +      root_tables.put(credentials, rootTable);
 +    }
 +    return rootTable;
 +  }
 +
 +  private static void putLockID(ZooLock zooLock, Mutation m) {
 +    TabletsSection.ServerColumnFamily.LOCK_COLUMN.put(m, new Value(zooLock.getLockID().serialize(ZooUtil.getRoot(HdfsZooInstance.getInstance()) + "/")
 +        .getBytes(Constants.UTF8)));
 +  }
 +
 +  private static void update(Credentials credentials, Mutation m, KeyExtent extent) {
 +    update(credentials, null, m, extent);
 +  }
 +
 +  public static void update(Credentials credentials, ZooLock zooLock, Mutation m, KeyExtent extent) {
 +    Writer t = extent.isMeta() ? getRootTable(credentials) : getMetadataTable(credentials);
 +    if (zooLock != null)
 +      putLockID(zooLock, m);
 +    while (true) {
 +      try {
 +        t.update(m);
 +        return;
 +      } catch (AccumuloException e) {
 +        log.error(e, e);
 +      } catch (AccumuloSecurityException e) {
 +        log.error(e, e);
 +      } catch (ConstraintViolationException e) {
 +        log.error(e, e);
 +      } catch (TableNotFoundException e) {
 +        log.error(e, e);
 +      }
 +      UtilWaitThread.sleep(1000);
 +    }
 +
 +  }
 +
 +  public static void updateTabletFlushID(KeyExtent extent, long flushID, Credentials credentials, ZooLock zooLock) {
 +    if (!extent.isRootTablet()) {
 +      Mutation m = new Mutation(extent.getMetadataEntry());
 +      TabletsSection.ServerColumnFamily.FLUSH_COLUMN.put(m, new Value((flushID + "").getBytes(Constants.UTF8)));
 +      update(credentials, zooLock, m, extent);
 +    }
 +  }
 +
 +  public static void updateTabletCompactID(KeyExtent extent, long compactID, Credentials credentials, ZooLock zooLock) {
 +    if (!extent.isRootTablet()) {
 +      Mutation m = new Mutation(extent.getMetadataEntry());
 +      TabletsSection.ServerColumnFamily.COMPACT_COLUMN.put(m, new Value((compactID + "").getBytes(Constants.UTF8)));
 +      update(credentials, zooLock, m, extent);
 +    }
 +  }
 +
 +  public static void updateTabletDataFile(long tid, KeyExtent extent, Map<FileRef,DataFileValue> estSizes, String time, Credentials credentials, ZooLock zooLock) {
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +    byte[] tidBytes = Long.toString(tid).getBytes(Constants.UTF8);
 +
 +    for (Entry<FileRef,DataFileValue> entry : estSizes.entrySet()) {
 +      Text file = entry.getKey().meta();
 +      m.put(DataFileColumnFamily.NAME, file, new Value(entry.getValue().encode()));
 +      m.put(TabletsSection.BulkFileColumnFamily.NAME, file, new Value(tidBytes));
 +    }
 +    TabletsSection.ServerColumnFamily.TIME_COLUMN.put(m, new Value(time.getBytes(Constants.UTF8)));
 +    update(credentials, zooLock, m, extent);
 +  }
 +
 +  public static void updateTabletDir(KeyExtent extent, String newDir, Credentials creds, ZooLock lock) {
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +    TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(m, new Value(newDir.getBytes(Constants.UTF8)));
 +    update(creds, lock, m, extent);
 +  }
 +
 +  public static void addTablet(KeyExtent extent, String path, Credentials credentials, char timeType, ZooLock lock) {
 +    Mutation m = extent.getPrevRowUpdateMutation();
 +
 +    TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(m, new Value(path.getBytes(Constants.UTF8)));
 +    TabletsSection.ServerColumnFamily.TIME_COLUMN.put(m, new Value((timeType + "0").getBytes(Constants.UTF8)));
 +
 +    update(credentials, lock, m, extent);
 +  }
 +
 +  public static void updateTabletPrevEndRow(KeyExtent extent, Credentials credentials) {
 +    Mutation m = extent.getPrevRowUpdateMutation(); //
 +    update(credentials, m, extent);
 +  }
 +
 +  public static void updateTabletVolumes(KeyExtent extent, List<LogEntry> logsToRemove, List<LogEntry> logsToAdd, List<FileRef> filesToRemove,
 +      SortedMap<FileRef,DataFileValue> filesToAdd, String newDir, ZooLock zooLock, Credentials credentials) {
 +
 +    if (extent.isRootTablet()) {
 +      if (newDir != null)
 +        throw new IllegalArgumentException("newDir not expected for " + extent);
 +
 +      if (filesToRemove.size() != 0 || filesToAdd.size() != 0)
 +        throw new IllegalArgumentException("files not expected for " + extent);
 +
 +      // add before removing in case of process death
 +      for (LogEntry logEntry : logsToAdd)
 +        addLogEntry(credentials, logEntry, zooLock);
 +
 +      removeUnusedWALEntries(extent, logsToRemove, zooLock);
 +    } else {
 +      Mutation m = new Mutation(extent.getMetadataEntry());
 +
 +      for (LogEntry logEntry : logsToRemove)
 +        m.putDelete(logEntry.getColumnFamily(), logEntry.getColumnQualifier());
 +
 +      for (LogEntry logEntry : logsToAdd)
 +        m.put(logEntry.getColumnFamily(), logEntry.getColumnQualifier(), logEntry.getValue());
 +
 +      for (FileRef fileRef : filesToRemove)
 +        m.putDelete(DataFileColumnFamily.NAME, fileRef.meta());
 +
 +      for (Entry<FileRef,DataFileValue> entry : filesToAdd.entrySet())
 +        m.put(DataFileColumnFamily.NAME, entry.getKey().meta(), new Value(entry.getValue().encode()));
 +
 +      if (newDir != null)
 +        ServerColumnFamily.DIRECTORY_COLUMN.put(m, new Value(newDir.getBytes(Constants.UTF8)));
 +
 +      update(credentials, m, extent);
 +    }
 +  }
 +
 +  public static SortedMap<FileRef,DataFileValue> getDataFileSizes(KeyExtent extent, Credentials credentials) throws IOException {
 +    TreeMap<FileRef,DataFileValue> sizes = new TreeMap<FileRef,DataFileValue>();
 +
 +    Scanner mdScanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, MetadataTable.ID, Authorizations.EMPTY);
 +    mdScanner.fetchColumnFamily(DataFileColumnFamily.NAME);
 +    Text row = extent.getMetadataEntry();
 +    VolumeManager fs = VolumeManagerImpl.get();
 +
 +    Key endKey = new Key(row, DataFileColumnFamily.NAME, new Text(""));
 +    endKey = endKey.followingKey(PartialKey.ROW_COLFAM);
 +
 +    mdScanner.setRange(new Range(new Key(row), endKey));
 +    for (Entry<Key,Value> entry : mdScanner) {
 +
 +      if (!entry.getKey().getRow().equals(row))
 +        break;
 +      DataFileValue dfv = new DataFileValue(entry.getValue().get());
 +      sizes.put(new FileRef(fs, entry.getKey()), dfv);
 +    }
 +
 +    return sizes;
 +  }
 +
 +  public static void rollBackSplit(Text metadataEntry, Text oldPrevEndRow, Credentials credentials, ZooLock zooLock) {
 +    KeyExtent ke = new KeyExtent(metadataEntry, oldPrevEndRow);
 +    Mutation m = ke.getPrevRowUpdateMutation();
 +    TabletsSection.TabletColumnFamily.SPLIT_RATIO_COLUMN.putDelete(m);
 +    TabletsSection.TabletColumnFamily.OLD_PREV_ROW_COLUMN.putDelete(m);
 +    update(credentials, zooLock, m, new KeyExtent(metadataEntry, (Text) null));
 +  }
 +
 +  public static void splitTablet(KeyExtent extent, Text oldPrevEndRow, double splitRatio, Credentials credentials, ZooLock zooLock) {
 +    Mutation m = extent.getPrevRowUpdateMutation(); //
 +
 +    TabletsSection.TabletColumnFamily.SPLIT_RATIO_COLUMN.put(m, new Value(Double.toString(splitRatio).getBytes(Constants.UTF8)));
 +
 +    TabletsSection.TabletColumnFamily.OLD_PREV_ROW_COLUMN.put(m, KeyExtent.encodePrevEndRow(oldPrevEndRow));
 +    ChoppedColumnFamily.CHOPPED_COLUMN.putDelete(m);
 +    update(credentials, zooLock, m, extent);
 +  }
 +
 +  public static void finishSplit(Text metadataEntry, Map<FileRef,DataFileValue> datafileSizes, List<FileRef> highDatafilesToRemove, Credentials credentials,
 +      ZooLock zooLock) {
 +    Mutation m = new Mutation(metadataEntry);
 +    TabletsSection.TabletColumnFamily.SPLIT_RATIO_COLUMN.putDelete(m);
 +    TabletsSection.TabletColumnFamily.OLD_PREV_ROW_COLUMN.putDelete(m);
 +    ChoppedColumnFamily.CHOPPED_COLUMN.putDelete(m);
 +
 +    for (Entry<FileRef,DataFileValue> entry : datafileSizes.entrySet()) {
 +      m.put(DataFileColumnFamily.NAME, entry.getKey().meta(), new Value(entry.getValue().encode()));
 +    }
 +
 +    for (FileRef pathToRemove : highDatafilesToRemove) {
 +      m.putDelete(DataFileColumnFamily.NAME, pathToRemove.meta());
 +    }
 +
 +    update(credentials, zooLock, m, new KeyExtent(metadataEntry, (Text) null));
 +  }
 +
 +  public static void finishSplit(KeyExtent extent, Map<FileRef,DataFileValue> datafileSizes, List<FileRef> highDatafilesToRemove, Credentials credentials,
 +      ZooLock zooLock) {
 +    finishSplit(extent.getMetadataEntry(), datafileSizes, highDatafilesToRemove, credentials, zooLock);
 +  }
 +
 +  public static void addDeleteEntries(KeyExtent extent, Set<FileRef> datafilesToDelete, Credentials credentials) throws IOException {
 +
 +    String tableId = extent.getTableId().toString();
 +
 +    // TODO could use batch writer,would need to handle failure and retry like update does - ACCUMULO-1294
 +    for (FileRef pathToRemove : datafilesToDelete) {
 +      update(credentials, createDeleteMutation(tableId, pathToRemove.path().toString()), extent);
 +    }
 +  }
 +
 +  public static void addDeleteEntry(String tableId, String path) throws IOException {
 +    update(SystemCredentials.get(), createDeleteMutation(tableId, path), new KeyExtent(new Text(tableId), null, null));
 +  }
 +
 +  public static Mutation createDeleteMutation(String tableId, String pathToRemove) throws IOException {
 +    Path path = VolumeManagerImpl.get().getFullPath(tableId, pathToRemove);
 +    Mutation delFlag = new Mutation(new Text(MetadataSchema.DeletesSection.getRowPrefix() + path.toString()));
 +    delFlag.put(EMPTY_TEXT, EMPTY_TEXT, new Value(new byte[] {}));
 +    return delFlag;
 +  }
 +
 +  public static void removeScanFiles(KeyExtent extent, Set<FileRef> scanFiles, Credentials credentials, ZooLock zooLock) {
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +
 +    for (FileRef pathToRemove : scanFiles)
 +      m.putDelete(ScanFileColumnFamily.NAME, pathToRemove.meta());
 +
 +    update(credentials, zooLock, m, extent);
 +  }
 +
 +  public static void splitDatafiles(Text table, Text midRow, double splitRatio, Map<FileRef,FileUtil.FileInfo> firstAndLastRows,
 +      SortedMap<FileRef,DataFileValue> datafiles, SortedMap<FileRef,DataFileValue> lowDatafileSizes, SortedMap<FileRef,DataFileValue> highDatafileSizes,
 +      List<FileRef> highDatafilesToRemove) {
 +
 +    for (Entry<FileRef,DataFileValue> entry : datafiles.entrySet()) {
 +
 +      Text firstRow = null;
 +      Text lastRow = null;
 +
 +      boolean rowsKnown = false;
 +
 +      FileUtil.FileInfo mfi = firstAndLastRows.get(entry.getKey());
 +
 +      if (mfi != null) {
 +        firstRow = mfi.getFirstRow();
 +        lastRow = mfi.getLastRow();
 +        rowsKnown = true;
 +      }
 +
 +      if (rowsKnown && firstRow.compareTo(midRow) > 0) {
 +        // only in high
 +        long highSize = entry.getValue().getSize();
 +        long highEntries = entry.getValue().getNumEntries();
 +        highDatafileSizes.put(entry.getKey(), new DataFileValue(highSize, highEntries, entry.getValue().getTime()));
 +      } else if (rowsKnown && lastRow.compareTo(midRow) <= 0) {
 +        // only in low
 +        long lowSize = entry.getValue().getSize();
 +        long lowEntries = entry.getValue().getNumEntries();
 +        lowDatafileSizes.put(entry.getKey(), new DataFileValue(lowSize, lowEntries, entry.getValue().getTime()));
 +
 +        highDatafilesToRemove.add(entry.getKey());
 +      } else {
 +        long lowSize = (long) Math.floor((entry.getValue().getSize() * splitRatio));
 +        long lowEntries = (long) Math.floor((entry.getValue().getNumEntries() * splitRatio));
 +        lowDatafileSizes.put(entry.getKey(), new DataFileValue(lowSize, lowEntries, entry.getValue().getTime()));
 +
 +        long highSize = (long) Math.ceil((entry.getValue().getSize() * (1.0 - splitRatio)));
 +        long highEntries = (long) Math.ceil((entry.getValue().getNumEntries() * (1.0 - splitRatio)));
 +        highDatafileSizes.put(entry.getKey(), new DataFileValue(highSize, highEntries, entry.getValue().getTime()));
 +      }
 +    }
 +  }
 +
 +  public static void deleteTable(String tableId, boolean insertDeletes, Credentials credentials, ZooLock lock) throws AccumuloException, IOException {
 +    Scanner ms = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, MetadataTable.ID, Authorizations.EMPTY);
 +    Text tableIdText = new Text(tableId);
 +    BatchWriter bw = new BatchWriterImpl(HdfsZooInstance.getInstance(), credentials, MetadataTable.ID, new BatchWriterConfig().setMaxMemory(1000000)
 +        .setMaxLatency(120000l, TimeUnit.MILLISECONDS).setMaxWriteThreads(2));
 +
 +    // scan metadata for our table and delete everything we find
 +    Mutation m = null;
 +    ms.setRange(new KeyExtent(tableIdText, null, null).toMetadataRange());
 +
 +    // insert deletes before deleting data from metadata... this makes the code fault tolerant
 +    if (insertDeletes) {
 +
 +      ms.fetchColumnFamily(DataFileColumnFamily.NAME);
 +      TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.fetch(ms);
 +
 +      for (Entry<Key,Value> cell : ms) {
 +        Key key = cell.getKey();
 +
 +        if (key.getColumnFamily().equals(DataFileColumnFamily.NAME)) {
 +          FileRef ref = new FileRef(VolumeManagerImpl.get(), key);
 +          bw.addMutation(createDeleteMutation(tableId, ref.meta().toString()));
 +        }
 +
 +        if (TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.hasColumns(key)) {
 +          bw.addMutation(createDeleteMutation(tableId, cell.getValue().toString()));
 +        }
 +      }
 +
 +      bw.flush();
 +
 +      ms.clearColumns();
 +    }
 +
 +    for (Entry<Key,Value> cell : ms) {
 +      Key key = cell.getKey();
 +
 +      if (m == null) {
 +        m = new Mutation(key.getRow());
 +        if (lock != null)
 +          putLockID(lock, m);
 +      }
 +
 +      if (key.getRow().compareTo(m.getRow(), 0, m.getRow().length) != 0) {
 +        bw.addMutation(m);
 +        m = new Mutation(key.getRow());
 +        if (lock != null)
 +          putLockID(lock, m);
 +      }
 +      m.putDelete(key.getColumnFamily(), key.getColumnQualifier());
 +    }
 +
 +    if (m != null)
 +      bw.addMutation(m);
 +
 +    bw.close();
 +  }
 +
 +  static String getZookeeperLogLocation() {
 +    return ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_WALOGS;
 +  }
 +
 +  public static void addLogEntry(Credentials credentials, LogEntry entry, ZooLock zooLock) {
 +    if (entry.extent.isRootTablet()) {
 +      String root = getZookeeperLogLocation();
 +      while (true) {
 +        try {
 +          IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +          if (zoo.isLockHeld(zooLock.getLockID())) {
 +            String[] parts = entry.filename.split("/");
 +            String uniqueId = parts[parts.length - 1];
 +            zoo.putPersistentData(root + "/" + uniqueId, entry.toBytes(), NodeExistsPolicy.OVERWRITE);
 +          }
 +          break;
 +        } catch (KeeperException e) {
 +          log.error(e, e);
 +        } catch (InterruptedException e) {
 +          log.error(e, e);
 +        } catch (IOException e) {
 +          log.error(e, e);
 +        }
 +        UtilWaitThread.sleep(1000);
 +      }
 +    } else {
 +      Mutation m = new Mutation(entry.getRow());
 +      m.put(entry.getColumnFamily(), entry.getColumnQualifier(), entry.getValue());
 +      update(credentials, zooLock, m, entry.extent);
 +    }
 +  }
 +
 +  public static void setRootTabletDir(String dir) throws IOException {
 +    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +    String zpath = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_PATH;
 +    try {
 +      zoo.putPersistentData(zpath, dir.getBytes(Constants.UTF8), -1, NodeExistsPolicy.OVERWRITE);
 +    } catch (KeeperException e) {
 +      throw new IOException(e);
 +    } catch (InterruptedException e) {
 +      Thread.currentThread().interrupt();
 +      throw new IOException(e);
 +    }
 +  }
 +
 +  public static String getRootTabletDir() throws IOException {
 +    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +    String zpath = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_PATH;
 +    try {
 +      return new String(zoo.getData(zpath, null), Constants.UTF8);
 +    } catch (KeeperException e) {
 +      throw new IOException(e);
 +    } catch (InterruptedException e) {
 +      Thread.currentThread().interrupt();
 +      throw new IOException(e);
 +    }
 +  }
 +
 +  public static Pair<List<LogEntry>,SortedMap<FileRef,DataFileValue>> getFileAndLogEntries(Credentials credentials, KeyExtent extent) throws KeeperException,
 +      InterruptedException, IOException {
 +    ArrayList<LogEntry> result = new ArrayList<LogEntry>();
 +    TreeMap<FileRef,DataFileValue> sizes = new TreeMap<FileRef,DataFileValue>();
 +
 +    VolumeManager fs = VolumeManagerImpl.get();
 +    if (extent.isRootTablet()) {
 +      getRootLogEntries(result);
 +      Path rootDir = new Path(getRootTabletDir());
 +      FileStatus[] files = fs.listStatus(rootDir);
 +      for (FileStatus fileStatus : files) {
 +        if (fileStatus.getPath().toString().endsWith("_tmp")) {
 +          continue;
 +        }
 +        DataFileValue dfv = new DataFileValue(0, 0);
 +        sizes.put(new FileRef(fileStatus.getPath().toString(), fileStatus.getPath()), dfv);
 +      }
 +
 +    } else {
 +      String systemTableToCheck = extent.isMeta() ? RootTable.ID : MetadataTable.ID;
 +      Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, systemTableToCheck, Authorizations.EMPTY);
 +      scanner.fetchColumnFamily(LogColumnFamily.NAME);
 +      scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
 +      scanner.setRange(extent.toMetadataRange());
 +
 +      for (Entry<Key,Value> entry : scanner) {
 +        if (!entry.getKey().getRow().equals(extent.getMetadataEntry())) {
 +          throw new RuntimeException("Unexpected row " + entry.getKey().getRow() + " expected " + extent.getMetadataEntry());
 +        }
 +
 +        if (entry.getKey().getColumnFamily().equals(LogColumnFamily.NAME)) {
 +          result.add(LogEntry.fromKeyValue(entry.getKey(), entry.getValue()));
 +        } else if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
 +          DataFileValue dfv = new DataFileValue(entry.getValue().get());
 +          sizes.put(new FileRef(fs, entry.getKey()), dfv);
 +        } else {
 +          throw new RuntimeException("Unexpected col fam " + entry.getKey().getColumnFamily());
 +        }
 +      }
 +    }
 +
 +    return new Pair<List<LogEntry>,SortedMap<FileRef,DataFileValue>>(result, sizes);
 +  }
 +
 +  public static List<LogEntry> getLogEntries(Credentials credentials, KeyExtent extent) throws IOException, KeeperException, InterruptedException {
 +    log.info("Scanning logging entries for " + extent);
 +    ArrayList<LogEntry> result = new ArrayList<LogEntry>();
 +    if (extent.equals(RootTable.EXTENT)) {
 +      log.info("Getting logs for root tablet from zookeeper");
 +      getRootLogEntries(result);
 +    } else {
 +      log.info("Scanning metadata for logs used for tablet " + extent);
 +      Scanner scanner = getTabletLogScanner(credentials, extent);
 +      Text pattern = extent.getMetadataEntry();
 +      for (Entry<Key,Value> entry : scanner) {
 +        Text row = entry.getKey().getRow();
 +        if (entry.getKey().getColumnFamily().equals(LogColumnFamily.NAME)) {
 +          if (row.equals(pattern)) {
 +            result.add(LogEntry.fromKeyValue(entry.getKey(), entry.getValue()));
 +          }
 +        }
 +      }
 +    }
 +
 +    Collections.sort(result, new Comparator<LogEntry>() {
 +      @Override
 +      public int compare(LogEntry o1, LogEntry o2) {
 +        long diff = o1.timestamp - o2.timestamp;
 +        if (diff < 0)
 +          return -1;
 +        if (diff > 0)
 +          return 1;
 +        return 0;
 +      }
 +    });
 +    log.info("Returning logs " + result + " for extent " + extent);
 +    return result;
 +  }
 +
 +  static void getRootLogEntries(ArrayList<LogEntry> result) throws KeeperException, InterruptedException, IOException {
 +    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +    String root = getZookeeperLogLocation();
 +    // there's a little race between getting the children and fetching
 +    // the data. The log can be removed in between.
 +    while (true) {
 +      result.clear();
 +      for (String child : zoo.getChildren(root)) {
 +        LogEntry e = new LogEntry();
 +        try {
 +          e.fromBytes(zoo.getData(root + "/" + child, null));
 +          // upgrade from !0;!0<< -> +r<<
 +          e.extent = RootTable.EXTENT;
 +          result.add(e);
 +        } catch (KeeperException.NoNodeException ex) {
 +          continue;
 +        }
 +      }
 +      break;
 +    }
 +  }
 +
 +  private static Scanner getTabletLogScanner(Credentials credentials, KeyExtent extent) {
 +    String tableId = MetadataTable.ID;
 +    if (extent.isMeta())
 +      tableId = RootTable.ID;
 +    Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, tableId, Authorizations.EMPTY);
 +    scanner.fetchColumnFamily(LogColumnFamily.NAME);
 +    Text start = extent.getMetadataEntry();
 +    Key endKey = new Key(start, LogColumnFamily.NAME);
 +    endKey = endKey.followingKey(PartialKey.ROW_COLFAM);
 +    scanner.setRange(new Range(new Key(start), endKey));
 +    return scanner;
 +  }
 +
 +  private static class LogEntryIterator implements Iterator<LogEntry> {
 +
 +    Iterator<LogEntry> zookeeperEntries = null;
 +    Iterator<LogEntry> rootTableEntries = null;
 +    Iterator<Entry<Key,Value>> metadataEntries = null;
 +
 +    LogEntryIterator(Credentials creds) throws IOException, KeeperException, InterruptedException {
 +      zookeeperEntries = getLogEntries(creds, RootTable.EXTENT).iterator();
 +      rootTableEntries = getLogEntries(creds, new KeyExtent(new Text(MetadataTable.ID), null, null)).iterator();
 +      try {
 +        Scanner scanner = HdfsZooInstance.getInstance().getConnector(creds.getPrincipal(), creds.getToken())
 +            .createScanner(MetadataTable.NAME, Authorizations.EMPTY);
 +        log.info("Setting range to " + MetadataSchema.TabletsSection.getRange());
 +        scanner.setRange(MetadataSchema.TabletsSection.getRange());
 +        scanner.fetchColumnFamily(LogColumnFamily.NAME);
 +        metadataEntries = scanner.iterator();
 +      } catch (Exception ex) {
 +        throw new IOException(ex);
 +      }
 +    }
 +
 +    @Override
 +    public boolean hasNext() {
 +      return zookeeperEntries.hasNext() || rootTableEntries.hasNext() || metadataEntries.hasNext();
 +    }
 +
 +    @Override
 +    public LogEntry next() {
 +      if (zookeeperEntries.hasNext()) {
 +        return zookeeperEntries.next();
 +      }
 +      if (rootTableEntries.hasNext()) {
 +        return rootTableEntries.next();
 +      }
 +      Entry<Key,Value> entry = metadataEntries.next();
 +      return LogEntry.fromKeyValue(entry.getKey(), entry.getValue());
 +    }
 +
 +    @Override
 +    public void remove() {
 +      throw new UnsupportedOperationException();
 +    }
 +  }
 +
 +  public static Iterator<LogEntry> getLogEntries(Credentials creds) throws IOException, KeeperException, InterruptedException {
 +    return new LogEntryIterator(creds);
 +  }
 +
 +  public static void removeUnusedWALEntries(KeyExtent extent, List<LogEntry> logEntries, ZooLock zooLock) {
 +    if (extent.isRootTablet()) {
 +      for (LogEntry entry : logEntries) {
 +        String root = getZookeeperLogLocation();
 +        while (true) {
 +          try {
 +            IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +            if (zoo.isLockHeld(zooLock.getLockID())) {
 +              String parts[] = entry.filename.split("/");
 +              zoo.recursiveDelete(root + "/" + parts[parts.length - 1], NodeMissingPolicy.SKIP);
 +            }
 +            break;
 +          } catch (Exception e) {
 +            log.error(e, e);
 +          }
 +          UtilWaitThread.sleep(1000);
 +        }
 +      }
 +    } else {
 +      Mutation m = new Mutation(extent.getMetadataEntry());
 +      for (LogEntry entry : logEntries) {
 +        m.putDelete(LogColumnFamily.NAME, new Text(entry.getName()));
 +      }
 +      update(SystemCredentials.get(), zooLock, m, extent);
 +    }
 +  }
 +
 +  private static void getFiles(Set<String> files, Map<Key,Value> tablet, String srcTableId) {
 +    for (Entry<Key,Value> entry : tablet.entrySet()) {
 +      if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
 +        String cf = entry.getKey().getColumnQualifier().toString();
 +        if (srcTableId != null && !cf.startsWith("../") && !cf.contains(":")) {
 +          cf = "../" + srcTableId + entry.getKey().getColumnQualifier();
 +        }
 +        files.add(cf);
 +      }
 +    }
 +  }
 +
 +  private static Mutation createCloneMutation(String srcTableId, String tableId, Map<Key,Value> tablet) {
 +
 +    KeyExtent ke = new KeyExtent(tablet.keySet().iterator().next().getRow(), (Text) null);
 +    Mutation m = new Mutation(KeyExtent.getMetadataEntry(new Text(tableId), ke.getEndRow()));
 +
 +    for (Entry<Key,Value> entry : tablet.entrySet()) {
 +      if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
 +        String cf = entry.getKey().getColumnQualifier().toString();
 +        if (!cf.startsWith("../") && !cf.contains(":"))
 +          cf = "../" + srcTableId + entry.getKey().getColumnQualifier();
 +        m.put(entry.getKey().getColumnFamily(), new Text(cf), entry.getValue());
 +      } else if (entry.getKey().getColumnFamily().equals(TabletsSection.CurrentLocationColumnFamily.NAME)) {
 +        m.put(TabletsSection.LastLocationColumnFamily.NAME, entry.getKey().getColumnQualifier(), entry.getValue());
 +      } else if (entry.getKey().getColumnFamily().equals(TabletsSection.LastLocationColumnFamily.NAME)) {
 +        // skip
 +      } else {
 +        m.put(entry.getKey().getColumnFamily(), entry.getKey().getColumnQualifier(), entry.getValue());
 +      }
 +    }
 +    return m;
 +  }
 +
 +  private static Scanner createCloneScanner(String tableId, Connector conn) throws TableNotFoundException {
 +    String tableName = MetadataTable.NAME;
 +    if (tableId.equals(MetadataTable.ID))
 +      tableName = RootTable.NAME;
 +    Scanner mscanner = new IsolatedScanner(conn.createScanner(tableName, Authorizations.EMPTY));
 +    mscanner.setRange(new KeyExtent(new Text(tableId), null, null).toMetadataRange());
 +    mscanner.fetchColumnFamily(DataFileColumnFamily.NAME);
 +    mscanner.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);
 +    mscanner.fetchColumnFamily(TabletsSection.LastLocationColumnFamily.NAME);
 +    mscanner.fetchColumnFamily(ClonedColumnFamily.NAME);
 +    TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(mscanner);
 +    TabletsSection.ServerColumnFamily.TIME_COLUMN.fetch(mscanner);
 +    return mscanner;
 +  }
 +
 +  static void initializeClone(String srcTableId, String tableId, Connector conn, BatchWriter bw) throws TableNotFoundException, MutationsRejectedException {
 +    TabletIterator ti;
 +    if (srcTableId.equals(MetadataTable.ID))
 +      ti = new TabletIterator(createCloneScanner(srcTableId, conn), new Range(), true, true);
 +    else
 +      ti = new TabletIterator(createCloneScanner(srcTableId, conn), new KeyExtent(new Text(srcTableId), null, null).toMetadataRange(), true, true);
 +
 +    if (!ti.hasNext())
 +      throw new RuntimeException(" table deleted during clone?  srcTableId = " + srcTableId);
 +
 +    while (ti.hasNext())
 +      bw.addMutation(createCloneMutation(srcTableId, tableId, ti.next()));
 +
 +    bw.flush();
 +  }
 +
 +  private static int compareEndRows(Text endRow1, Text endRow2) {
 +    return new KeyExtent(new Text("0"), endRow1, null).compareTo(new KeyExtent(new Text("0"), endRow2, null));
 +  }
 +
 +  static int checkClone(String srcTableId, String tableId, Connector conn, BatchWriter bw) throws TableNotFoundException, MutationsRejectedException {
 +    TabletIterator srcIter = new TabletIterator(createCloneScanner(srcTableId, conn), new KeyExtent(new Text(srcTableId), null, null).toMetadataRange(), true,
 +        true);
 +    TabletIterator cloneIter = new TabletIterator(createCloneScanner(tableId, conn), new KeyExtent(new Text(tableId), null, null).toMetadataRange(), true, true);
 +
 +    if (!cloneIter.hasNext() || !srcIter.hasNext())
 +      throw new RuntimeException(" table deleted during clone?  srcTableId = " + srcTableId + " tableId=" + tableId);
 +
 +    int rewrites = 0;
 +
 +    while (cloneIter.hasNext()) {
 +      Map<Key,Value> cloneTablet = cloneIter.next();
 +      Text cloneEndRow = new KeyExtent(cloneTablet.keySet().iterator().next().getRow(), (Text) null).getEndRow();
 +      HashSet<String> cloneFiles = new HashSet<String>();
 +
 +      boolean cloneSuccessful = false;
 +      for (Entry<Key,Value> entry : cloneTablet.entrySet()) {
 +        if (entry.getKey().getColumnFamily().equals(ClonedColumnFamily.NAME)) {
 +          cloneSuccessful = true;
 +          break;
 +        }
 +      }
 +
 +      if (!cloneSuccessful)
 +        getFiles(cloneFiles, cloneTablet, null);
 +
 +      List<Map<Key,Value>> srcTablets = new ArrayList<Map<Key,Value>>();
 +      Map<Key,Value> srcTablet = srcIter.next();
 +      srcTablets.add(srcTablet);
 +
 +      Text srcEndRow = new KeyExtent(srcTablet.keySet().iterator().next().getRow(), (Text) null).getEndRow();
 +
 +      int cmp = compareEndRows(cloneEndRow, srcEndRow);
 +      if (cmp < 0)
 +        throw new TabletIterator.TabletDeletedException("Tablets deleted from src during clone : " + cloneEndRow + " " + srcEndRow);
 +
 +      HashSet<String> srcFiles = new HashSet<String>();
 +      if (!cloneSuccessful)
 +        getFiles(srcFiles, srcTablet, srcTableId);
 +
 +      while (cmp > 0) {
 +        srcTablet = srcIter.next();
 +        srcTablets.add(srcTablet);
 +        srcEndRow = new KeyExtent(srcTablet.keySet().iterator().next().getRow(), (Text) null).getEndRow();
 +        cmp = compareEndRows(cloneEndRow, srcEndRow);
 +        if (cmp < 0)
 +          throw new TabletIterator.TabletDeletedException("Tablets deleted from src during clone : " + cloneEndRow + " " + srcEndRow);
 +
 +        if (!cloneSuccessful)
 +          getFiles(srcFiles, srcTablet, srcTableId);
 +      }
 +
 +      if (cloneSuccessful)
 +        continue;
 +
 +      if (!srcFiles.containsAll(cloneFiles)) {
 +        // delete existing cloned tablet entry
 +        Mutation m = new Mutation(cloneTablet.keySet().iterator().next().getRow());
 +
 +        for (Entry<Key,Value> entry : cloneTablet.entrySet()) {
 +          Key k = entry.getKey();
 +          m.putDelete(k.getColumnFamily(), k.getColumnQualifier(), k.getTimestamp());
 +        }
 +
 +        bw.addMutation(m);
 +
 +        for (Map<Key,Value> st : srcTablets)
 +          bw.addMutation(createCloneMutation(srcTableId, tableId, st));
 +
 +        rewrites++;
 +      } else {
 +        // write out marker that this tablet was successfully cloned
 +        Mutation m = new Mutation(cloneTablet.keySet().iterator().next().getRow());
 +        m.put(ClonedColumnFamily.NAME, new Text(""), new Value("OK".getBytes(Constants.UTF8)));
 +        bw.addMutation(m);
 +      }
 +    }
 +
 +    bw.flush();
 +    return rewrites;
 +  }
 +
 +  public static void cloneTable(Instance instance, String srcTableId, String tableId, VolumeManager volumeManager) throws Exception {
 +
 +    Connector conn = instance.getConnector(SystemCredentials.get().getPrincipal(), SystemCredentials.get().getToken());
 +    BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
 +
 +    while (true) {
 +
 +      try {
 +        initializeClone(srcTableId, tableId, conn, bw);
 +
 +        // the following loop looks changes in the file that occurred during the copy.. if files were dereferenced then they could have been GCed
 +
 +        while (true) {
 +          int rewrites = checkClone(srcTableId, tableId, conn, bw);
 +
 +          if (rewrites == 0)
 +            break;
 +        }
 +
 +        bw.flush();
 +        break;
 +
 +      } catch (TabletIterator.TabletDeletedException tde) {
 +        // tablets were merged in the src table
 +        bw.flush();
 +
 +        // delete what we have cloned and try again
 +        deleteTable(tableId, false, SystemCredentials.get(), null);
 +
 +        log.debug("Tablets merged in table " + srcTableId + " while attempting to clone, trying again");
 +
 +        UtilWaitThread.sleep(100);
 +      }
 +    }
 +
 +    // delete the clone markers and create directory entries
 +    Scanner mscanner = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
 +    mscanner.setRange(new KeyExtent(new Text(tableId), null, null).toMetadataRange());
 +    mscanner.fetchColumnFamily(ClonedColumnFamily.NAME);
 +
 +    int dirCount = 0;
 +
 +    for (Entry<Key,Value> entry : mscanner) {
 +      Key k = entry.getKey();
 +      Mutation m = new Mutation(k.getRow());
 +      m.putDelete(k.getColumnFamily(), k.getColumnQualifier());
 +      String dir = volumeManager.choose(ServerConstants.getTablesDirs()) + "/" + tableId
 +          + new String(FastFormat.toZeroPaddedString(dirCount++, 8, 16, "/c-".getBytes(Constants.UTF8)));
 +      TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(m, new Value(dir.getBytes(Constants.UTF8)));
 +      bw.addMutation(m);
 +    }
 +
 +    bw.close();
 +
 +  }
 +
 +  public static void chopped(KeyExtent extent, ZooLock zooLock) {
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +    ChoppedColumnFamily.CHOPPED_COLUMN.put(m, new Value("chopped".getBytes(Constants.UTF8)));
 +    update(SystemCredentials.get(), zooLock, m, extent);
 +  }
 +
 +  public static void removeBulkLoadEntries(Connector conn, String tableId, long tid) throws Exception {
 +    Scanner mscanner = new IsolatedScanner(conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY));
 +    mscanner.setRange(new KeyExtent(new Text(tableId), null, null).toMetadataRange());
 +    mscanner.fetchColumnFamily(TabletsSection.BulkFileColumnFamily.NAME);
 +    BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
 +    for (Entry<Key,Value> entry : mscanner) {
 +      log.debug("Looking at entry " + entry + " with tid " + tid);
 +      if (Long.parseLong(entry.getValue().toString()) == tid) {
 +        log.debug("deleting entry " + entry);
 +        Mutation m = new Mutation(entry.getKey().getRow());
 +        m.putDelete(entry.getKey().getColumnFamily(), entry.getKey().getColumnQualifier());
 +        bw.addMutation(m);
 +      }
 +    }
 +    bw.close();
 +  }
 +
 +  public static List<FileRef> getBulkFilesLoaded(Connector conn, KeyExtent extent, long tid) throws IOException {
 +    List<FileRef> result = new ArrayList<FileRef>();
 +    try {
 +      VolumeManager fs = VolumeManagerImpl.get();
 +      Scanner mscanner = new IsolatedScanner(conn.createScanner(extent.isMeta() ? RootTable.NAME : MetadataTable.NAME, Authorizations.EMPTY));
 +      mscanner.setRange(extent.toMetadataRange());
 +      mscanner.fetchColumnFamily(TabletsSection.BulkFileColumnFamily.NAME);
 +      for (Entry<Key,Value> entry : mscanner) {
 +        if (Long.parseLong(entry.getValue().toString()) == tid) {
 +          result.add(new FileRef(fs, entry.getKey()));
 +        }
 +      }
 +      return result;
 +    } catch (TableNotFoundException ex) {
 +      // unlikely
 +      throw new RuntimeException("Onos! teh metadata table has vanished!!");
 +    }
 +  }
 +
 +  public static Map<FileRef,Long> getBulkFilesLoaded(Credentials credentials, KeyExtent extent) throws IOException {
 +    Text metadataRow = extent.getMetadataEntry();
 +    Map<FileRef,Long> ret = new HashMap<FileRef,Long>();
 +
 +    VolumeManager fs = VolumeManagerImpl.get();
 +    Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, extent.isMeta() ? RootTable.ID : MetadataTable.ID, Authorizations.EMPTY);
 +    scanner.setRange(new Range(metadataRow));
 +    scanner.fetchColumnFamily(TabletsSection.BulkFileColumnFamily.NAME);
 +    for (Entry<Key,Value> entry : scanner) {
 +      Long tid = Long.parseLong(entry.getValue().toString());
 +      ret.put(new FileRef(fs, entry.getKey()), tid);
 +    }
 +    return ret;
 +  }
 +
 +  public static void addBulkLoadInProgressFlag(String path) {
 +
 +    Mutation m = new Mutation(MetadataSchema.BlipSection.getRowPrefix() + path);
 +    m.put(EMPTY_TEXT, EMPTY_TEXT, new Value(new byte[] {}));
 +
 +    // new KeyExtent is only added to force update to write to the metadata table, not the root table
 +    // because bulk loads aren't supported to the metadata table
 +    update(SystemCredentials.get(), m, new KeyExtent(new Text("anythingNotMetadata"), null, null));
 +  }
 +
 +  public static void removeBulkLoadInProgressFlag(String path) {
 +
 +    Mutation m = new Mutation(MetadataSchema.BlipSection.getRowPrefix() + path);
 +    m.putDelete(EMPTY_TEXT, EMPTY_TEXT);
 +
 +    // new KeyExtent is only added to force update to write to the metadata table, not the root table
 +    // because bulk loads aren't supported to the metadata table
 +    update(SystemCredentials.get(), m, new KeyExtent(new Text("anythingNotMetadata"), null, null));
 +  }
 +
++  /**
++   * During an upgrade we need to move deletion requests for files under the !METADATA table to the root tablet.
++   */
 +  public static void moveMetaDeleteMarkers(Instance instance, Credentials creds) {
 +    // move old delete markers to new location, to standardize table schema between all metadata tables
 +    byte[] EMPTY_BYTES = new byte[0];
 +    Scanner scanner = new ScannerImpl(instance, creds, RootTable.ID, Authorizations.EMPTY);
 +    String oldDeletesPrefix = "!!~del";
 +    Range oldDeletesRange = new Range(oldDeletesPrefix, true, "!!~dem", false);
 +    scanner.setRange(oldDeletesRange);
 +    for (Entry<Key,Value> entry : scanner) {
 +      String row = entry.getKey().getRow().toString();
 +      if (row.startsWith(oldDeletesPrefix)) {
 +        String filename = row.substring(oldDeletesPrefix.length());
 +        // add the new entry first
 +        log.info("Moving " + filename + " marker in " + RootTable.NAME);
 +        Mutation m = new Mutation(MetadataSchema.DeletesSection.getRowPrefix() + filename);
 +        m.put(EMPTY_BYTES, EMPTY_BYTES, EMPTY_BYTES);
 +        update(creds, m, RootTable.EXTENT);
 +        // remove the old entry
 +        m = new Mutation(entry.getKey().getRow());
 +        m.putDelete(EMPTY_BYTES, EMPTY_BYTES);
 +        update(creds, m, RootTable.OLD_EXTENT);
 +      } else {
 +        break;
 +      }
 +    }
 +
 +  }
 +
 +  public static SortedMap<Text,SortedMap<ColumnFQ,Value>> getTabletEntries(SortedMap<Key,Value> tabletKeyValues, List<ColumnFQ> columns) {
 +    TreeMap<Text,SortedMap<ColumnFQ,Value>> tabletEntries = new TreeMap<Text,SortedMap<ColumnFQ,Value>>();
 +
 +    HashSet<ColumnFQ> colSet = null;
 +    if (columns != null) {
 +      colSet = new HashSet<ColumnFQ>(columns);
 +    }
 +
 +    for (Entry<Key,Value> entry : tabletKeyValues.entrySet()) {
 +
 +      if (columns != null && !colSet.contains(new ColumnFQ(entry.getKey()))) {
 +        continue;
 +      }
 +
 +      Text row = entry.getKey().getRow();
 +
 +      SortedMap<ColumnFQ,Value> colVals = tabletEntries.get(row);
 +      if (colVals == null) {
 +        colVals = new TreeMap<ColumnFQ,Value>();
 +        tabletEntries.put(row, colVals);
 +      }
 +
 +      colVals.put(new ColumnFQ(entry.getKey()), entry.getValue());
 +    }
 +
 +    return tabletEntries;
 +  }
 +}


[04/15] git commit: ACCUMULO-2551 adds read only fate operations.

Posted by bu...@apache.org.
ACCUMULO-2551 adds read only fate operations.


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

Branch: refs/heads/master
Commit: a904f69110942bd139e701d617d0aa3b647001f8
Parents: f67c386
Author: Sean Busbey <bu...@cloudera.com>
Authored: Tue Mar 25 17:29:59 2014 -0500
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Apr 4 17:27:05 2014 -0700

----------------------------------------------------------------------
 fate/pom.xml                                    |   9 ++
 .../org/apache/accumulo/fate/AdminUtil.java     |  12 +-
 .../java/org/apache/accumulo/fate/Fate.java     |   2 +-
 .../org/apache/accumulo/fate/ReadOnlyRepo.java  |  32 +++++
 .../org/apache/accumulo/fate/ReadOnlyStore.java | 111 ++++++++++++++++
 .../apache/accumulo/fate/ReadOnlyTStore.java    | 125 +++++++++++++++++++
 .../java/org/apache/accumulo/fate/Repo.java     |   5 +-
 .../java/org/apache/accumulo/fate/TStore.java   |  72 ++---------
 .../apache/accumulo/fate/AgeOffStoreTest.java   |   2 +-
 .../apache/accumulo/fate/ReadOnlyStoreTest.java |  72 +++++++++++
 .../org/apache/accumulo/fate/SimpleStore.java   |   2 +-
 .../org/apache/accumulo/server/fate/Admin.java  |   3 +-
 .../apache/accumulo/server/master/Master.java   |   2 +-
 13 files changed, 370 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/pom.xml
----------------------------------------------------------------------
diff --git a/fate/pom.xml b/fate/pom.xml
index 433e61a..0868e4c 100644
--- a/fate/pom.xml
+++ b/fate/pom.xml
@@ -26,6 +26,10 @@
   <name>Fate</name>
   <dependencies>
     <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
       <groupId>commons-lang</groupId>
       <artifactId>commons-lang</artifactId>
       <scope>provided</scope>
@@ -45,5 +49,10 @@
       <artifactId>junit</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.easymock</groupId>
+      <artifactId>easymock</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java b/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
index 0162466..0238fde 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
@@ -24,7 +24,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
-import org.apache.accumulo.fate.TStore.TStatus;
+import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooLock;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
@@ -36,7 +36,7 @@ import org.apache.zookeeper.KeeperException;
 public class AdminUtil<T> {
   private static final Charset UTF8 = Charset.forName("UTF-8");
   
-  public void print(ZooStore<T> zs, IZooReaderWriter zk, String lockPath) throws KeeperException, InterruptedException {
+  public void print(ReadOnlyTStore<T> zs, IZooReaderWriter zk, String lockPath) throws KeeperException, InterruptedException {
     Map<Long,List<String>> heldLocks = new HashMap<Long,List<String>>();
     Map<Long,List<String>> waitingLocks = new HashMap<Long,List<String>>();
     
@@ -108,7 +108,7 @@ public class AdminUtil<T> {
         wlocks = Collections.emptyList();
       
       String top = null;
-      Repo<T> repo = zs.top(tid);
+      ReadOnlyRepo<T> repo = zs.top(tid);
       if (repo != null)
         top = repo.getDescription();
       
@@ -132,7 +132,7 @@ public class AdminUtil<T> {
     }
   }
   
-  public boolean prepDelete(ZooStore<T> zs, IZooReaderWriter zk, String path, String txidStr) {
+  public boolean prepDelete(TStore<T> zs, IZooReaderWriter zk, String path, String txidStr) {
     if (!checkGlobalLock(zk, path)) {
       return false;
     }
@@ -145,7 +145,7 @@ public class AdminUtil<T> {
     return true;
   }
   
-  public boolean prepFail(ZooStore<T> zs, IZooReaderWriter zk, String path, String txidStr) {
+  public boolean prepFail(TStore<T> zs, IZooReaderWriter zk, String path, String txidStr) {
     if (!checkGlobalLock(zk, path)) {
       return false;
     }
@@ -158,7 +158,7 @@ public class AdminUtil<T> {
     return true;
   }
   
-  public void deleteLocks(ZooStore<T> zs, IZooReaderWriter zk, String path, String txidStr) throws KeeperException, InterruptedException {
+  public void deleteLocks(TStore<T> zs, IZooReaderWriter zk, String path, String txidStr) throws KeeperException, InterruptedException {
     // delete any locks assoc w/ fate operation
     List<String> lockedIds = zk.getChildren(path);
     

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/main/java/org/apache/accumulo/fate/Fate.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/Fate.java b/fate/src/main/java/org/apache/accumulo/fate/Fate.java
index 9d24b0b..b2eb681 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/Fate.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/Fate.java
@@ -18,7 +18,7 @@ package org.apache.accumulo.fate;
 
 import java.util.EnumSet;
 
-import org.apache.accumulo.fate.TStore.TStatus;
+import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
 import org.apache.accumulo.fate.util.Daemon;
 import org.apache.accumulo.fate.util.LoggingRunnable;
 import org.apache.log4j.Logger;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyRepo.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyRepo.java b/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyRepo.java
new file mode 100644
index 0000000..24d00d9
--- /dev/null
+++ b/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyRepo.java
@@ -0,0 +1,32 @@
+/*
+ * 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.fate;
+
+/**
+ * Read only access to a repeatable persisted operation.
+ *
+ * By definition, these methods are safe to call without impacting the state of FATE. They should also be
+ * safe to call without impacting the state of system components.
+ *
+ */
+public interface ReadOnlyRepo<T> {
+
+  long isReady(long tid, T environment) throws Exception;
+
+  String getDescription();
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyStore.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyStore.java b/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyStore.java
new file mode 100644
index 0000000..7cb20ff
--- /dev/null
+++ b/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyStore.java
@@ -0,0 +1,111 @@
+/*
+ * 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.fate;
+
+import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
+import com.google.common.base.Preconditions;
+
+import java.io.Serializable;
+import java.util.EnumSet;
+import java.util.List;
+
+/**
+ * This store decorates a TStore to make sure it can not be modified.
+ *
+ * Unlike relying directly on the ReadOnlyTStore interface, this class will not allow subsequent users to cast back to a
+ * mutable TStore successfully.
+ *
+ */
+public class ReadOnlyStore<T> implements ReadOnlyTStore<T> {
+
+  private final TStore<T> store;
+
+  /**
+   * @param store may not be null
+   */
+  public ReadOnlyStore(TStore<T> store) {
+    Preconditions.checkNotNull(store);
+    this.store = store;
+  }
+
+  @Override
+  public long reserve() {
+    return store.reserve();
+  }
+
+  @Override
+  public void reserve(long tid) {
+    store.reserve(tid);
+  }
+
+  @Override
+  public void unreserve(long tid, long deferTime) {
+    store.unreserve(tid, deferTime);
+  }
+
+  /**
+   * Decorates a Repo to make sure it is treated as a ReadOnlyRepo.
+   *
+   * Similar to ReadOnlyStore, won't allow subsequent user to cast a ReadOnlyRepo back to a mutable Repo.
+   */
+  protected static class ReadOnlyRepoWrapper<X> implements ReadOnlyRepo<X> {
+    private final Repo<X> repo;
+
+    /**
+     * @param repo may not be null
+     */
+    public ReadOnlyRepoWrapper(Repo<X> repo) {
+      Preconditions.checkNotNull(repo);
+      this.repo = repo;
+    }
+
+    @Override
+    public long isReady(long tid, X environment) throws Exception {
+      return repo.isReady(tid, environment);
+    }
+
+    @Override
+    public String getDescription() {
+      return repo.getDescription();
+    }
+  }
+
+  @Override
+  public ReadOnlyRepo<T> top(long tid) {
+    return new ReadOnlyRepoWrapper(store.top(tid));
+  }
+
+  @Override
+  public TStatus getStatus(long tid) {
+    return store.getStatus(tid);
+  }
+
+  @Override
+  public TStatus waitForStatusChange(long tid, EnumSet<TStatus> expected) {
+    return store.waitForStatusChange(tid, expected);
+  }
+
+  @Override
+  public Serializable getProperty(long tid, String prop) {
+    return store.getProperty(tid, prop);
+  }
+
+  @Override
+  public List<Long> list() {
+    return store.list();
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyTStore.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyTStore.java b/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyTStore.java
new file mode 100644
index 0000000..d390139
--- /dev/null
+++ b/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyTStore.java
@@ -0,0 +1,125 @@
+/*
+ * 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.fate;
+
+import java.io.Serializable;
+import java.util.EnumSet;
+import java.util.List;
+
+/**
+ * Read only access to a Transaction Store.
+ *
+ * A transaction consists of a number of operations. Instances of this class may check on the queue of outstanding
+ * transactions but may neither modify them nor create new ones.
+ */
+public interface ReadOnlyTStore<T> {
+
+  /**
+   * Possible operational status codes. Serialized by name within stores.
+   */
+  enum TStatus {
+    /** Unseeded transaction */
+    NEW,
+    /** Transaction is eligible to be executing */
+    IN_PROGRESS,
+    /** Transaction has failed, and is in the process of being rolled back */
+    FAILED_IN_PROGRESS,
+    /** Transaction has failed and has been fully rolled back */
+    FAILED,
+    /** Transaction has succeeded */
+    SUCCESSFUL,
+    /** Unrecognized or unknown transaction state */
+    UNKNOWN
+  }
+
+  /**
+   * Reserve a transaction that is IN_PROGRESS or FAILED_IN_PROGRESS.
+   *
+   * Reserving a transaction id ensures that nothing else in-process interacting via the same instance
+   * will be operating on that transaction id.
+   *
+   * @return a transaction id that is safe to interact with, chosen by the store.
+   */
+  long reserve();
+
+  /**
+   * Reserve the specific tid.
+   *
+   * Reserving a transaction id ensures that nothing else in-process interacting via the same instance
+   * will be operating on that transaction id.
+   *
+   */
+  void reserve(long tid);
+
+  /**
+   * Return the given transaction to the store.
+   *
+   * upon successful return the store now controls the referenced transaction id. caller should no longer interact with it.
+   *
+   * @param tid transaction id, previously reserved.
+   * @param deferTime time in millis to keep this transaction out of the pool used in the {@link #reserve() reserve} method. must be non-negative.
+   */
+  void unreserve(long tid, long deferTime);
+
+
+  /**
+   * Get the current operation for the given transaction id.
+   *
+   * Caller must have already reserved tid.
+   *
+   * @param tid transaction id, previously reserved.
+   * @return a read-only view of the operation
+   */
+  ReadOnlyRepo<T> top(long tid);
+
+  /**
+   * Get the state of a given transaction.
+   *
+   * Caller must have already reserved tid.
+   *
+   * @param tid transaction id, previously reserved.
+   * @return execution status
+   */
+  TStatus getStatus(long tid);
+
+  /**
+   * Wait for the satus of a transaction to change
+   *
+   * @param tid transaction id, need not have been reserved.
+   * @param expected a set of possible statuses we are interested in being notified about. may not be null.
+   * @return execution status.
+   */
+  TStatus waitForStatusChange(long tid, EnumSet<TStatus> expected);
+
+  /**
+   * Retrieve a transaction-specific property.
+   *
+   * Caller must have already reserved tid.
+   *
+   * @param tid transaction id, previously reserved.
+   * @param prop name of property to retrieve.
+   */
+  Serializable getProperty(long tid, String prop);
+
+  /**
+   * list all transaction ids in store.
+   *
+   * @return all outstanding transactions, including those reserved by others.
+   */
+  List<Long> list();
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/main/java/org/apache/accumulo/fate/Repo.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/Repo.java b/fate/src/main/java/org/apache/accumulo/fate/Repo.java
index 8bdca10..b0ebd1a 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/Repo.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/Repo.java
@@ -22,15 +22,12 @@ import java.io.Serializable;
  * Repeatable persisted operation
  * 
  */
-public interface Repo<T> extends Serializable {
-  long isReady(long tid, T environment) throws Exception;
+public interface Repo<T> extends ReadOnlyRepo<T>, Serializable {
   
   Repo<T> call(long tid, T environment) throws Exception;
   
   void undo(long tid, T environment) throws Exception;
   
-  String getDescription();
-  
   // this allows the last fate op to return something to the user
   String getReturn();
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/main/java/org/apache/accumulo/fate/TStore.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/TStore.java b/fate/src/main/java/org/apache/accumulo/fate/TStore.java
index 3554064..5ca24fc 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/TStore.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/TStore.java
@@ -27,45 +27,14 @@ import java.util.List;
  * service can then execute the transaction's operation, possibly pushing more operations onto the transaction as each step successfully completes. If a step
  * fails, the stack can be unwound, undoing each operation.
  */
-public interface TStore<T> {
-  
-  public enum TStatus {
-    /** Unseeded transaction */
-    NEW,
-    /** Transaction is eligible to be executing */
-    IN_PROGRESS,
-    /** Transaction has failed, and is in the process of being rolled back */
-    FAILED_IN_PROGRESS,
-    /** Transaction has failed and has been fully rolled back */
-    FAILED,
-    /** Transaction has succeeded */
-    SUCCESSFUL,
-    /** Unrecognized or unknown transaction state */
-    UNKNOWN
-  }
+public interface TStore<T> extends ReadOnlyTStore<T> {
   
   /**
    * Create a new transaction id
    * 
    * @return a transaction id
    */
-  public long create();
-  
-  /**
-   * Reserve a transaction that is IN_PROGRESS or FAILED_IN_PROGRESS.
-   * 
-   */
-  long reserve();
-  
-  public void reserve(long tid);
-  
-  /**
-   * Return the given transaction to the store
-   * 
-   * @param tid
-   * @param deferTime
-   */
-  void unreserve(long tid, long deferTime);
+  long create();
   
   /**
    * Get the current operation for the given transaction id.
@@ -74,6 +43,7 @@ public interface TStore<T> {
    *          transaction id
    * @return the operation
    */
+  @Override
   Repo<T> top(long tid);
   
   /**
@@ -84,7 +54,7 @@ public interface TStore<T> {
    * @param repo
    *          the operation
    */
-  public void push(long tid, Repo<T> repo) throws StackOverflowException;
+  void push(long tid, Repo<T> repo) throws StackOverflowException;
   
   /**
    * Remove the last pushed operation from the given transaction.
@@ -94,15 +64,6 @@ public interface TStore<T> {
   void pop(long tid);
   
   /**
-   * Get the state of a given transaction.
-   * 
-   * @param tid
-   *          transaction id
-   * @return execution status
-   */
-  public TStatus getStatus(long tid);
-  
-  /**
    * Update the state of a given transaction
    * 
    * @param tid
@@ -110,19 +71,9 @@ public interface TStore<T> {
    * @param status
    *          execution status
    */
-  public void setStatus(long tid, TStatus status);
-  
-  /**
-   * Wait for the satus of a transaction to change
-   * 
-   * @param tid
-   *          transaction id
-   */
-  public TStatus waitForStatusChange(long tid, EnumSet<TStatus> expected);
-  
-  public void setProperty(long tid, String prop, Serializable val);
+  void setStatus(long tid, TStatus status);
   
-  public Serializable getProperty(long tid, String prop);
+  void setProperty(long tid, String prop, Serializable val);
   
   /**
    * Remove the transaction from the store.
@@ -130,13 +81,6 @@ public interface TStore<T> {
    * @param tid
    *          the transaction id
    */
-  public void delete(long tid);
-  
-  /**
-   * list all transaction ids in store
-   * 
-   */
-  
-  public List<Long> list();
-  
+  void delete(long tid);
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/test/java/org/apache/accumulo/fate/AgeOffStoreTest.java
----------------------------------------------------------------------
diff --git a/fate/src/test/java/org/apache/accumulo/fate/AgeOffStoreTest.java b/fate/src/test/java/org/apache/accumulo/fate/AgeOffStoreTest.java
index c212649..4f5b112 100644
--- a/fate/src/test/java/org/apache/accumulo/fate/AgeOffStoreTest.java
+++ b/fate/src/test/java/org/apache/accumulo/fate/AgeOffStoreTest.java
@@ -20,7 +20,7 @@ import java.util.Arrays;
 import java.util.HashSet;
 
 import org.apache.accumulo.fate.AgeOffStore.TimeSource;
-import org.apache.accumulo.fate.TStore.TStatus;
+import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
 import org.junit.Assert;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/test/java/org/apache/accumulo/fate/ReadOnlyStoreTest.java
----------------------------------------------------------------------
diff --git a/fate/src/test/java/org/apache/accumulo/fate/ReadOnlyStoreTest.java b/fate/src/test/java/org/apache/accumulo/fate/ReadOnlyStoreTest.java
new file mode 100644
index 0000000..c2d5f92
--- /dev/null
+++ b/fate/src/test/java/org/apache/accumulo/fate/ReadOnlyStoreTest.java
@@ -0,0 +1,72 @@
+/*
+ * 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.fate;
+
+import java.util.Collections;
+import java.util.EnumSet;
+
+import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
+
+import org.easymock.EasyMock;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Make sure read only decorate passes read methods.
+ */
+public class ReadOnlyStoreTest {
+
+  @Test
+  public void everythingPassesThrough() throws Exception {
+    @SuppressWarnings("unchecked")
+    Repo<String> repo = EasyMock.createMock(Repo.class);
+    EasyMock.expect(repo.getDescription()).andReturn("description");
+    EasyMock.expect(repo.isReady(0xdeadbeefl, null)).andReturn(0x0l);
+
+    @SuppressWarnings("unchecked")
+    TStore<String> mock = EasyMock.createNiceMock(TStore.class);
+    EasyMock.expect(mock.reserve()).andReturn(0xdeadbeefl);
+    mock.reserve(0xdeadbeefl);
+    EasyMock.expect(mock.top(0xdeadbeefl)).andReturn(repo);
+    EasyMock.expect(mock.getStatus(0xdeadbeefl)).andReturn(TStatus.UNKNOWN);
+    mock.unreserve(0xdeadbeefl, 30);
+
+    EasyMock.expect(mock.waitForStatusChange(0xdeadbeefl, EnumSet.allOf(TStatus.class))).andReturn(TStatus.UNKNOWN);
+    EasyMock.expect(mock.getProperty(0xdeadbeefl, "com.example.anyproperty")).andReturn("property");
+    EasyMock.expect(mock.list()).andReturn(Collections.<Long>emptyList());
+
+    EasyMock.replay(repo);
+    EasyMock.replay(mock);
+
+    ReadOnlyTStore<String> store = new ReadOnlyStore<String>(mock);
+    Assert.assertEquals(0xdeadbeefl, store.reserve());
+    store.reserve(0xdeadbeefl);
+    ReadOnlyRepo<String> top = store.top(0xdeadbeefl);
+    Assert.assertFalse(top instanceof Repo);
+    Assert.assertEquals("description", top.getDescription());
+    Assert.assertEquals(0x0l, top.isReady(0xdeadbeefl, null));
+    Assert.assertEquals(TStatus.UNKNOWN, store.getStatus(0xdeadbeefl));
+    store.unreserve(0xdeadbeefl, 30);
+
+    Assert.assertEquals(TStatus.UNKNOWN, store.waitForStatusChange(0xdeadbeefl, EnumSet.allOf(TStatus.class)));
+    Assert.assertEquals("property", store.getProperty(0xdeadbeefl, "com.example.anyproperty"));
+    Assert.assertEquals(Collections.<Long>emptyList(), store.list());
+
+    EasyMock.verify(repo);
+    EasyMock.verify(mock);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/test/java/org/apache/accumulo/fate/SimpleStore.java
----------------------------------------------------------------------
diff --git a/fate/src/test/java/org/apache/accumulo/fate/SimpleStore.java b/fate/src/test/java/org/apache/accumulo/fate/SimpleStore.java
index 3b78131..60eabfb 100644
--- a/fate/src/test/java/org/apache/accumulo/fate/SimpleStore.java
+++ b/fate/src/test/java/org/apache/accumulo/fate/SimpleStore.java
@@ -28,7 +28,7 @@ import java.util.Set;
 import org.apache.commons.lang.NotImplementedException;
 
 /**
- * 
+ * Transient in memory store for transactions.
  */
 public class SimpleStore<T> implements TStore<T> {
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/server/src/main/java/org/apache/accumulo/server/fate/Admin.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/fate/Admin.java b/server/src/main/java/org/apache/accumulo/server/fate/Admin.java
index 4a5f0bc..fc9e342 100644
--- a/server/src/main/java/org/apache/accumulo/server/fate/Admin.java
+++ b/server/src/main/java/org/apache/accumulo/server/fate/Admin.java
@@ -25,6 +25,7 @@ import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.AdminUtil;
 import org.apache.accumulo.fate.ZooStore;
+import org.apache.accumulo.fate.ReadOnlyStore;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.master.Master;
@@ -88,7 +89,7 @@ public class Admin {
       }
       admin.deleteLocks(zs, zk, ZooUtil.getRoot(instance) + Constants.ZTABLE_LOCKS, args[1]);
     } else if (jc.getParsedCommand().equals("print")) {
-      admin.print(zs, zk, ZooUtil.getRoot(instance) + Constants.ZTABLE_LOCKS);
+      admin.print(new ReadOnlyStore(zs), zk, ZooUtil.getRoot(instance) + Constants.ZTABLE_LOCKS);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/server/src/main/java/org/apache/accumulo/server/master/Master.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/Master.java b/server/src/main/java/org/apache/accumulo/server/master/Master.java
index 8c4c864..270eb18 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/Master.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/Master.java
@@ -90,7 +90,7 @@ 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.ReadOnlyTStore.TStatus;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooLock.LockLossReason;
 import org.apache.accumulo.fate.zookeeper.ZooReaderWriter.Mutator;


[03/15] git commit: ACCUMULO-2551 adds read only fate operations.

Posted by bu...@apache.org.
ACCUMULO-2551 adds read only fate operations.


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

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: a904f69110942bd139e701d617d0aa3b647001f8
Parents: f67c386
Author: Sean Busbey <bu...@cloudera.com>
Authored: Tue Mar 25 17:29:59 2014 -0500
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Apr 4 17:27:05 2014 -0700

----------------------------------------------------------------------
 fate/pom.xml                                    |   9 ++
 .../org/apache/accumulo/fate/AdminUtil.java     |  12 +-
 .../java/org/apache/accumulo/fate/Fate.java     |   2 +-
 .../org/apache/accumulo/fate/ReadOnlyRepo.java  |  32 +++++
 .../org/apache/accumulo/fate/ReadOnlyStore.java | 111 ++++++++++++++++
 .../apache/accumulo/fate/ReadOnlyTStore.java    | 125 +++++++++++++++++++
 .../java/org/apache/accumulo/fate/Repo.java     |   5 +-
 .../java/org/apache/accumulo/fate/TStore.java   |  72 ++---------
 .../apache/accumulo/fate/AgeOffStoreTest.java   |   2 +-
 .../apache/accumulo/fate/ReadOnlyStoreTest.java |  72 +++++++++++
 .../org/apache/accumulo/fate/SimpleStore.java   |   2 +-
 .../org/apache/accumulo/server/fate/Admin.java  |   3 +-
 .../apache/accumulo/server/master/Master.java   |   2 +-
 13 files changed, 370 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/pom.xml
----------------------------------------------------------------------
diff --git a/fate/pom.xml b/fate/pom.xml
index 433e61a..0868e4c 100644
--- a/fate/pom.xml
+++ b/fate/pom.xml
@@ -26,6 +26,10 @@
   <name>Fate</name>
   <dependencies>
     <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
       <groupId>commons-lang</groupId>
       <artifactId>commons-lang</artifactId>
       <scope>provided</scope>
@@ -45,5 +49,10 @@
       <artifactId>junit</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.easymock</groupId>
+      <artifactId>easymock</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java b/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
index 0162466..0238fde 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
@@ -24,7 +24,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
-import org.apache.accumulo.fate.TStore.TStatus;
+import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooLock;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
@@ -36,7 +36,7 @@ import org.apache.zookeeper.KeeperException;
 public class AdminUtil<T> {
   private static final Charset UTF8 = Charset.forName("UTF-8");
   
-  public void print(ZooStore<T> zs, IZooReaderWriter zk, String lockPath) throws KeeperException, InterruptedException {
+  public void print(ReadOnlyTStore<T> zs, IZooReaderWriter zk, String lockPath) throws KeeperException, InterruptedException {
     Map<Long,List<String>> heldLocks = new HashMap<Long,List<String>>();
     Map<Long,List<String>> waitingLocks = new HashMap<Long,List<String>>();
     
@@ -108,7 +108,7 @@ public class AdminUtil<T> {
         wlocks = Collections.emptyList();
       
       String top = null;
-      Repo<T> repo = zs.top(tid);
+      ReadOnlyRepo<T> repo = zs.top(tid);
       if (repo != null)
         top = repo.getDescription();
       
@@ -132,7 +132,7 @@ public class AdminUtil<T> {
     }
   }
   
-  public boolean prepDelete(ZooStore<T> zs, IZooReaderWriter zk, String path, String txidStr) {
+  public boolean prepDelete(TStore<T> zs, IZooReaderWriter zk, String path, String txidStr) {
     if (!checkGlobalLock(zk, path)) {
       return false;
     }
@@ -145,7 +145,7 @@ public class AdminUtil<T> {
     return true;
   }
   
-  public boolean prepFail(ZooStore<T> zs, IZooReaderWriter zk, String path, String txidStr) {
+  public boolean prepFail(TStore<T> zs, IZooReaderWriter zk, String path, String txidStr) {
     if (!checkGlobalLock(zk, path)) {
       return false;
     }
@@ -158,7 +158,7 @@ public class AdminUtil<T> {
     return true;
   }
   
-  public void deleteLocks(ZooStore<T> zs, IZooReaderWriter zk, String path, String txidStr) throws KeeperException, InterruptedException {
+  public void deleteLocks(TStore<T> zs, IZooReaderWriter zk, String path, String txidStr) throws KeeperException, InterruptedException {
     // delete any locks assoc w/ fate operation
     List<String> lockedIds = zk.getChildren(path);
     

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/main/java/org/apache/accumulo/fate/Fate.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/Fate.java b/fate/src/main/java/org/apache/accumulo/fate/Fate.java
index 9d24b0b..b2eb681 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/Fate.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/Fate.java
@@ -18,7 +18,7 @@ package org.apache.accumulo.fate;
 
 import java.util.EnumSet;
 
-import org.apache.accumulo.fate.TStore.TStatus;
+import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
 import org.apache.accumulo.fate.util.Daemon;
 import org.apache.accumulo.fate.util.LoggingRunnable;
 import org.apache.log4j.Logger;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyRepo.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyRepo.java b/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyRepo.java
new file mode 100644
index 0000000..24d00d9
--- /dev/null
+++ b/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyRepo.java
@@ -0,0 +1,32 @@
+/*
+ * 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.fate;
+
+/**
+ * Read only access to a repeatable persisted operation.
+ *
+ * By definition, these methods are safe to call without impacting the state of FATE. They should also be
+ * safe to call without impacting the state of system components.
+ *
+ */
+public interface ReadOnlyRepo<T> {
+
+  long isReady(long tid, T environment) throws Exception;
+
+  String getDescription();
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyStore.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyStore.java b/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyStore.java
new file mode 100644
index 0000000..7cb20ff
--- /dev/null
+++ b/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyStore.java
@@ -0,0 +1,111 @@
+/*
+ * 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.fate;
+
+import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
+import com.google.common.base.Preconditions;
+
+import java.io.Serializable;
+import java.util.EnumSet;
+import java.util.List;
+
+/**
+ * This store decorates a TStore to make sure it can not be modified.
+ *
+ * Unlike relying directly on the ReadOnlyTStore interface, this class will not allow subsequent users to cast back to a
+ * mutable TStore successfully.
+ *
+ */
+public class ReadOnlyStore<T> implements ReadOnlyTStore<T> {
+
+  private final TStore<T> store;
+
+  /**
+   * @param store may not be null
+   */
+  public ReadOnlyStore(TStore<T> store) {
+    Preconditions.checkNotNull(store);
+    this.store = store;
+  }
+
+  @Override
+  public long reserve() {
+    return store.reserve();
+  }
+
+  @Override
+  public void reserve(long tid) {
+    store.reserve(tid);
+  }
+
+  @Override
+  public void unreserve(long tid, long deferTime) {
+    store.unreserve(tid, deferTime);
+  }
+
+  /**
+   * Decorates a Repo to make sure it is treated as a ReadOnlyRepo.
+   *
+   * Similar to ReadOnlyStore, won't allow subsequent user to cast a ReadOnlyRepo back to a mutable Repo.
+   */
+  protected static class ReadOnlyRepoWrapper<X> implements ReadOnlyRepo<X> {
+    private final Repo<X> repo;
+
+    /**
+     * @param repo may not be null
+     */
+    public ReadOnlyRepoWrapper(Repo<X> repo) {
+      Preconditions.checkNotNull(repo);
+      this.repo = repo;
+    }
+
+    @Override
+    public long isReady(long tid, X environment) throws Exception {
+      return repo.isReady(tid, environment);
+    }
+
+    @Override
+    public String getDescription() {
+      return repo.getDescription();
+    }
+  }
+
+  @Override
+  public ReadOnlyRepo<T> top(long tid) {
+    return new ReadOnlyRepoWrapper(store.top(tid));
+  }
+
+  @Override
+  public TStatus getStatus(long tid) {
+    return store.getStatus(tid);
+  }
+
+  @Override
+  public TStatus waitForStatusChange(long tid, EnumSet<TStatus> expected) {
+    return store.waitForStatusChange(tid, expected);
+  }
+
+  @Override
+  public Serializable getProperty(long tid, String prop) {
+    return store.getProperty(tid, prop);
+  }
+
+  @Override
+  public List<Long> list() {
+    return store.list();
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyTStore.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyTStore.java b/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyTStore.java
new file mode 100644
index 0000000..d390139
--- /dev/null
+++ b/fate/src/main/java/org/apache/accumulo/fate/ReadOnlyTStore.java
@@ -0,0 +1,125 @@
+/*
+ * 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.fate;
+
+import java.io.Serializable;
+import java.util.EnumSet;
+import java.util.List;
+
+/**
+ * Read only access to a Transaction Store.
+ *
+ * A transaction consists of a number of operations. Instances of this class may check on the queue of outstanding
+ * transactions but may neither modify them nor create new ones.
+ */
+public interface ReadOnlyTStore<T> {
+
+  /**
+   * Possible operational status codes. Serialized by name within stores.
+   */
+  enum TStatus {
+    /** Unseeded transaction */
+    NEW,
+    /** Transaction is eligible to be executing */
+    IN_PROGRESS,
+    /** Transaction has failed, and is in the process of being rolled back */
+    FAILED_IN_PROGRESS,
+    /** Transaction has failed and has been fully rolled back */
+    FAILED,
+    /** Transaction has succeeded */
+    SUCCESSFUL,
+    /** Unrecognized or unknown transaction state */
+    UNKNOWN
+  }
+
+  /**
+   * Reserve a transaction that is IN_PROGRESS or FAILED_IN_PROGRESS.
+   *
+   * Reserving a transaction id ensures that nothing else in-process interacting via the same instance
+   * will be operating on that transaction id.
+   *
+   * @return a transaction id that is safe to interact with, chosen by the store.
+   */
+  long reserve();
+
+  /**
+   * Reserve the specific tid.
+   *
+   * Reserving a transaction id ensures that nothing else in-process interacting via the same instance
+   * will be operating on that transaction id.
+   *
+   */
+  void reserve(long tid);
+
+  /**
+   * Return the given transaction to the store.
+   *
+   * upon successful return the store now controls the referenced transaction id. caller should no longer interact with it.
+   *
+   * @param tid transaction id, previously reserved.
+   * @param deferTime time in millis to keep this transaction out of the pool used in the {@link #reserve() reserve} method. must be non-negative.
+   */
+  void unreserve(long tid, long deferTime);
+
+
+  /**
+   * Get the current operation for the given transaction id.
+   *
+   * Caller must have already reserved tid.
+   *
+   * @param tid transaction id, previously reserved.
+   * @return a read-only view of the operation
+   */
+  ReadOnlyRepo<T> top(long tid);
+
+  /**
+   * Get the state of a given transaction.
+   *
+   * Caller must have already reserved tid.
+   *
+   * @param tid transaction id, previously reserved.
+   * @return execution status
+   */
+  TStatus getStatus(long tid);
+
+  /**
+   * Wait for the satus of a transaction to change
+   *
+   * @param tid transaction id, need not have been reserved.
+   * @param expected a set of possible statuses we are interested in being notified about. may not be null.
+   * @return execution status.
+   */
+  TStatus waitForStatusChange(long tid, EnumSet<TStatus> expected);
+
+  /**
+   * Retrieve a transaction-specific property.
+   *
+   * Caller must have already reserved tid.
+   *
+   * @param tid transaction id, previously reserved.
+   * @param prop name of property to retrieve.
+   */
+  Serializable getProperty(long tid, String prop);
+
+  /**
+   * list all transaction ids in store.
+   *
+   * @return all outstanding transactions, including those reserved by others.
+   */
+  List<Long> list();
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/main/java/org/apache/accumulo/fate/Repo.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/Repo.java b/fate/src/main/java/org/apache/accumulo/fate/Repo.java
index 8bdca10..b0ebd1a 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/Repo.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/Repo.java
@@ -22,15 +22,12 @@ import java.io.Serializable;
  * Repeatable persisted operation
  * 
  */
-public interface Repo<T> extends Serializable {
-  long isReady(long tid, T environment) throws Exception;
+public interface Repo<T> extends ReadOnlyRepo<T>, Serializable {
   
   Repo<T> call(long tid, T environment) throws Exception;
   
   void undo(long tid, T environment) throws Exception;
   
-  String getDescription();
-  
   // this allows the last fate op to return something to the user
   String getReturn();
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/main/java/org/apache/accumulo/fate/TStore.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/TStore.java b/fate/src/main/java/org/apache/accumulo/fate/TStore.java
index 3554064..5ca24fc 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/TStore.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/TStore.java
@@ -27,45 +27,14 @@ import java.util.List;
  * service can then execute the transaction's operation, possibly pushing more operations onto the transaction as each step successfully completes. If a step
  * fails, the stack can be unwound, undoing each operation.
  */
-public interface TStore<T> {
-  
-  public enum TStatus {
-    /** Unseeded transaction */
-    NEW,
-    /** Transaction is eligible to be executing */
-    IN_PROGRESS,
-    /** Transaction has failed, and is in the process of being rolled back */
-    FAILED_IN_PROGRESS,
-    /** Transaction has failed and has been fully rolled back */
-    FAILED,
-    /** Transaction has succeeded */
-    SUCCESSFUL,
-    /** Unrecognized or unknown transaction state */
-    UNKNOWN
-  }
+public interface TStore<T> extends ReadOnlyTStore<T> {
   
   /**
    * Create a new transaction id
    * 
    * @return a transaction id
    */
-  public long create();
-  
-  /**
-   * Reserve a transaction that is IN_PROGRESS or FAILED_IN_PROGRESS.
-   * 
-   */
-  long reserve();
-  
-  public void reserve(long tid);
-  
-  /**
-   * Return the given transaction to the store
-   * 
-   * @param tid
-   * @param deferTime
-   */
-  void unreserve(long tid, long deferTime);
+  long create();
   
   /**
    * Get the current operation for the given transaction id.
@@ -74,6 +43,7 @@ public interface TStore<T> {
    *          transaction id
    * @return the operation
    */
+  @Override
   Repo<T> top(long tid);
   
   /**
@@ -84,7 +54,7 @@ public interface TStore<T> {
    * @param repo
    *          the operation
    */
-  public void push(long tid, Repo<T> repo) throws StackOverflowException;
+  void push(long tid, Repo<T> repo) throws StackOverflowException;
   
   /**
    * Remove the last pushed operation from the given transaction.
@@ -94,15 +64,6 @@ public interface TStore<T> {
   void pop(long tid);
   
   /**
-   * Get the state of a given transaction.
-   * 
-   * @param tid
-   *          transaction id
-   * @return execution status
-   */
-  public TStatus getStatus(long tid);
-  
-  /**
    * Update the state of a given transaction
    * 
    * @param tid
@@ -110,19 +71,9 @@ public interface TStore<T> {
    * @param status
    *          execution status
    */
-  public void setStatus(long tid, TStatus status);
-  
-  /**
-   * Wait for the satus of a transaction to change
-   * 
-   * @param tid
-   *          transaction id
-   */
-  public TStatus waitForStatusChange(long tid, EnumSet<TStatus> expected);
-  
-  public void setProperty(long tid, String prop, Serializable val);
+  void setStatus(long tid, TStatus status);
   
-  public Serializable getProperty(long tid, String prop);
+  void setProperty(long tid, String prop, Serializable val);
   
   /**
    * Remove the transaction from the store.
@@ -130,13 +81,6 @@ public interface TStore<T> {
    * @param tid
    *          the transaction id
    */
-  public void delete(long tid);
-  
-  /**
-   * list all transaction ids in store
-   * 
-   */
-  
-  public List<Long> list();
-  
+  void delete(long tid);
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/test/java/org/apache/accumulo/fate/AgeOffStoreTest.java
----------------------------------------------------------------------
diff --git a/fate/src/test/java/org/apache/accumulo/fate/AgeOffStoreTest.java b/fate/src/test/java/org/apache/accumulo/fate/AgeOffStoreTest.java
index c212649..4f5b112 100644
--- a/fate/src/test/java/org/apache/accumulo/fate/AgeOffStoreTest.java
+++ b/fate/src/test/java/org/apache/accumulo/fate/AgeOffStoreTest.java
@@ -20,7 +20,7 @@ import java.util.Arrays;
 import java.util.HashSet;
 
 import org.apache.accumulo.fate.AgeOffStore.TimeSource;
-import org.apache.accumulo.fate.TStore.TStatus;
+import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
 import org.junit.Assert;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/test/java/org/apache/accumulo/fate/ReadOnlyStoreTest.java
----------------------------------------------------------------------
diff --git a/fate/src/test/java/org/apache/accumulo/fate/ReadOnlyStoreTest.java b/fate/src/test/java/org/apache/accumulo/fate/ReadOnlyStoreTest.java
new file mode 100644
index 0000000..c2d5f92
--- /dev/null
+++ b/fate/src/test/java/org/apache/accumulo/fate/ReadOnlyStoreTest.java
@@ -0,0 +1,72 @@
+/*
+ * 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.fate;
+
+import java.util.Collections;
+import java.util.EnumSet;
+
+import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
+
+import org.easymock.EasyMock;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Make sure read only decorate passes read methods.
+ */
+public class ReadOnlyStoreTest {
+
+  @Test
+  public void everythingPassesThrough() throws Exception {
+    @SuppressWarnings("unchecked")
+    Repo<String> repo = EasyMock.createMock(Repo.class);
+    EasyMock.expect(repo.getDescription()).andReturn("description");
+    EasyMock.expect(repo.isReady(0xdeadbeefl, null)).andReturn(0x0l);
+
+    @SuppressWarnings("unchecked")
+    TStore<String> mock = EasyMock.createNiceMock(TStore.class);
+    EasyMock.expect(mock.reserve()).andReturn(0xdeadbeefl);
+    mock.reserve(0xdeadbeefl);
+    EasyMock.expect(mock.top(0xdeadbeefl)).andReturn(repo);
+    EasyMock.expect(mock.getStatus(0xdeadbeefl)).andReturn(TStatus.UNKNOWN);
+    mock.unreserve(0xdeadbeefl, 30);
+
+    EasyMock.expect(mock.waitForStatusChange(0xdeadbeefl, EnumSet.allOf(TStatus.class))).andReturn(TStatus.UNKNOWN);
+    EasyMock.expect(mock.getProperty(0xdeadbeefl, "com.example.anyproperty")).andReturn("property");
+    EasyMock.expect(mock.list()).andReturn(Collections.<Long>emptyList());
+
+    EasyMock.replay(repo);
+    EasyMock.replay(mock);
+
+    ReadOnlyTStore<String> store = new ReadOnlyStore<String>(mock);
+    Assert.assertEquals(0xdeadbeefl, store.reserve());
+    store.reserve(0xdeadbeefl);
+    ReadOnlyRepo<String> top = store.top(0xdeadbeefl);
+    Assert.assertFalse(top instanceof Repo);
+    Assert.assertEquals("description", top.getDescription());
+    Assert.assertEquals(0x0l, top.isReady(0xdeadbeefl, null));
+    Assert.assertEquals(TStatus.UNKNOWN, store.getStatus(0xdeadbeefl));
+    store.unreserve(0xdeadbeefl, 30);
+
+    Assert.assertEquals(TStatus.UNKNOWN, store.waitForStatusChange(0xdeadbeefl, EnumSet.allOf(TStatus.class)));
+    Assert.assertEquals("property", store.getProperty(0xdeadbeefl, "com.example.anyproperty"));
+    Assert.assertEquals(Collections.<Long>emptyList(), store.list());
+
+    EasyMock.verify(repo);
+    EasyMock.verify(mock);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/fate/src/test/java/org/apache/accumulo/fate/SimpleStore.java
----------------------------------------------------------------------
diff --git a/fate/src/test/java/org/apache/accumulo/fate/SimpleStore.java b/fate/src/test/java/org/apache/accumulo/fate/SimpleStore.java
index 3b78131..60eabfb 100644
--- a/fate/src/test/java/org/apache/accumulo/fate/SimpleStore.java
+++ b/fate/src/test/java/org/apache/accumulo/fate/SimpleStore.java
@@ -28,7 +28,7 @@ import java.util.Set;
 import org.apache.commons.lang.NotImplementedException;
 
 /**
- * 
+ * Transient in memory store for transactions.
  */
 public class SimpleStore<T> implements TStore<T> {
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/server/src/main/java/org/apache/accumulo/server/fate/Admin.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/fate/Admin.java b/server/src/main/java/org/apache/accumulo/server/fate/Admin.java
index 4a5f0bc..fc9e342 100644
--- a/server/src/main/java/org/apache/accumulo/server/fate/Admin.java
+++ b/server/src/main/java/org/apache/accumulo/server/fate/Admin.java
@@ -25,6 +25,7 @@ import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.AdminUtil;
 import org.apache.accumulo.fate.ZooStore;
+import org.apache.accumulo.fate.ReadOnlyStore;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.master.Master;
@@ -88,7 +89,7 @@ public class Admin {
       }
       admin.deleteLocks(zs, zk, ZooUtil.getRoot(instance) + Constants.ZTABLE_LOCKS, args[1]);
     } else if (jc.getParsedCommand().equals("print")) {
-      admin.print(zs, zk, ZooUtil.getRoot(instance) + Constants.ZTABLE_LOCKS);
+      admin.print(new ReadOnlyStore(zs), zk, ZooUtil.getRoot(instance) + Constants.ZTABLE_LOCKS);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a904f691/server/src/main/java/org/apache/accumulo/server/master/Master.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/Master.java b/server/src/main/java/org/apache/accumulo/server/master/Master.java
index 8c4c864..270eb18 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/Master.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/Master.java
@@ -90,7 +90,7 @@ 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.ReadOnlyTStore.TStatus;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooLock.LockLossReason;
 import org.apache.accumulo.fate.zookeeper.ZooReaderWriter.Mutator;


[05/15] git commit: ACCUMULO-2519 Aborts upgrade if there are Fate transactions from an old version.

Posted by bu...@apache.org.
ACCUMULO-2519 Aborts upgrade if there are Fate transactions from an old version.


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

Branch: refs/heads/master
Commit: 5a504b311c0e5f59ff5b14221c6bf61f43b4d093
Parents: a904f69
Author: Sean Busbey <bu...@cloudera.com>
Authored: Fri Mar 28 01:46:09 2014 -0500
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Apr 4 17:27:05 2014 -0700

----------------------------------------------------------------------
 README                                          |  14 +++
 .../org/apache/accumulo/server/Accumulo.java    |  31 ++++++
 .../apache/accumulo/server/master/Master.java   | 100 ++++++++++++-------
 .../server/tabletserver/TabletServer.java       |   5 +
 .../accumulo/server/util/MetadataTable.java     |   3 +
 5 files changed, 116 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/5a504b31/README
----------------------------------------------------------------------
diff --git a/README b/README
index 115a9b7..0bb1030 100644
--- a/README
+++ b/README
@@ -54,12 +54,26 @@ accumulo.
 
  This happens automatically the first time Accumulo 1.5 is started.  
 
+  * Verify that there are no outstanding FATE operations
+    - Under 1.4 you can list what's in FATE by running
+      $ACCUMULO_HOME/bin/accumulo org.apache.accumulo.server.fate.Admin print
+    - Note that operations in any state will prevent an upgrade. It is safe
+      to delete operations with status SUCCESSFUL. For others, you should restart
+      your 1.4 cluster and allow them to finish.
   * Stop the 1.4 instance.  
   * Configure 1.5 to use the hdfs directory, walog directories, and zookeepers
     that 1.4 was using.
   * Copy other 1.4 configuration options as needed.
   * Start Accumulo 1.5. 
 
+  The upgrade process must make changes to Accumulo's internal state in both ZooKeeper and
+  the table metadata. This process may take some time as Tablet Servers move write-ahead
+  logs to HDFS and then do recovery. During this time, the Monitor will claim that the
+  Master is down and some services may send the Monitor log messages about failure to
+  communicate with each other. These messages are safe to ignore. If you need detail on
+  the upgrade's progress you should view the local logs on the Tablet Servers and active
+  Master.
+
 ******************************************************************************
 4. Configuring
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5a504b31/server/src/main/java/org/apache/accumulo/server/Accumulo.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/Accumulo.java b/server/src/main/java/org/apache/accumulo/server/Accumulo.java
index 99ec7e4..420b6cc 100644
--- a/server/src/main/java/org/apache/accumulo/server/Accumulo.java
+++ b/server/src/main/java/org/apache/accumulo/server/Accumulo.java
@@ -27,11 +27,16 @@ import java.util.Map.Entry;
 import java.util.TreeMap;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.trace.DistributedTrace;
 import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.util.Version;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.ReadOnlyTStore;
+import org.apache.accumulo.fate.ReadOnlyStore;
+import org.apache.accumulo.fate.ZooStore;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.util.time.SimpleTimer;
@@ -53,6 +58,7 @@ public class Accumulo {
     try {
       if (getAccumuloPersistentVersion(fs) == Constants.PREV_DATA_VERSION) {
         fs.create(new Path(ServerConstants.getDataVersionLocation() + "/" + Constants.DATA_VERSION));
+        // TODO document failure mode & recovery if FS permissions cause above to work and below to fail ACCUMULO-2596
         fs.delete(new Path(ServerConstants.getDataVersionLocation() + "/" + Constants.PREV_DATA_VERSION), false);
       }
     } catch (IOException e) {
@@ -263,4 +269,29 @@ public class Accumulo {
       throw new RuntimeException("cannot find method setSafeMode", ex);
     }
   }
+
+  /**
+   * Exit loudly if there are outstanding Fate operations.
+   * Since Fate serializes class names, we need to make sure there are no queued
+   * transactions from a previous version before continuing an upgrade. The status of the operations is
+   * irrelevant; those in SUCCESSFUL status cause the same problem as those just queued.
+   *
+   * Note that the Master should not allow write access to Fate until after all upgrade steps are complete.
+   *
+   * Should be called as a guard before performing any upgrade steps, after determining that an upgrade is needed.
+   *
+   * see ACCUMULO-2519
+   */
+  public static void abortIfFateTransactions() {
+    try {
+      final ReadOnlyTStore<Accumulo> fate = new ReadOnlyStore<Accumulo>(new ZooStore<Accumulo>(ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZFATE,
+          ZooReaderWriter.getRetryingInstance()));
+      if (!(fate.list().isEmpty())) {
+        throw new AccumuloException("Aborting upgrade because there are outstanding FATE transactions from a previous Accumulo version. Please see the README document for instructions on what to do under your previous version.");
+      }
+    } catch (Exception exception) {
+      log.fatal("Problem verifying Fate readiness", exception);
+      System.exit(1);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5a504b31/server/src/main/java/org/apache/accumulo/server/master/Master.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/Master.java b/server/src/main/java/org/apache/accumulo/server/master/Master.java
index 270eb18..a2ad2e6 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/Master.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/Master.java
@@ -34,6 +34,7 @@ import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -271,7 +272,9 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       upgradeMetadata();
     }
   }
-  
+
+  private boolean haveUpgradedZooKeeper = false;
+
   private void upgradeZookeeper() {
     // 1.5.1 and 1.6.0 both do some state checking after obtaining the zoolock for the
     // monitor and before starting up. It's not tied to the data version at all (and would
@@ -279,59 +282,79 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
     // that the master is not the only thing that may alter zookeeper before starting.
 
     if (Accumulo.getAccumuloPersistentVersion(fs) == Constants.PREV_DATA_VERSION) {
+      // This Master hasn't started Fate yet, so any outstanding transactions must be from before the upgrade.
+      // Change to Guava's Verify once we use Guava 17.
+      if (null != fate) {
+        throw new IllegalStateException("Access to Fate should not have been initialized prior to the Master transitioning to active. Please save all logs and file a bug.");
+      }
+      Accumulo.abortIfFateTransactions();
       try {
         log.info("Upgrading zookeeper");
-        
+
         IZooReaderWriter zoo = ZooReaderWriter.getInstance();
-        
+
         zoo.recursiveDelete(ZooUtil.getRoot(instance) + "/loggers", NodeMissingPolicy.SKIP);
         zoo.recursiveDelete(ZooUtil.getRoot(instance) + "/dead/loggers", NodeMissingPolicy.SKIP);
 
         zoo.putPersistentData(ZooUtil.getRoot(instance) + Constants.ZRECOVERY, new byte[] {'0'}, NodeExistsPolicy.SKIP);
-        
+
         for (String id : Tables.getIdToNameMap(instance).keySet()) {
-          
+
           zoo.putPersistentData(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + id + Constants.ZTABLE_COMPACT_CANCEL_ID, "0".getBytes(Constants.UTF8),
               NodeExistsPolicy.SKIP);
         }
+        haveUpgradedZooKeeper = true;
       } catch (Exception ex) {
         log.fatal("Error performing upgrade", ex);
         System.exit(1);
       }
     }
   }
-  
+
   private final AtomicBoolean upgradeMetadataRunning = new AtomicBoolean(false);
-  
+  private final CountDownLatch waitForMetadataUpgrade = new CountDownLatch(1);
+
   private final ServerConfiguration serverConfig;
   
   private void upgradeMetadata() {
-    if (Accumulo.getAccumuloPersistentVersion(fs) == Constants.PREV_DATA_VERSION) {
-      if (upgradeMetadataRunning.compareAndSet(false, true)) {
+    // we make sure we're only doing the rest of this method once so that we can signal to other threads that an upgrade wasn't needed.
+    if (upgradeMetadataRunning.compareAndSet(false, true)) {
+      if (Accumulo.getAccumuloPersistentVersion(fs) == Constants.PREV_DATA_VERSION) {
+        // sanity check that we passed the Fate verification prior to ZooKeeper upgrade, and that Fate still hasn't been started.
+        // Change both to use Guava's Verify once we use Guava 17.
+        if (!haveUpgradedZooKeeper) {
+          throw new IllegalStateException("We should only attempt to upgrade Accumulo's !METADATA table if we've already upgraded ZooKeeper. Please save all logs and file a bug.");
+        }
+        if (null != fate) {
+          throw new IllegalStateException("Access to Fate should not have been initialized prior to the Master finishing upgrades. Please save all logs and file a bug.");
+        }
         Runnable upgradeTask = new Runnable() {
           @Override
           public void run() {
             try {
+              log.info("Starting to upgrade !METADATA table.");
               MetadataTable.moveMetaDeleteMarkers(instance, SecurityConstants.getSystemCredentials());
+              log.info("Updating persistent data version.");
               Accumulo.updateAccumuloVersion(fs);
-              
               log.info("Upgrade complete");
-              
+              waitForMetadataUpgrade.countDown();
             } catch (Exception ex) {
               log.fatal("Error performing upgrade", ex);
               System.exit(1);
             }
-            
+
           }
         };
-        
+
         // need to run this in a separate thread because a lock is held that prevents !METADATA tablets from being assigned and this task writes to the
         // !METADATA table
         new Thread(upgradeTask).start();
+      } else {
+        waitForMetadataUpgrade.countDown();
       }
     }
   }
-  
+
   private int assignedOrHosted(Text tableId) {
     int result = 0;
     for (TabletGroupWatcher watcher : watchers) {
@@ -2136,28 +2159,6 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
     
     tserverSet.startListeningForTabletServerChanges();
     
-    // TODO: add shutdown for fate object - ACCUMULO-1307
-    try {
-      final AgeOffStore<Master> store = new AgeOffStore<Master>(new org.apache.accumulo.fate.ZooStore<Master>(ZooUtil.getRoot(instance) + Constants.ZFATE,
-          ZooReaderWriter.getRetryingInstance()), 1000 * 60 * 60 * 8);
-      
-      int threads = this.getConfiguration().getConfiguration().getCount(Property.MASTER_FATE_THREADPOOL_SIZE);
-      
-      fate = new Fate<Master>(this, store, threads);
-      
-      SimpleTimer.getInstance().schedule(new Runnable() {
-        
-        @Override
-        public void run() {
-          store.ageOff();
-        }
-      }, 63000, 63000);
-    } catch (KeeperException e) {
-      throw new IOException(e);
-    } catch (InterruptedException e) {
-      throw new IOException(e);
-    }
-    
     ZooReaderWriter.getInstance().getChildren(zroot + Constants.ZRECOVERY, new Watcher() {
       @Override
       public void process(WatchedEvent event) {
@@ -2183,7 +2184,32 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
     for (TabletGroupWatcher watcher : watchers) {
       watcher.start();
     }
-    
+
+    // Once we are sure tablet servers are no longer checking for an empty Fate transaction queue before doing WAL upgrades, we can safely start using Fate ourselves.
+    waitForMetadataUpgrade.await();
+
+    // TODO: add shutdown for fate object - ACCUMULO-1307
+    try {
+      final AgeOffStore<Master> store = new AgeOffStore<Master>(new org.apache.accumulo.fate.ZooStore<Master>(ZooUtil.getRoot(instance) + Constants.ZFATE,
+          ZooReaderWriter.getRetryingInstance()), 1000 * 60 * 60 * 8);
+
+      int threads = this.getConfiguration().getConfiguration().getCount(Property.MASTER_FATE_THREADPOOL_SIZE);
+
+      fate = new Fate<Master>(this, store, threads);
+
+      SimpleTimer.getInstance().schedule(new Runnable() {
+
+        @Override
+        public void run() {
+          store.ageOff();
+        }
+      }, 63000, 63000);
+    } catch (KeeperException e) {
+      throw new IOException(e);
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
+
     Processor<Iface> processor = new Processor<Iface>(TraceWrap.service(new MasterClientServiceHandler()));
     ServerPort serverPort = TServerUtils.startServer(getSystemConfiguration(), Property.MASTER_CLIENTPORT, processor, "Master",
         "Master Client Service Handler", null, Property.MASTER_MINTHREADS, Property.MASTER_THREADCHECK, Property.GENERAL_MAX_MESSAGE_SIZE);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5a504b31/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java b/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
index d76946d..ad3d615 100644
--- a/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
+++ b/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
@@ -3322,6 +3322,11 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
    * 
    */
   public static void recoverLocalWriteAheadLogs(FileSystem fs, ServerConfiguration serverConf) throws IOException {
+    if (Accumulo.getAccumuloPersistentVersion(fs) == Constants.PREV_DATA_VERSION) {
+      // If the Master has not yet signaled a finish to upgrading, we need to make sure we can rollback in the
+      // event of outstanding transactions in Fate from the previous version.
+      Accumulo.abortIfFateTransactions();
+    }
     FileSystem localfs = FileSystem.getLocal(fs.getConf()).getRawFileSystem();
     AccumuloConfiguration conf = serverConf.getConfiguration();
     String localWalDirectories = conf.get(Property.LOGGER_DIR);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5a504b31/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java b/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
index 7328a55..d6e0a3c 100644
--- a/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
+++ b/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
@@ -1233,6 +1233,9 @@ public class MetadataTable extends org.apache.accumulo.core.util.MetadataTable {
     update(SecurityConstants.getSystemCredentials(), m);
   }
 
+  /**
+   * During an upgrade from Accumulo 1.4 -> 1.5, we need to move deletion requests for files under the !METADATA table to the root tablet.
+   */
   public static void moveMetaDeleteMarkers(Instance instance, TCredentials creds) {
     // move delete markers from the normal delete keyspace to the root tablet delete keyspace if the files are for the !METADATA table
     Scanner scanner = new ScannerImpl(instance, creds, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);


[11/15] git commit: ACCUMULO-2148 change upgrade docs to cover 1.5 -> 1.6

Posted by bu...@apache.org.
ACCUMULO-2148 change upgrade docs to cover 1.5 -> 1.6


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

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: f5a94f041ebe6b2b14624e4a6d25625bda6b5df9
Parents: e4aa11e
Author: Sean Busbey <bu...@cloudera.com>
Authored: Fri Apr 4 16:22:21 2014 -0700
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Apr 4 17:28:54 2014 -0700

----------------------------------------------------------------------
 README | 38 +++++++++++++++++++++++---------------
 1 file changed, 23 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f5a94f04/README
----------------------------------------------------------------------
diff --git a/README b/README
index 173d788..f513b75 100644
--- a/README
+++ b/README
@@ -100,29 +100,37 @@ a profile to maven's settings.xml file. Below is an example of $HOME/.m2/setting
  </settings>
 
 ******************************************************************************
-3. Upgrading from 1.4 to 1.5
+3. Upgrading from 1.5 to 1.6
 
- This happens automatically the first time Accumulo 1.5 is started.  
+ This happens automatically the first time Accumulo 1.6 is started.
+
+ If your instance previously upgraded from 1.4 to 1.5, you must verify that your
+ 1.5 instance has no outstanding local write ahead logs. You can do this by ensuring
+ either:
+
+  - All of your tables are online and the Monitor shows all tablets hosted
+  - The directory for write ahead logs (logger.dir.walog) from 1.4 has no files remaining
+    on any tablet server / logger hosts
+
+ To upgrade from 1.5 to 1.6 you must:
 
   * Verify that there are no outstanding FATE operations
-    - Under 1.4 you can list what's in FATE by running
+    - Under 1.5 you can list what's in FATE by running
       $ACCUMULO_HOME/bin/accumulo org.apache.accumulo.server.fate.Admin print
     - Note that operations in any state will prevent an upgrade. It is safe
       to delete operations with status SUCCESSFUL. For others, you should restart
-      your 1.4 cluster and allow them to finish.
-  * Stop the 1.4 instance.  
-  * Configure 1.5 to use the hdfs directory, walog directories, and zookeepers
-    that 1.4 was using.
-  * Copy other 1.4 configuration options as needed.
-  * Start Accumulo 1.5. 
+      your 1.5 cluster and allow them to finish.
+  * Stop the 1.5 instance.
+  * Configure 1.6 to use the hdfs directory and zookeepers that 1.5 was using.
+  * Copy other 1.5 configuration options as needed.
+  * Start Accumulo 1.6.
 
   The upgrade process must make changes to Accumulo's internal state in both ZooKeeper and
-  the table metadata. This process may take some time as Tablet Servers move write-ahead
-  logs to HDFS and then do recovery. During this time, the Monitor will claim that the
-  Master is down and some services may send the Monitor log messages about failure to
-  communicate with each other. These messages are safe to ignore. If you need detail on
-  the upgrade's progress you should view the local logs on the Tablet Servers and active
-  Master.
+  the table metadata. This process may take some time if Tablet Servers have to go through
+  recovery. During this time, the Monitor will claim that the Master is down and some
+  services may send the Monitor log messages about failure to communicate with each other.
+  These messages are safe to ignore. If you need detail on the upgrade's progress you should
+  view the local logs on the Tablet Servers and active Master.
 
 ******************************************************************************
 4. Configuring


[13/15] git commit: ACCUMULO-2148 change upgrade docs to cover 1.5 -> 1.6

Posted by bu...@apache.org.
ACCUMULO-2148 change upgrade docs to cover 1.5 -> 1.6


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

Branch: refs/heads/master
Commit: f5a94f041ebe6b2b14624e4a6d25625bda6b5df9
Parents: e4aa11e
Author: Sean Busbey <bu...@cloudera.com>
Authored: Fri Apr 4 16:22:21 2014 -0700
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Apr 4 17:28:54 2014 -0700

----------------------------------------------------------------------
 README | 38 +++++++++++++++++++++++---------------
 1 file changed, 23 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f5a94f04/README
----------------------------------------------------------------------
diff --git a/README b/README
index 173d788..f513b75 100644
--- a/README
+++ b/README
@@ -100,29 +100,37 @@ a profile to maven's settings.xml file. Below is an example of $HOME/.m2/setting
  </settings>
 
 ******************************************************************************
-3. Upgrading from 1.4 to 1.5
+3. Upgrading from 1.5 to 1.6
 
- This happens automatically the first time Accumulo 1.5 is started.  
+ This happens automatically the first time Accumulo 1.6 is started.
+
+ If your instance previously upgraded from 1.4 to 1.5, you must verify that your
+ 1.5 instance has no outstanding local write ahead logs. You can do this by ensuring
+ either:
+
+  - All of your tables are online and the Monitor shows all tablets hosted
+  - The directory for write ahead logs (logger.dir.walog) from 1.4 has no files remaining
+    on any tablet server / logger hosts
+
+ To upgrade from 1.5 to 1.6 you must:
 
   * Verify that there are no outstanding FATE operations
-    - Under 1.4 you can list what's in FATE by running
+    - Under 1.5 you can list what's in FATE by running
       $ACCUMULO_HOME/bin/accumulo org.apache.accumulo.server.fate.Admin print
     - Note that operations in any state will prevent an upgrade. It is safe
       to delete operations with status SUCCESSFUL. For others, you should restart
-      your 1.4 cluster and allow them to finish.
-  * Stop the 1.4 instance.  
-  * Configure 1.5 to use the hdfs directory, walog directories, and zookeepers
-    that 1.4 was using.
-  * Copy other 1.4 configuration options as needed.
-  * Start Accumulo 1.5. 
+      your 1.5 cluster and allow them to finish.
+  * Stop the 1.5 instance.
+  * Configure 1.6 to use the hdfs directory and zookeepers that 1.5 was using.
+  * Copy other 1.5 configuration options as needed.
+  * Start Accumulo 1.6.
 
   The upgrade process must make changes to Accumulo's internal state in both ZooKeeper and
-  the table metadata. This process may take some time as Tablet Servers move write-ahead
-  logs to HDFS and then do recovery. During this time, the Monitor will claim that the
-  Master is down and some services may send the Monitor log messages about failure to
-  communicate with each other. These messages are safe to ignore. If you need detail on
-  the upgrade's progress you should view the local logs on the Tablet Servers and active
-  Master.
+  the table metadata. This process may take some time if Tablet Servers have to go through
+  recovery. During this time, the Monitor will claim that the Master is down and some
+  services may send the Monitor log messages about failure to communicate with each other.
+  These messages are safe to ignore. If you need detail on the upgrade's progress you should
+  view the local logs on the Tablet Servers and active Master.
 
 ******************************************************************************
 4. Configuring


[14/15] git commit: ACCUMULO-2519 Updates Classes added in 1.6.0 for read only fate changes.

Posted by bu...@apache.org.
ACCUMULO-2519 Updates Classes added in 1.6.0 for read only fate changes.


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

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: e4aa11e1b1a046dec9116273eb57f053aa68fd3f
Parents: 957c9d1
Author: Sean Busbey <bu...@cloudera.com>
Authored: Fri Apr 4 01:35:01 2014 -0700
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Apr 4 17:28:54 2014 -0700

----------------------------------------------------------------------
 .../org/apache/accumulo/core/util/shell/commands/FateCommand.java  | 2 +-
 .../main/java/org/apache/accumulo/master/FateServiceHandler.java   | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/e4aa11e1/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FateCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FateCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FateCommand.java
index 6bf4f30..0196baf 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FateCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FateCommand.java
@@ -31,7 +31,7 @@ import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.AdminUtil;
-import org.apache.accumulo.fate.TStore.TStatus;
+import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
 import org.apache.accumulo.fate.ZooStore;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e4aa11e1/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
index 555e3e4..d63a63e 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
@@ -41,7 +41,7 @@ import org.apache.accumulo.core.master.thrift.FateService;
 import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.util.ArgumentChecker.Validator;
 import org.apache.accumulo.core.util.ByteBufferUtil;
-import org.apache.accumulo.fate.TStore.TStatus;
+import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
 import org.apache.accumulo.master.tableOps.BulkImport;
 import org.apache.accumulo.master.tableOps.CancelCompactions;
 import org.apache.accumulo.master.tableOps.ChangeTableState;