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:41 UTC

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

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;