Modified: 
accumulo/trunk/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooLock.java
URL: 
http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooLock.java?rev=1355481&r1=1355480&r2=1355481&view=diff
==============================================================================
--- 
accumulo/trunk/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooLock.java
 (original)
+++ 
accumulo/trunk/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooLock.java
 Fri Jun 29 17:42:35 2012
@@ -16,443 +16,20 @@
  */
 package org.apache.accumulo.server.zookeeper;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.accumulo.core.zookeeper.ZooUtil.LockID;
-import org.apache.accumulo.core.zookeeper.ZooUtil.NodeMissingPolicy;
-import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.Watcher.Event.EventType;
-import org.apache.zookeeper.Watcher.Event.KeeperState;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.data.Stat;
 
-public class ZooLock implements Watcher {
-  
-  private static final Logger log = Logger.getLogger(ZooLock.class);
-  
-  public static final String LOCK_PREFIX = "zlock-";
-  
-  public enum LockLossReason {
-    LOCK_DELETED, SESSION_EXPIRED
-  }
-  
-  public interface LockWatcher {
-    void lostLock(LockLossReason reason);
-  }
-  
-  public interface AsyncLockWatcher extends LockWatcher {
-    void acquiredLock();
-    
-    void failedToAcquireLock(Exception e);
-  }
-  
-  private boolean lockWasAcquired;
-  final private String path;
-  final private IZooReaderWriter zooKeeper;
-  private String lock;
-  private LockWatcher lockWatcher;
-  
-  private String asyncLock;
+public class ZooLock extends org.apache.accumulo.fate.zookeeper.ZooLock {
   
   public ZooLock(String path) {
-    this.path = path;
-    zooKeeper = ZooReaderWriter.getInstance();
-    try {
-      zooKeeper.getStatus(path, this);
-    } catch (Exception ex) {
-      log.warn("Error getting setting initial watch on ZooLock", ex);
-    }
-  }
-  
-  private static class TryLockAsyncLockWatcher implements AsyncLockWatcher {
-    
-    boolean acquiredLock = false;
-    LockWatcher lw;
-    
-    public TryLockAsyncLockWatcher(LockWatcher lw2) {
-      this.lw = lw2;
-    }
-    
-    @Override
-    public void acquiredLock() {
-      acquiredLock = true;
-    }
-    
-    @Override
-    public void failedToAcquireLock(Exception e) {}
-    
-    @Override
-    public void lostLock(LockLossReason reason) {
-      lw.lostLock(reason);
-    }
-    
-  }
-  
-  public synchronized boolean tryLock(LockWatcher lw, byte data[]) throws 
KeeperException, InterruptedException {
-    
-    TryLockAsyncLockWatcher tlalw = new TryLockAsyncLockWatcher(lw);
-    
-    lockAsync(tlalw, data);
-    
-    if (tlalw.acquiredLock) {
-      return true;
-    }
-    
-    if (asyncLock != null) {
-      zooKeeper.recursiveDelete(path + "/" + asyncLock, 
NodeMissingPolicy.SKIP);
-      asyncLock = null;
-    }
-    
-    return false;
-  }
-  
-  private synchronized void lockAsync(final String myLock, final 
AsyncLockWatcher lw) throws KeeperException, InterruptedException {
-    
-    if (asyncLock == null) {
-      throw new IllegalStateException("Called lockAsync() when asyncLock == 
null");
-    }
-    
-    List<String> children = zooKeeper.getChildren(path);
-    
-    if (!children.contains(myLock)) {
-      throw new RuntimeException("Lock attempt ephemeral node no longer exist 
" + myLock);
-    }
-    
-    Collections.sort(children);
-    
-    if (children.get(0).equals(myLock)) {
-      this.lockWatcher = lw;
-      this.lock = myLock;
-      asyncLock = null;
-      lockWasAcquired = true;
-      lw.acquiredLock();
-      return;
-    }
-    String prev = null;
-    for (String child : children) {
-      if (child.equals(myLock)) {
-        break;
-      }
-      
-      prev = child;
-    }
-    
-    final String lockToWatch = path + "/" + prev;
-    
-    Stat stat = zooKeeper.getStatus(path + "/" + prev, new Watcher() {
-      
-      @Override
-      public void process(WatchedEvent event) {
-        
-        if (event.getType() == EventType.NodeDeleted && 
event.getPath().equals(lockToWatch)) {
-          synchronized (ZooLock.this) {
-            try {
-              if (asyncLock != null) {
-                lockAsync(myLock, lw);
-              } else if (log.isTraceEnabled()) {
-                log.trace("While waiting for another lock " + lockToWatch + " 
" + myLock + " was deleted");
-              }
-            } catch (Exception e) {
-              if (lock == null) {
-                // have not acquired lock yet
-                lw.failedToAcquireLock(e);
-              }
-            }
-          }
-        }
-        
-        if (event.getState() == KeeperState.Expired) {
-          synchronized (ZooLock.this) {
-            if (lock == null) {
-              lw.failedToAcquireLock(new Exception("Zookeeper Session 
expired"));
-            }
-          }
-        }
-      }
-      
-    });
-    
-    if (stat == null)
-      lockAsync(myLock, lw);
-  }
-  
-  public synchronized void lockAsync(final AsyncLockWatcher lw, byte data[]) {
-    
-    if (lockWatcher != null || lock != null || asyncLock != null) {
-      throw new IllegalStateException();
-    }
-    
-    lockWasAcquired = false;
-    
-    try {
-      String asyncLockPath = zooKeeper.putEphemeralSequential(path + "/" + 
LOCK_PREFIX, data);
-      
-      Stat stat = zooKeeper.getStatus(asyncLockPath, new Watcher() {
-        public void process(WatchedEvent event) {
-          synchronized (ZooLock.this) {
-            if (lock != null && event.getType() == EventType.NodeDeleted && 
event.getPath().equals(path + "/" + lock)) {
-              LockWatcher localLw = lockWatcher;
-              lock = null;
-              lockWatcher = null;
-              
-              localLw.lostLock(LockLossReason.LOCK_DELETED);
-              
-            } else if (asyncLock != null && event.getType() == 
EventType.NodeDeleted && event.getPath().equals(path + "/" + asyncLock)) {
-              lw.failedToAcquireLock(new Exception("Lock deleted before 
acquired"));
-              asyncLock = null;
-            }
-          }
-        }
-      });
-      
-      if (stat == null) {
-        lw.failedToAcquireLock(new Exception("Lock does not exist after 
create"));
-        return;
-      }
-      
-      asyncLock = asyncLockPath.substring(path.length() + 1);
-      
-      lockAsync(asyncLock, lw);
-      
-    } catch (KeeperException e) {
-      lw.failedToAcquireLock(e);
-    } catch (InterruptedException e) {
-      lw.failedToAcquireLock(e);
-    }
-  }
-  
-  public synchronized boolean tryToCancelAsyncLockOrUnlock() throws 
InterruptedException, KeeperException {
-    boolean del = false;
-    
-    if (asyncLock != null) {
-      zooKeeper.recursiveDelete(path + "/" + asyncLock, 
NodeMissingPolicy.SKIP);
-      del = true;
-    }
-    
-    if (lock != null) {
-      unlock();
-      del = true;
-    }
-    
-    return del;
-  }
-  
-  public synchronized void unlock() throws InterruptedException, 
KeeperException {
-    if (lock == null) {
-      throw new IllegalStateException();
-    }
-    
-    LockWatcher localLw = lockWatcher;
-    String localLock = lock;
-    
-    lock = null;
-    lockWatcher = null;
-    
-    zooKeeper.recursiveDelete(path + "/" + localLock, NodeMissingPolicy.SKIP);
-    
-    localLw.lostLock(LockLossReason.LOCK_DELETED);
-  }
-  
-  public synchronized String getLockPath() {
-    if (lock == null) {
-      return null;
-    }
-    return path + "/" + lock;
-  }
-  
-  public synchronized String getLockName() {
-    return lock;
-  }
-  
-  public synchronized LockID getLockID() {
-    if (lock == null) {
-      throw new IllegalStateException("Lock not held");
-    }
-    return new LockID(path, lock, zooKeeper.getZooKeeper().getSessionId());
-  }
-  
-  /**
-   * indicates if the lock was acquired in the past.... helps discriminate 
between the case where the lock was never held, or held and lost....
-   * 
-   * @return true if the lock was aquired, otherwise false.
-   */
-  public synchronized boolean wasLockAcquired() {
-    return lockWasAcquired;
-  }
-  
-  public synchronized boolean isLocked() {
-    return lock != null;
-  }
-  
-  @Override
-  public synchronized void process(WatchedEvent event) {
-    log.debug("event " + event.getPath() + " " + event.getType() + " " + 
event.getState());
-    
-    if (event.getState() == KeeperState.Expired && lock != null) {
-      LockWatcher localLw = lockWatcher;
-      lock = null;
-      lockWatcher = null;
-      localLw.lostLock(LockLossReason.SESSION_EXPIRED);
-    }
-  }
-  
-  public static boolean isLockHeld(ZooKeeper zk, LockID lid) throws 
KeeperException, InterruptedException {
-    
-    List<String> children = zk.getChildren(lid.path, false);
-    
-    if (children == null || children.size() == 0) {
-      return false;
-    }
-    
-    Collections.sort(children);
-    
-    String lockNode = children.get(0);
-    if (!lid.node.equals(lockNode))
-      return false;
-    
-    Stat stat = zk.exists(lid.path + "/" + lid.node, false);
-    return stat != null && stat.getEphemeralOwner() == lid.eid;
-  }
-  
-  public static boolean isLockHeld(ZooCache zc, LockID lid) {
-    
-    List<String> children = zc.getChildren(lid.path);
-    
-    if (children == null || children.size() == 0) {
-      return false;
-    }
-    
-    children = new ArrayList<String>(children);
-    Collections.sort(children);
-    
-    String lockNode = children.get(0);
-    if (!lid.node.equals(lockNode))
-      return false;
-    
-    Stat stat = new Stat();
-    return zc.get(lid.path + "/" + lid.node, stat) != null && 
stat.getEphemeralOwner() == lid.eid;
-  }
-  
-  public static byte[] getLockData(ZooKeeper zk, String path) throws 
KeeperException, InterruptedException {
-    List<String> children = zk.getChildren(path, false);
-    
-    if (children == null || children.size() == 0) {
-      return null;
-    }
-    
-    Collections.sort(children);
-    
-    String lockNode = children.get(0);
-    
-    return zk.getData(path + "/" + lockNode, false, null);
-  }
-  
-  public static byte[] getLockData(org.apache.accumulo.core.zookeeper.ZooCache 
zc, String path, Stat stat) {
-    
-    List<String> children = zc.getChildren(path);
-    
-    if (children == null || children.size() == 0) {
-      return null;
-    }
-    
-    children = new ArrayList<String>(children);
-    Collections.sort(children);
-    
-    String lockNode = children.get(0);
-    
-    if (!lockNode.startsWith(LOCK_PREFIX)) {
-      throw new RuntimeException("Node " + lockNode + " at " + path + " is not 
a lock node");
-    }
-    
-    return zc.get(path + "/" + lockNode, stat);
-  }
-  
-  private static ZooCache getLockDataZooCache = new ZooCache();
-  
-  public static byte[] getLockData(String path) {
-    return getLockData(path, null);
-  }
-
-  public static byte[] getLockData(String path, Stat stat) {
-    return getLockData(getLockDataZooCache, path, stat);
-  }
-  
-  public static long getSessionId(ZooCache zc, String path) throws 
KeeperException, InterruptedException {
-    List<String> children = zc.getChildren(path);
-    
-    if (children == null || children.size() == 0) {
-      return 0;
-    }
-    
-    children = new ArrayList<String>(children);
-    Collections.sort(children);
-    
-    String lockNode = children.get(0);
-    
-    Stat stat = new Stat();
-    if (zc.get(path + "/" + lockNode, stat) != null)
-      return stat.getEphemeralOwner();
-    return 0;
-  }
-  
-  public long getSessionId() throws KeeperException, InterruptedException {
-    return getSessionId(getLockDataZooCache, path);
+    super(new ZooCache(), ZooReaderWriter.getInstance(), path);
   }
   
   public static void deleteLock(String path) throws InterruptedException, 
KeeperException {
-    List<String> children;
-    
-    IZooReaderWriter zk = ZooReaderWriter.getInstance();
-    children = zk.getChildren(path);
-    
-    if (children == null || children.size() == 0) {
-      throw new IllegalStateException("No lock is held at " + path);
-    }
-    
-    Collections.sort(children);
-    
-    String lockNode = children.get(0);
-    
-    if (!lockNode.startsWith(LOCK_PREFIX)) {
-      throw new RuntimeException("Node " + lockNode + " at " + path + " is not 
a lock node");
-    }
-    
-    zk.recursiveDelete(path + "/" + lockNode, NodeMissingPolicy.SKIP);
-    
+    deleteLock(ZooReaderWriter.getInstance(), path);
   }
   
   public static boolean deleteLock(String path, String lockData) throws 
InterruptedException, KeeperException {
-    List<String> children;
-    
-    IZooReaderWriter zk = ZooReaderWriter.getInstance();
-    children = zk.getChildren(path);
-    
-    if (children == null || children.size() == 0) {
-      throw new IllegalStateException("No lock is held at " + path);
-    }
-    
-    Collections.sort(children);
-    
-    String lockNode = children.get(0);
-    
-    if (!lockNode.startsWith(LOCK_PREFIX)) {
-      throw new RuntimeException("Node " + lockNode + " at " + path + " is not 
a lock node");
-    }
-    
-    Stat stat = new Stat();
-    byte[] data = zk.getData(path + "/" + lockNode, stat);
-    
-    if (lockData.equals(new String(data))) {
-      zk.recursiveDelete(path + "/" + lockNode, stat.getVersion(), 
NodeMissingPolicy.FAIL);
-      return true;
-    }
-    
-    return false;
+    return deleteLock(ZooReaderWriter.getInstance(), path, lockData);
   }
   
   public static void main(String[] args) throws Exception {
@@ -484,5 +61,4 @@ public class ZooLock implements Watcher 
       Thread.sleep(1000);
     }
   }
-  
 }

Modified: 
accumulo/trunk/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooQueueLock.java
URL: 
http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooQueueLock.java?rev=1355481&r1=1355480&r2=1355481&view=diff
==============================================================================
--- 
accumulo/trunk/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooQueueLock.java
 (original)
+++ 
accumulo/trunk/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooQueueLock.java
 Fri Jun 29 17:42:35 2012
@@ -16,103 +16,16 @@
  */
 package org.apache.accumulo.server.zookeeper;
 
-import java.util.Collections;
-import java.util.List;
-import java.util.SortedMap;
-import java.util.TreeMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.Lock;
 
-import org.apache.accumulo.core.zookeeper.ZooUtil.NodeExistsPolicy;
-import org.apache.accumulo.core.zookeeper.ZooUtil.NodeMissingPolicy;
-import org.apache.accumulo.server.zookeeper.DistributedReadWriteLock.QueueLock;
+import org.apache.accumulo.fate.zookeeper.DistributedReadWriteLock;
 import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.NoNodeException;
-import org.apache.zookeeper.KeeperException.NotEmptyException;
 
-public class ZooQueueLock implements QueueLock {
-  
-  private static final String PREFIX = "lock-";
-  
-  // private static final Logger log = Logger.getLogger(ZooQueueLock.class);
-  
-  private IZooReaderWriter zoo;
-  private String path;
-  private boolean ephemeral;
+public class ZooQueueLock extends 
org.apache.accumulo.fate.zookeeper.ZooQueueLock {
   
   public ZooQueueLock(String path, boolean ephemeral) throws KeeperException, 
InterruptedException {
-    this.zoo = ZooReaderWriter.getRetryingInstance();
-    this.path = path;
-    this.ephemeral = ephemeral;
-  }
-  
-  @Override
-  public long addEntry(byte[] data) {
-    String newPath;
-    try {
-      while (true) {
-        try {
-          if (ephemeral) {
-            newPath = zoo.putEphemeralSequential(path + "/" + PREFIX, data);
-          } else {
-            newPath = zoo.putPersistentSequential(path + "/" + PREFIX, data);
-          }
-          String[] parts = newPath.split("/");
-          String last = parts[parts.length - 1];
-          return Long.parseLong(last.substring(PREFIX.length()));
-        } catch (NoNodeException nne) {
-          // the parent does not exist so try to create it
-          zoo.putPersistentData(path, new byte[] {}, NodeExistsPolicy.SKIP);
-        }
-      }
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
-    }
-  }
-  
-  @Override
-  public SortedMap<Long,byte[]> getEarlierEntries(long entry) {
-    SortedMap<Long,byte[]> result = new TreeMap<Long,byte[]>();
-    try {
-      List<String> children = Collections.emptyList();
-      try {
-        children = zoo.getChildren(path);
-      } catch (KeeperException.NoNodeException ex) {
-        // the path does not exist (it was deleted or not created yet), that 
is ok there are no earlier entries then
-      }
-      
-      for (String name : children) {
-        // this try catch must be done inside the loop because some subset of 
the children may exist
-        try {
-          byte[] data = zoo.getData(path + "/" + name, null);
-          long order = Long.parseLong(name.substring(PREFIX.length()));
-          if (order <= entry)
-            result.put(order, data);
-        } catch (KeeperException.NoNodeException ex) {
-          // ignored
-        }
-      }
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
-    }
-    return result;
-  }
-  
-  @Override
-  public void removeEntry(long entry) {
-    try {
-      zoo.recursiveDelete(path + String.format("/%s%010d", PREFIX, entry), 
NodeMissingPolicy.SKIP);
-      try {
-        // try to delete the parent if it has no children
-        zoo.delete(path, -1);
-      } catch (NotEmptyException nee) {
-        // the path had other lock nodes, no big deal
-      } catch (NoNodeException nne) {
-        // someone else deleted the lock path
-      }
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
-    }
+    super(ZooReaderWriter.getRetryingInstance(), path, ephemeral);
   }
   
   public static void main(String args[]) throws InterruptedException, 
KeeperException {

Modified: 
accumulo/trunk/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriter.java
URL: 
http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriter.java?rev=1355481&r1=1355480&r2=1355481&view=diff
==============================================================================
--- 
accumulo/trunk/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriter.java
 (original)
+++ 
accumulo/trunk/server/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriter.java
 Fri Jun 29 17:42:35 2012
@@ -1,4 +1,4 @@
-/*
+/**
  * 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.
@@ -20,125 +20,21 @@ import java.lang.reflect.InvocationHandl
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.lang.reflect.Proxy;
-import java.security.SecurityPermission;
-import java.util.List;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.core.zookeeper.ZooReader;
-import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.apache.accumulo.core.zookeeper.ZooUtil.NodeExistsPolicy;
-import org.apache.accumulo.core.zookeeper.ZooUtil.NodeMissingPolicy;
+import org.apache.accumulo.fate.util.UtilWaitThread;
+import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.log4j.Logger;
-import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.BadVersionException;
-import org.apache.zookeeper.KeeperException.NodeExistsException;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.data.ACL;
-import org.apache.zookeeper.data.Stat;
 
-public class ZooReaderWriter extends ZooReader implements IZooReaderWriter {
-  
-  private static SecurityPermission ZOOWRITER_PERMISSION = new 
SecurityPermission("zookeeperWriterPermission");
-  
+public class ZooReaderWriter extends 
org.apache.accumulo.fate.zookeeper.ZooReaderWriter {
   private static ZooReaderWriter instance = null;
   private static IZooReaderWriter retryingInstance = null;
-  private final String auth;
-  
-  @Override
-  public ZooKeeper getZooKeeper() {
-    SecurityManager sm = System.getSecurityManager();
-    if (sm != null) {
-      sm.checkPermission(ZOOWRITER_PERMISSION);
-    }
-    return getSession(keepers, timeout, auth);
-  }
   
   public ZooReaderWriter(String string, int timeInMillis, String auth) {
-    super(string, timeInMillis);
-    this.auth = "accumulo:" + auth;
-  }
-  
-  @Override
-  public void recursiveDelete(String zPath, NodeMissingPolicy policy) throws 
KeeperException, InterruptedException {
-    ZooUtil.recursiveDelete(getZooKeeper(), zPath, policy);
-  }
-  
-  @Override
-  public void recursiveDelete(String zPath, int version, NodeMissingPolicy 
policy) throws KeeperException, InterruptedException {
-    ZooUtil.recursiveDelete(getZooKeeper(), zPath, version, policy);
-  }
-  
-  /**
-   * Create a persistent node with the default ACL
-   * 
-   * @return true if the node was created or altered; false if it was skipped
-   */
-  @Override
-  public boolean putPersistentData(String zPath, byte[] data, NodeExistsPolicy 
policy) throws KeeperException, InterruptedException {
-    return ZooUtil.putPersistentData(getZooKeeper(), zPath, data, policy);
-  }
-  
-  @Override
-  public boolean putPrivatePersistentData(String zPath, byte[] data, 
NodeExistsPolicy policy) throws KeeperException, InterruptedException {
-    return ZooUtil.putPrivatePersistentData(getZooKeeper(), zPath, data, 
policy);
-  }
-  
-  @Override
-  public void putPersistentData(String zPath, byte[] data, int version, 
NodeExistsPolicy policy) throws KeeperException, InterruptedException {
-    ZooUtil.putPersistentData(getZooKeeper(), zPath, data, version, policy);
-  }
-  
-  @Override
-  public String putPersistentSequential(String zPath, byte[] data) throws 
KeeperException, InterruptedException {
-    return ZooUtil.putPersistentSequential(getZooKeeper(), zPath, data);
-  }
-  
-  @Override
-  public String putEphemeralSequential(String zPath, byte[] data) throws 
KeeperException, InterruptedException {
-    return ZooUtil.putEphemeralSequential(getZooKeeper(), zPath, data);
-  }
-  
-  @Override
-  public void recursiveCopyPersistent(String source, String destination, 
NodeExistsPolicy policy) throws KeeperException, InterruptedException {
-    ZooUtil.recursiveCopyPersistent(getZooKeeper(), source, destination, 
policy);
-  }
-  
-  @Override
-  public void delete(String path, int version) throws InterruptedException, 
KeeperException {
-    getZooKeeper().delete(path, version);
-  }
-  
-  public interface Mutator {
-    byte[] mutate(byte[] currentValue) throws Exception;
-  }
-  
-  @Override
-  public byte[] mutate(String zPath, byte[] createValue, List<ACL> acl, 
Mutator mutator) throws Exception {
-    if (createValue != null) {
-      try {
-        getZooKeeper().create(zPath, createValue, acl, CreateMode.PERSISTENT);
-        return createValue;
-      } catch (NodeExistsException ex) {
-        // expected
-      }
-    }
-    do {
-      Stat stat = new Stat();
-      byte[] data = getZooKeeper().getData(zPath, false, stat);
-      data = mutator.mutate(data);
-      if (data == null)
-        return data;
-      try {
-        getZooKeeper().setData(zPath, data, stat.getVersion());
-        return data;
-      } catch (BadVersionException ex) {
-        //
-      }
-    } while (true);
+    super(string, timeInMillis, auth);
   }
   
   public static synchronized ZooReaderWriter getInstance() {
@@ -185,23 +81,4 @@ public class ZooReaderWriter extends Zoo
     
     return retryingInstance;
   }
-  
-  @Override
-  public boolean isLockHeld(ZooUtil.LockID lockID) throws KeeperException, 
InterruptedException {
-    return ZooUtil.isLockHeld(getZooKeeper(), lockID);
-  }
-  
-  @Override
-  public void mkdirs(String path) throws KeeperException, InterruptedException 
{
-    if (path.equals(""))
-      return;
-    if (!path.startsWith("/"))
-      throw new IllegalArgumentException(path + "does not start with /");
-    if (getZooKeeper().exists(path, false) != null)
-      return;
-    String parent = path.substring(0, path.lastIndexOf("/"));
-    mkdirs(parent);
-    putPersistentData(path, new byte[] {}, NodeExistsPolicy.SKIP);
-  }
-  
 }

Modified: accumulo/trunk/test/system/auto/simple/examples.py
URL: 
http://svn.apache.org/viewvc/accumulo/trunk/test/system/auto/simple/examples.py?rev=1355481&r1=1355480&r2=1355481&view=diff
==============================================================================
--- accumulo/trunk/test/system/auto/simple/examples.py (original)
+++ accumulo/trunk/test/system/auto/simple/examples.py Fri Jun 29 17:42:35 2012
@@ -78,14 +78,14 @@ class Examples(TestUtilsMixin, unittest.
                      'dataTable',
                      visibility,
                      100000,
-                     ACCUMULO_HOME+"/server")
+                     ACCUMULO_HOME+"/fate")
         self.comment("  searching for a file")
         handle = self.runOn('localhost', [self.accumulo_sh(), 
'org.apache.accumulo.examples.simple.dirlist.QueryUtil',
                                           INSTANCE_NAME, ZOOKEEPERS, ROOT, 
ROOT_PASSWORD,
                                           'indexTable', auths, 'Fate.java', 
'-search'])
         out, err = handle.communicate()
         self.assert_(handle.returncode == 0)
-        self.assert_(out.find('accumulo/server/fate/Fate.java') >= 0)
+        self.assert_(out.find('accumulo/fate/Fate.java') >= 0)
         self.comment("  found file at " + out)
 
     

Modified: accumulo/trunk/test/system/auto/simple/mapreduce.py
URL: 
http://svn.apache.org/viewvc/accumulo/trunk/test/system/auto/simple/mapreduce.py?rev=1355481&r1=1355480&r2=1355481&view=diff
==============================================================================
--- accumulo/trunk/test/system/auto/simple/mapreduce.py (original)
+++ accumulo/trunk/test/system/auto/simple/mapreduce.py Fri Jun 29 17:42:35 2012
@@ -68,6 +68,7 @@ class MapReduceTest(TestUtilsMixin,unitt
         thriftjar = globa(os.path.join('lib','libthrift*.jar'))
         examples = 
globa(os.path.join('lib','examples-simple*[!javadoc|sources].jar'))
         core = 
globa(os.path.join('lib','accumulo-core*[!javadoc|sources].jar'))
+        fate = 
globa(os.path.join('lib','accumulo-fate*[!javadoc|sources].jar'))
         start = 
globa(os.path.join('lib','accumulo-start*[!javadoc|sources].jar'))
         trace = globa(os.path.join('lib','cloudtrace*[!javadoc|sources].jar'))
         zkjar = 
globbase(os.getenv("ZOOKEEPER_HOME"),"zookeeper*[!javadoc|src|bin].jar")
@@ -83,7 +84,7 @@ class MapReduceTest(TestUtilsMixin,unitt
         #MapReduce class to run
         mapred_class= [self.accumulo_sh(),self.example_class_to_run]
         #classes needed to run the mapreduce
-        libjars = ["-libjars",",".join([zkjar,thriftjar,examples,core,trace])]
+        libjars = 
["-libjars",",".join([zkjar,thriftjar,examples,core,fate,trace])]
         cmd = mapred_class+libjars+arg_list
         if(self.isAccumuloRunning()):
             log.debug("COMMAND:"+str(cmd))


Reply via email to