You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by vi...@apache.org on 2012/07/18 20:45:17 UTC

svn commit: r1363041 - in /accumulo/branches/ACCUMULO-259: ./ bin/ core/ core/src/main/java/org/apache/accumulo/core/client/admin/ core/src/main/java/org/apache/accumulo/core/file/blockfile/ core/src/main/java/org/apache/accumulo/core/file/blockfile/ca...

Author: vines
Date: Wed Jul 18 18:45:16 2012
New Revision: 1363041

URL: http://svn.apache.org/viewvc?rev=1363041&view=rev
Log:
Accumulo-259 - some random walk test fixes found, also merging


Added:
    accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/rfile/BlockIndex.java
      - copied unchanged from r1362262, accumulo/trunk/core/src/main/java/org/apache/accumulo/core/file/rfile/BlockIndex.java
Modified:
    accumulo/branches/ACCUMULO-259/   (props changed)
    accumulo/branches/ACCUMULO-259/README
    accumulo/branches/ACCUMULO-259/bin/start-all.sh
    accumulo/branches/ACCUMULO-259/core/   (props changed)
    accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
    accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/ABlockReader.java
    accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/BlockCache.java
    accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/CacheEntry.java
    accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/CachedBlock.java
    accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/LruBlockCache.java
    accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/SimpleBlockCache.java
    accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/CachableBlockFile.java
    accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java
    accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java
    accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
    accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/file/blockfile/cache/TestLruBlockCache.java
    accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java
    accumulo/branches/ACCUMULO-259/server/   (props changed)
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/SecurityOperationImpl.java
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/AlterTable.java
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/AlterTablePerm.java
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/Authenticate.java
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/ChangePass.java
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/DropTable.java
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/SecurityHelper.java
    accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/TableOp.java
    accumulo/branches/ACCUMULO-259/src/   (props changed)
    accumulo/branches/ACCUMULO-259/test/system/auto/simple/shell.py

Propchange: accumulo/branches/ACCUMULO-259/
------------------------------------------------------------------------------
  Merged /accumulo/trunk:r1361355-1362560

Modified: accumulo/branches/ACCUMULO-259/README
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/README?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/README (original)
+++ accumulo/branches/ACCUMULO-259/README Wed Jul 18 18:45:16 2012
@@ -62,8 +62,12 @@ the machines in the cluster and that had
 found in the same location on every machine in the cluster.  You will need to
 have password-less ssh set up as described in the hadoop documentation. 
 
-You will need to have hadoop installed and configured on your system.
-Accumulo 1.5.0-SNAPSHOT has been tested with hadoop version 0.20.2.
+You will need to have hadoop installed and configured on your system.  Accumulo
+1.5.0-SNAPSHOT has been tested with hadoop version 0.20.2.  To avoid data loss,
+you must enable HDFS durable sync.  How you enable this depends on your version
+of Hadoop.  For older versions of Hadoop set dfs.support.append to true in
+hdfs-site.xml.  For newer versions set dfs.durable.sync to true.  After setting
+these properties restart HDFS.  See ACCUMULO-623 for more information.
 
 The example accumulo configuration files are placed in directories based on the 
 memory footprint for the accumulo processes.  If you are using native libraries

Modified: accumulo/branches/ACCUMULO-259/bin/start-all.sh
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/bin/start-all.sh?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/bin/start-all.sh (original)
+++ accumulo/branches/ACCUMULO-259/bin/start-all.sh Wed Jul 18 18:45:16 2012
@@ -42,6 +42,8 @@ if [ "$ZOOKEEPER_VERSION" '<' "3.3.0" ] 
 	echo "WARN : Using Zookeeper $ZOOKEEPER_VERSION.  Use version 3.3.0 or greater to avoid zookeeper deadlock bug.";
 fi
 
+${bin}/start-server.sh $MONITOR monitor 
+
 if [ "$1" != "--notSlaves" ] ; then
 	${bin}/tup.sh
 fi
@@ -54,7 +56,6 @@ done
 
 ${bin}/start-server.sh $GC gc "garbage collector"
 
-${bin}/start-server.sh $MONITOR monitor 
 
 for tracer in `grep -v '^#' "$ACCUMULO_HOME/conf/tracers"`
 do

Propchange: accumulo/branches/ACCUMULO-259/core/
------------------------------------------------------------------------------
  Merged /accumulo/trunk/core:r1361355-1362560

Modified: accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java Wed Jul 18 18:45:16 2012
@@ -102,7 +102,7 @@ public class TableOperationsImpl extends
   private Instance instance;
   private AuthInfo credentials;
   
-  private static final Logger log = Logger.getLogger(TableOperations.class);
+  private static final Logger log = Logger.getLogger(TableOperationsImpl.class);
   
   /**
    * @param instance

Modified: accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/ABlockReader.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/ABlockReader.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/ABlockReader.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/ABlockReader.java Wed Jul 18 18:45:16 2012
@@ -34,4 +34,13 @@ public interface ABlockReader extends Da
   
   public void close() throws IOException;
   
+  /**
+   * An indexable block supports seeking, getting a position, and associating an arbitrary index with the block
+   * 
+   * @return
+   */
+  public boolean isIndexable();
+  public void seek(int position);
+  public int getPosition();
+  <T> T getIndex(Class<T> clazz);
 }

Modified: accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/BlockCache.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/BlockCache.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/BlockCache.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/BlockCache.java Wed Jul 18 18:45:16 2012
@@ -33,7 +33,7 @@ public interface BlockCache {
    * @param inMemory
    *          Whether block should be treated as in-memory
    */
-  public void cacheBlock(String blockName, byte buf[], boolean inMemory);
+  public CacheEntry cacheBlock(String blockName, byte buf[], boolean inMemory);
   
   /**
    * Add block to cache (defaults to not in-memory).
@@ -43,7 +43,7 @@ public interface BlockCache {
    * @param buf
    *          The block contents wrapped in a ByteBuffer.
    */
-  public void cacheBlock(String blockName, byte buf[]);
+  public CacheEntry cacheBlock(String blockName, byte buf[]);
   
   /**
    * Fetch block from cache.
@@ -52,10 +52,15 @@ public interface BlockCache {
    *          Block number to fetch.
    * @return Block or null if block is not in the cache.
    */
-  public byte[] getBlock(String blockName);
+  public CacheEntry getBlock(String blockName);
   
   /**
    * Shutdown the cache.
    */
   public void shutdown();
+  
+  /**
+   * @return
+   */
+  public long getMaxSize();
 }

Modified: accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/CacheEntry.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/CacheEntry.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/CacheEntry.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/CacheEntry.java Wed Jul 18 18:45:16 2012
@@ -16,40 +16,11 @@
  */
 package org.apache.accumulo.core.file.blockfile.cache;
 
-public class CacheEntry {
-  private String fName;
-  private Long hash;
+public interface CacheEntry {
+  byte[] getBuffer();
   
-  public CacheEntry(String name, Long time) {
-    this.hash = time;
-    this.fName = name;
-  }
+  public Object getIndex();
   
-  @Override
-  public boolean equals(Object other) {
-    return
-    
-    ((CacheEntry) other).getName().equals(fName) && ((CacheEntry) other).getHashInfo().equals(hash) && ((CacheEntry) other).getName().equals(fName)
-        && ((CacheEntry) other).getHashInfo().equals(hash);
-    
-  }
-  
-  @Override
-  public int hashCode() {
-    return fName.hashCode() + hash.hashCode();
-  }
-  
-  public String getName() {
-    return fName;
-  }
-  
-  public Long getHashInfo() {
-    
-    return this.hash;
-  }
-  
-  public long length() {
-    return fName.length() + Long.SIZE;
-  }
+  public void setIndex(Object idx);
   
 }

Modified: accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/CachedBlock.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/CachedBlock.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/CachedBlock.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/CachedBlock.java Wed Jul 18 18:45:16 2012
@@ -19,6 +19,7 @@
  */
 package org.apache.accumulo.core.file.blockfile.cache;
 
+
 /**
  * Represents an entry in the {@link LruBlockCache}.
  * 
@@ -26,7 +27,7 @@ package org.apache.accumulo.core.file.bl
  * Makes the block memory-aware with {@link HeapSize} and Comparable to sort by access time for the LRU. It also takes care of priority by either instantiating
  * as in-memory or handling the transition from single to multiple access.
  */
-public class CachedBlock implements HeapSize, Comparable<CachedBlock> {
+public class CachedBlock implements HeapSize, Comparable<CachedBlock>, CacheEntry {
   
   public final static long PER_BLOCK_OVERHEAD = ClassSize.align(ClassSize.OBJECT + (3 * ClassSize.REFERENCE) + (2 * SizeConstants.SIZEOF_LONG)
       + ClassSize.STRING + ClassSize.BYTE_BUFFER);
@@ -51,6 +52,7 @@ public class CachedBlock implements Heap
   private volatile long accessTime;
   private long size;
   private BlockPriority priority;
+  private Object index;
   
   public CachedBlock(String blockName, byte buf[], long accessTime) {
     this(blockName, buf, accessTime, false);
@@ -88,6 +90,7 @@ public class CachedBlock implements Heap
     return this.accessTime < that.accessTime ? 1 : -1;
   }
   
+  @Override
   public byte[] getBuffer() {
     return this.buf;
   }
@@ -99,4 +102,14 @@ public class CachedBlock implements Heap
   public BlockPriority getPriority() {
     return this.priority;
   }
+  
+  @Override
+  public Object getIndex() {
+    return index;
+  }
+  
+  @Override
+  public void setIndex(Object idx) {
+    this.index = idx;
+  }
 }

Modified: accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/LruBlockCache.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/LruBlockCache.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/LruBlockCache.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/LruBlockCache.java Wed Jul 18 18:45:16 2012
@@ -247,7 +247,7 @@ public class LruBlockCache implements Bl
    * @param inMemory
    *          if block is in-memory
    */
-  public void cacheBlock(String blockName, byte buf[], boolean inMemory) {
+  public CacheEntry cacheBlock(String blockName, byte buf[], boolean inMemory) {
     CachedBlock cb = map.get(blockName);
     if (cb != null) {
       stats.duplicateReads();
@@ -262,6 +262,8 @@ public class LruBlockCache implements Bl
         runEviction();
       }
     }
+    
+    return cb;
   }
   
   /**
@@ -275,8 +277,8 @@ public class LruBlockCache implements Bl
    * @param buf
    *          block buffer
    */
-  public void cacheBlock(String blockName, byte buf[]) {
-    cacheBlock(blockName, buf, false);
+  public CacheEntry cacheBlock(String blockName, byte buf[]) {
+    return cacheBlock(blockName, buf, false);
   }
   
   /**
@@ -286,7 +288,8 @@ public class LruBlockCache implements Bl
    *          block name
    * @return buffer of specified block name, or null if not in cache
    */
-  public byte[] getBlock(String blockName) {
+  
+  public CachedBlock getBlock(String blockName) {
     CachedBlock cb = map.get(blockName);
     if (cb == null) {
       stats.miss();
@@ -294,7 +297,7 @@ public class LruBlockCache implements Bl
     }
     stats.hit();
     cb.access(count.incrementAndGet());
-    return cb.getBuffer();
+    return cb;
   }
   
   protected long evictBlock(CachedBlock block) {

Modified: accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/SimpleBlockCache.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/SimpleBlockCache.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/SimpleBlockCache.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/SimpleBlockCache.java Wed Jul 18 18:45:16 2012
@@ -27,18 +27,45 @@ import java.util.Map;
  * Simple one RFile soft reference cache.
  */
 public class SimpleBlockCache implements BlockCache {
-  private static class Ref extends SoftReference<byte[]> {
+  
+  private static class SimpleCacheEntry implements CacheEntry {
+    
+    private byte[] buffer;
+    private Object index;
+    
+    SimpleCacheEntry(byte[] buffer) {
+      this.buffer = buffer;
+    }
+    
+    @Override
+    public byte[] getBuffer() {
+      return buffer;
+    }
+    
+    @Override
+    public Object getIndex() {
+      return index;
+    }
+    
+    @Override
+    public void setIndex(Object idx) {
+      this.index = idx;
+    }
+    
+  }
+  
+  private static class Ref extends SoftReference<SimpleCacheEntry> {
     public String blockId;
     
-    public Ref(String blockId, byte buf[], ReferenceQueue<byte[]> q) {
-      super(buf, q);
+    public Ref(String blockId, SimpleCacheEntry sce, ReferenceQueue<SimpleCacheEntry> q) {
+      super(sce, q);
       this.blockId = blockId;
     }
   }
   
   private Map<String,Ref> cache = new HashMap<String,Ref>();
   
-  private ReferenceQueue<byte[]> q = new ReferenceQueue<byte[]>();
+  private ReferenceQueue<SimpleCacheEntry> q = new ReferenceQueue<SimpleCacheEntry>();
   public int dumps = 0;
   
   /**
@@ -64,7 +91,7 @@ public class SimpleBlockCache implements
     return cache.size();
   }
   
-  public synchronized byte[] getBlock(String blockName) {
+  public synchronized SimpleCacheEntry getBlock(String blockName) {
     processQueue(); // clear out some crap.
     Ref ref = cache.get(blockName);
     if (ref == null)
@@ -72,15 +99,24 @@ public class SimpleBlockCache implements
     return ref.get();
   }
   
-  public synchronized void cacheBlock(String blockName, byte buf[]) {
-    cache.put(blockName, new Ref(blockName, buf, q));
+  public synchronized SimpleCacheEntry cacheBlock(String blockName, byte buf[]) {
+    SimpleCacheEntry sce = new SimpleCacheEntry(buf);
+    cache.put(blockName, new Ref(blockName, sce, q));
+    return sce;
   }
   
-  public synchronized void cacheBlock(String blockName, byte buf[], boolean inMemory) {
-    cache.put(blockName, new Ref(blockName, buf, q));
+  public synchronized SimpleCacheEntry cacheBlock(String blockName, byte buf[], boolean inMemory) {
+    SimpleCacheEntry sce = new SimpleCacheEntry(buf);
+    cache.put(blockName, new Ref(blockName, sce, q));
+    return sce;
   }
   
   public void shutdown() {
     // noop
   }
+  
+  @Override
+  public long getMaxSize() {
+    return Long.MAX_VALUE;
+  }
 }

Modified: accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/CachableBlockFile.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/CachableBlockFile.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/CachableBlockFile.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/CachableBlockFile.java Wed Jul 18 18:45:16 2012
@@ -21,13 +21,14 @@ import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.lang.ref.SoftReference;
 
 import org.apache.accumulo.core.file.blockfile.ABlockReader;
 import org.apache.accumulo.core.file.blockfile.ABlockWriter;
 import org.apache.accumulo.core.file.blockfile.BlockFileReader;
 import org.apache.accumulo.core.file.blockfile.BlockFileWriter;
 import org.apache.accumulo.core.file.blockfile.cache.BlockCache;
-import org.apache.accumulo.core.file.blockfile.cache.LruBlockCache;
+import org.apache.accumulo.core.file.blockfile.cache.CacheEntry;
 import org.apache.accumulo.core.file.rfile.bcfile.BCFile;
 import org.apache.accumulo.core.file.rfile.bcfile.BCFile.Reader.BlockReader;
 import org.apache.accumulo.core.file.rfile.bcfile.BCFile.Writer.BlockAppender;
@@ -142,8 +143,8 @@ public class CachableBlockFile {
   public static class Reader implements BlockFileReader {
     private BCFile.Reader _bc;
     private String fileName = "not_available";
-    private LruBlockCache _dCache = null;
-    private LruBlockCache _iCache = null;
+    private BlockCache _dCache = null;
+    private BlockCache _iCache = null;
     private FSDataInputStream fin = null;
     private FileSystem fs;
     private Configuration conf;
@@ -224,12 +225,18 @@ public class CachableBlockFile {
        */
       
       fileName = dataFile.toString();
-      this._dCache = (LruBlockCache) data;
-      this._iCache = (LruBlockCache) index;
+      this._dCache = data;
+      this._iCache = index;
       this.fs = fs;
       this.conf = conf;
     }
     
+    public Reader(FSDataInputStream fsin, long len, Configuration conf, BlockCache data, BlockCache index) throws IOException {
+      this._dCache = data;
+      this._iCache = index;
+      init(fsin, len, conf);
+    }
+
     public Reader(FSDataInputStream fsin, long len, Configuration conf) throws IOException {
       // this.fin = fsin;
       init(fsin, len, conf);
@@ -255,13 +262,12 @@ public class CachableBlockFile {
     
     public BlockRead getCachedMetaBlock(String blockName) throws IOException {
       String _lookup = fileName + "M" + blockName;
-      byte b[] = null;
       
       if (_iCache != null) {
-        b = _iCache.getBlock(_lookup);
+        CacheEntry cacheEntry = _iCache.getBlock(_lookup);
         
-        if (b != null) {
-          return new BlockRead(new DataInputStream(new ByteArrayInputStream(b)), b.length);
+        if (cacheEntry != null) {
+          return new CachedBlockRead(cacheEntry, cacheEntry.getBuffer());
         }
         
       }
@@ -287,16 +293,16 @@ public class CachableBlockFile {
       }
     }
     
-    private BlockRead getBlock(String _lookup, LruBlockCache cache, BlockLoader loader) throws IOException {
+    private BlockRead getBlock(String _lookup, BlockCache cache, BlockLoader loader) throws IOException {
       
       BlockReader _currBlock;
       
       if (cache != null) {
-        byte b[] = null;
-        b = cache.getBlock(_lookup);
+        CacheEntry cb = null;
+        cb = cache.getBlock(_lookup);
         
-        if (b != null) {
-          return new BlockRead(new DataInputStream(new ByteArrayInputStream(b)), b.length);
+        if (cb != null) {
+          return new CachedBlockRead(cb, cb.getBuffer());
         }
         
       }
@@ -313,7 +319,7 @@ public class CachableBlockFile {
       
     }
     
-    private BlockRead cacheBlock(String _lookup, LruBlockCache cache, BlockReader _currBlock, String block) throws IOException {
+    private BlockRead cacheBlock(String _lookup, BlockCache cache, BlockReader _currBlock, String block) throws IOException {
       
       if ((cache == null) || (_currBlock.getRawSize() > cache.getMaxSize())) {
         return new BlockRead(_currBlock, _currBlock.getRawSize());
@@ -334,13 +340,17 @@ public class CachableBlockFile {
           _currBlock.close();
         }
         
+        CacheEntry ce = null;
         try {
-          cache.cacheBlock(_lookup, b);
+          ce = cache.cacheBlock(_lookup, b);
         } catch (Exception e) {
           log.warn("Already cached block: " + _lookup, e);
         }
         
-        return new BlockRead(new DataInputStream(new ByteArrayInputStream(b)), b.length);
+        if (ce == null)
+          return new BlockRead(new DataInputStream(new ByteArrayInputStream(b)), b.length);
+        else
+          return new CachedBlockRead(ce, ce.getBuffer());
         
       }
     }
@@ -399,6 +409,82 @@ public class CachableBlockFile {
     
   }
   
+  static class SeekableByteArrayInputStream extends ByteArrayInputStream {
+    
+    public SeekableByteArrayInputStream(byte[] buf) {
+      super(buf);
+    }
+    
+    public SeekableByteArrayInputStream(byte buf[], int offset, int length) {
+      super(buf, offset, length);
+      throw new UnsupportedOperationException("Seek code assumes offset is zero"); // do not need this constructor, documenting that seek will not work
+                                                                                  // unless offset it kept track of
+    }
+    
+    public void seek(int position) {
+      if (pos < 0 || pos >= buf.length)
+        throw new IllegalArgumentException("pos = " + pos + " buf.lenght = " + buf.length);
+      this.pos = position;
+    }
+    
+    public int getPosition() {
+      return this.pos;
+    }
+    
+  }
+
+  public static class CachedBlockRead extends BlockRead {
+    private SeekableByteArrayInputStream seekableInput;
+    private CacheEntry cb;
+    
+    public CachedBlockRead(CacheEntry cb, byte buf[]) {
+      this(new SeekableByteArrayInputStream(buf), buf.length);
+      this.cb = cb;
+    }
+    
+    private CachedBlockRead(SeekableByteArrayInputStream seekableInput, long size) {
+      super(seekableInput, size);
+      this.seekableInput = seekableInput;
+    }
+
+    @Override
+    public void seek(int position) {
+      seekableInput.seek(position);
+    }
+    
+    @Override
+    public int getPosition() {
+      return seekableInput.getPosition();
+    }
+    
+    @Override
+    public boolean isIndexable() {
+      return true;
+    }
+    
+    @Override
+    public <T> T getIndex(Class<T> clazz) {
+      T bi = null;
+      synchronized (cb) {
+        @SuppressWarnings("unchecked")
+        SoftReference<T> softRef = (SoftReference<T>) cb.getIndex();
+        if (softRef != null)
+          bi = softRef.get();
+        
+        if (bi == null) {
+          try {
+            bi = clazz.newInstance();
+          } catch (Exception e) {
+            throw new RuntimeException(e);
+          }
+          cb.setIndex(new SoftReference<T>(bi));
+        }
+      }
+      
+      return bi;
+    }
+  }
+
   /**
    * 
    * Class provides functionality to read one block from the underlying BCFile Since We are caching blocks in the Reader class as bytearrays, this class will
@@ -430,5 +516,25 @@ public class CachableBlockFile {
       return this;
     }
     
+    @Override
+    public boolean isIndexable() {
+      return false;
+    }
+    
+    @Override
+    public void seek(int position) {
+      throw new UnsupportedOperationException();
+    }
+    
+    @Override
+    public int getPosition() {
+      throw new UnsupportedOperationException();
+    }
+    
+    @Override
+    public <T> T getIndex(Class<T> clazz) {
+      throw new UnsupportedOperationException();
+    }
+    
   }
 }

Modified: accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java Wed Jul 18 18:45:16 2012
@@ -52,6 +52,7 @@ import org.apache.accumulo.core.file.blo
 import org.apache.accumulo.core.file.blockfile.BlockFileReader;
 import org.apache.accumulo.core.file.blockfile.BlockFileWriter;
 import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;
+import org.apache.accumulo.core.file.rfile.BlockIndex.BlockIndexEntry;
 import org.apache.accumulo.core.file.rfile.MultiLevelIndex.IndexEntry;
 import org.apache.accumulo.core.file.rfile.MultiLevelIndex.Reader.IndexIterator;
 import org.apache.accumulo.core.file.rfile.RelativeKey.MByteSequence;
@@ -670,6 +671,12 @@ public class RFile {
         if (startKey.compareTo(getTopKey()) >= 0 && startKey.compareTo(iiter.peekPrevious().getKey()) <= 0) {
           // start key is within the unconsumed portion of the current block
           
+          // this code intentionally does not use the index associated with a cached block
+          // because if only forward seeks are being done, then there is no benefit to building
+          // and index for the block... could consider using the index if it exist but not
+          // causing the build of an index... doing this could slow down some use cases and
+          // and speed up others.
+
           MByteSequence valbs = new MByteSequence(new byte[64], 0, 0);
           RelativeKey tmpRk = new RelativeKey();
           Key pKey = new Key(getTopKey());
@@ -717,9 +724,35 @@ public class RFile {
           entriesLeft = indexEntry.getNumEntries();
           currBlock = getDataBlock(indexEntry);
 
-          MByteSequence valbs = new MByteSequence(new byte[64], 0, 0);
           RelativeKey tmpRk = new RelativeKey();
-          fastSkipped = tmpRk.fastSkip(currBlock, startKey, valbs, prevKey, null);
+          MByteSequence valbs = new MByteSequence(new byte[64], 0, 0);
+
+          Key currKey = null;
+
+          if (currBlock.isIndexable()) {
+            BlockIndex blockIndex = BlockIndex.getIndex(currBlock, indexEntry);
+            if (blockIndex != null) {
+              BlockIndexEntry bie = blockIndex.seekBlock(startKey, currBlock);
+              if (bie != null) {
+                // we are seeked to the current position of the key in the index
+                // need to prime the read process and read this key from the block
+                tmpRk.setPrevKey(bie.getKey());
+                tmpRk.readFields(currBlock);
+                val = new Value();
+
+                val.readFields(currBlock);
+                valbs = new MByteSequence(val.get(), 0, val.getSize());
+                
+                // just consumed one key from the input stream, so subtract one from entries left
+                entriesLeft = bie.getEntriesLeft() - 1;
+                prevKey = new Key(bie.getKey());
+                currKey = bie.getKey();
+              }
+            }
+            
+          }
+
+          fastSkipped = tmpRk.fastSkip(currBlock, startKey, valbs, prevKey, currKey);
           entriesLeft -= fastSkipped;
           val = new Value(valbs.toArray());
           // set rk when everything above is successful, if exception
@@ -789,7 +822,7 @@ public class RFile {
     
     private AtomicBoolean interruptFlag;
     
-    Reader(BlockFileReader rdr) throws IOException {
+    public Reader(BlockFileReader rdr) throws IOException {
       this.reader = rdr;
       
       ABlockReader mb = reader.getMetaBlock("RFile.index");

Modified: accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java Wed Jul 18 18:45:16 2012
@@ -111,6 +111,10 @@ public class RelativeKey implements Writ
       fieldsSame |= DELETED;
   }
   
+  public void setPrevKey(Key pk) {
+    this.prevKey = pk;
+  }
+  
   @Override
   public void readFields(DataInput in) throws IOException {
     fieldsSame = in.readByte();

Modified: accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java Wed Jul 18 18:45:16 2012
@@ -268,7 +268,7 @@ public class Shell extends ShellOptions 
       this.setTableName("");
       connector = instance.getConnector(user, pass);
       this.credentials = new AuthInfo(user, ByteBuffer.wrap(pass), connector.getInstance().getInstanceID());
-      
+      updateUser(credentials);
     } catch (Exception e) {
       printException(e);
       configError = true;
@@ -911,6 +911,8 @@ public class Shell extends ShellOptions 
   public void updateUser(AuthInfo authInfo) throws AccumuloException, AccumuloSecurityException {
     connector = instance.getConnector(authInfo);
     credentials = authInfo;
+    if (!connector.securityOperations().authenticateUser(authInfo.user, authInfo.getPassword()))
+      throw new RuntimeException("Unable to authenticate user " + authInfo.user);
   }
   
   public AuthInfo getCredentials() {

Modified: accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/file/blockfile/cache/TestLruBlockCache.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/file/blockfile/cache/TestLruBlockCache.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/file/blockfile/cache/TestLruBlockCache.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/file/blockfile/cache/TestLruBlockCache.java Wed Jul 18 18:45:16 2012
@@ -83,9 +83,9 @@ public class TestLruBlockCache extends T
     
     // Check if all blocks are properly cached and retrieved
     for (Block block : blocks) {
-      byte buf1[] = cache.getBlock(block.blockName);
-      assertTrue(buf1 != null);
-      assertEquals(buf1.length, block.buf.length);
+      CacheEntry ce = cache.getBlock(block.blockName);
+      assertTrue(ce != null);
+      assertEquals(ce.getBuffer().length, block.buf.length);
     }
     
     // Verify correctly calculated cache heap size
@@ -93,9 +93,9 @@ public class TestLruBlockCache extends T
     
     // Check if all blocks are properly cached and retrieved
     for (Block block : blocks) {
-      byte buf1[] = cache.getBlock(block.blockName);
-      assertTrue(buf1 != null);
-      assertEquals(buf1.length, block.buf.length);
+      CacheEntry ce = cache.getBlock(block.blockName);
+      assertTrue(ce != null);
+      assertEquals(ce.getBuffer().length, block.buf.length);
     }
     
     // Expect no evictions
@@ -138,7 +138,7 @@ public class TestLruBlockCache extends T
     assertTrue(cache.getBlock(blocks[0].blockName) == null);
     assertTrue(cache.getBlock(blocks[1].blockName) == null);
     for (int i = 2; i < blocks.length; i++) {
-      assertEquals(cache.getBlock(blocks[i].blockName), blocks[i].buf);
+      assertEquals(cache.getBlock(blocks[i].blockName).getBuffer(), blocks[i].buf);
     }
   }
   
@@ -163,7 +163,7 @@ public class TestLruBlockCache extends T
     for (Block block : multiBlocks) {
       cache.cacheBlock(block.blockName, block.buf);
       expectedCacheSize += block.heapSize();
-      assertEquals(cache.getBlock(block.blockName), block.buf);
+      assertEquals(cache.getBlock(block.blockName).getBuffer(), block.buf);
     }
     
     // Add the single blocks (no get)
@@ -196,8 +196,8 @@ public class TestLruBlockCache extends T
     
     // And all others to be cached
     for (int i = 1; i < 4; i++) {
-      assertEquals(cache.getBlock(singleBlocks[i].blockName), singleBlocks[i].buf);
-      assertEquals(cache.getBlock(multiBlocks[i].blockName), multiBlocks[i].buf);
+      assertEquals(cache.getBlock(singleBlocks[i].blockName).getBuffer(), singleBlocks[i].buf);
+      assertEquals(cache.getBlock(multiBlocks[i].blockName).getBuffer(), multiBlocks[i].buf);
     }
   }
   
@@ -429,9 +429,9 @@ public class TestLruBlockCache extends T
     
     // And the newest 5 blocks should still be accessible
     for (int i = 5; i < 10; i++) {
-      assertEquals(singleBlocks[i].buf, cache.getBlock(singleBlocks[i].blockName));
-      assertEquals(multiBlocks[i].buf, cache.getBlock(multiBlocks[i].blockName));
-      assertEquals(memoryBlocks[i].buf, cache.getBlock(memoryBlocks[i].blockName));
+      assertEquals(singleBlocks[i].buf, cache.getBlock(singleBlocks[i].blockName).getBuffer());
+      assertEquals(multiBlocks[i].buf, cache.getBlock(multiBlocks[i].blockName).getBuffer());
+      assertEquals(memoryBlocks[i].buf, cache.getBlock(memoryBlocks[i].blockName).getBuffer());
     }
   }
   

Modified: accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java Wed Jul 18 18:45:16 2012
@@ -26,6 +26,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.Random;
 import java.util.Set;
 
 import junit.framework.TestCase;
@@ -37,6 +38,7 @@ import org.apache.accumulo.core.data.Par
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileSKVIterator;
+import org.apache.accumulo.core.file.blockfile.cache.LruBlockCache;
 import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;
 import org.apache.accumulo.core.file.rfile.RFile.Reader;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
@@ -177,7 +179,11 @@ public class RFileTest extends TestCase 
       byte[] data = baos.toByteArray();
       bais = new SeekableByteArrayInputStream(data);
       in = new FSDataInputStream(bais);
-      CachableBlockFile.Reader _cbr = new CachableBlockFile.Reader(in, data.length, conf);
+      
+      LruBlockCache indexCache = new LruBlockCache(100000000, 100000);
+      LruBlockCache dataCache = new LruBlockCache(100000000, 100000);
+      
+      CachableBlockFile.Reader _cbr = new CachableBlockFile.Reader(in, data.length, conf, dataCache, indexCache);
       reader = new RFile.Reader(_cbr);
       iter = new ColumnFamilySkippingIterator(reader);
       
@@ -301,10 +307,10 @@ public class RFileTest extends TestCase 
         }
       }
     }
-    
+		
     // trf.writer.append(nk("r1","cf1","cq1","L1", 55), nv("foo"));
     trf.closeWriter();
-    
+
     trf.openReader();
     // seek before everything
     trf.iter.seek(new Range((Key) null, null), EMPTY_COL_FAMS, false);
@@ -384,6 +390,20 @@ public class RFileTest extends TestCase 
     
     assertEquals(expectedKeys.get(expectedKeys.size() - 1), trf.reader.getLastKey());
     
+    // test seeking to random location and reading all data from that point
+    // there was an off by one bug with this in the transient index
+    Random rand = new Random();
+    for (int i = 0; i < 12; i++) {
+      index = rand.nextInt(expectedKeys.size());
+      trf.seek(expectedKeys.get(index));
+      for (; index < expectedKeys.size(); index++) {
+        assertTrue(trf.iter.hasTop());
+        assertEquals(expectedKeys.get(index), trf.iter.getTopKey());
+        assertEquals(expectedValues.get(index), trf.iter.getTopValue());
+        trf.iter.next();
+      }
+    }
+
     trf.closeReader();
   }
   
@@ -1203,7 +1223,7 @@ public class RFileTest extends TestCase 
     assertFalse(trf.iter.hasTop());
     
     trf.iter.seek(new Range(nk("r0000", "cf1", "cq1", "", 1), false, nk("r0001", "cf1", "cq1", "", 1), true), EMPTY_COL_FAMS, false);
-    
+		
     for (int i = 2048; i < 4096; i++) {
       assertTrue(trf.iter.hasTop());
       assertEquals(nk("r0001", "cf1", "cq1", "", 1), trf.iter.getTopKey());

Propchange: accumulo/branches/ACCUMULO-259/server/
------------------------------------------------------------------------------
  Merged /accumulo/trunk/server:r1361355-1362560

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java Wed Jul 18 18:45:16 2012
@@ -27,7 +27,6 @@ import java.util.Set;
 import java.util.TimerTask;
 
 import org.apache.accumulo.cloudtrace.instrument.Tracer;
-import org.apache.accumulo.cloudtrace.instrument.thrift.TraceWrap;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/SecurityOperationImpl.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/SecurityOperationImpl.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/SecurityOperationImpl.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/SecurityOperationImpl.java Wed Jul 18 18:45:16 2012
@@ -161,14 +161,10 @@ public class SecurityOperationImpl imple
    * @throws ThriftSecurityException
    */
   public boolean authenticateUser(AuthInfo credentials, String user, ByteBuffer password) throws ThriftSecurityException {
-    authenticate(credentials);
-    
-    if (credentials.user.equals(user))
-      return true;
-    
-    if (!canPerformSystemActions(credentials))
+    // Authentication done in canPerformSystemActions
+    if (!(canPerformSystemActions(credentials) || credentials.user.equals(user)))
       throw new ThriftSecurityException(credentials.user, SecurityErrorCode.PERMISSION_DENIED);
-    
+
     return authenticator.authenticateUser(user, password, credentials.instanceId);
     
   }
@@ -475,7 +471,7 @@ public class SecurityOperationImpl imple
     authenticate(c);
     if (user.equals(SecurityConstants.SYSTEM_USERNAME))
       throw new ThriftSecurityException(c.user, SecurityErrorCode.PERMISSION_DENIED);
-    return c.user.equals(user) || hasSystemPermission(c.user, SystemPermission.ALTER_TABLE, false);
+    return c.user.equals(user) || hasSystemPermission(c.user, SystemPermission.ALTER_USER, false);
   }
   
   /**

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java Wed Jul 18 18:45:16 2012
@@ -199,6 +199,7 @@ import org.apache.hadoop.fs.FSDataOutput
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.SequenceFile.Reader;
 import org.apache.hadoop.io.Text;
@@ -3118,6 +3119,7 @@ public class TabletServer extends Abstra
       Instance instance = HdfsZooInstance.getInstance();
       ServerConfiguration conf = new ServerConfiguration(instance);
       Accumulo.init(fs, conf, "tserver");
+      ensureHdfsSyncIsEnabled(fs);
       recoverLocalWriteAheadLogs(fs, conf);
       TabletServer server = new TabletServer(conf, fs);
       server.config(hostname);
@@ -3128,6 +3130,17 @@ public class TabletServer extends Abstra
     }
   }
 
+  private static void ensureHdfsSyncIsEnabled(FileSystem fs) {
+    if (fs instanceof DistributedFileSystem) {
+      if (!fs.getConf().getBoolean("dfs.durable.sync", false) && !fs.getConf().getBoolean("dfs.support.append", false)) {
+        String msg = "Must set dfs.durable.sync OR dfs.support.append to true.  Which one needs to be set depends on your version of HDFS.  See ACCUMULO-623.";
+        log.fatal(msg);
+        System.exit(-1);
+      }
+    }
+    
+  }
+
   /**
    * Copy local walogs into HDFS on an upgrade
    * 

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/AlterTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/AlterTable.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/AlterTable.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/AlterTable.java Wed Jul 18 18:45:16 2012
@@ -59,6 +59,9 @@ public class AlterTable extends Test {
           throw new AccumuloException("Got a security exception when I should have had permission.", ae);
         else
           return;
+      } else if (ae.getErrorCode().equals(SecurityErrorCode.BAD_CREDENTIALS)) {
+        if (SecurityHelper.sysUserPassTransient(state))
+          return;
       }
       throw new AccumuloException("Got unexpected ae error code", ae);
     } catch (TableNotFoundException tnfe) {

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/AlterTablePerm.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/AlterTablePerm.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/AlterTablePerm.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/AlterTablePerm.java Wed Jul 18 18:45:16 2012
@@ -124,6 +124,10 @@ public class AlterTablePerm extends Test
             if (tableExists)
               throw new AccumuloException("Table doesn't exist but it should", ae);
             return;
+          case BAD_CREDENTIALS:
+            if (!SecurityHelper.sysUserPassTransient(state))
+              throw new AccumuloException("Bad credentials for user " + conn.whoami());
+            return;
           default:
             throw new AccumuloException("Got unexpected exception", ae);
         }
@@ -150,6 +154,10 @@ public class AlterTablePerm extends Test
             if (tableExists)
               throw new AccumuloException("Table doesn't exist but it should", ae);
             return;
+          case BAD_CREDENTIALS:
+            if (!SecurityHelper.sysUserPassTransient(state))
+              throw new AccumuloException("Bad credentials for user " + conn.whoami());
+            return;
           default:
             throw new AccumuloException("Got unexpected exception", ae);
         }

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/Authenticate.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/Authenticate.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/Authenticate.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/Authenticate.java Wed Jul 18 18:45:16 2012
@@ -77,6 +77,7 @@ public class Authenticate extends Test {
     if (!hasPermission)
       throw new AccumuloException("Didn't get Security Exception when we should have");
     if (result != (success && exists))
-      throw new AccumuloException("Got " + result + " as the result when it should be " + success);
+      throw new AccumuloException("Authentication " + (result ? "succeeded" : "failed") + " when it should have "
+          + ((success && exists) ? "succeeded" : "failed") + " while the user exists? " + exists);
   }
 }

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/ChangePass.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/ChangePass.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/ChangePass.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/ChangePass.java Wed Jul 18 18:45:16 2012
@@ -89,6 +89,10 @@ public class ChangePass extends Test {
           if (targetExists)
             throw new AccumuloException("User " + target + " doesn't exist and they SHOULD.", ae);
           return;
+        case BAD_CREDENTIALS:
+          if (!SecurityHelper.sysUserPassTransient(state))
+            throw new AccumuloException("Bad credentials for user " + conn.whoami());
+          return;
         default:
           throw new AccumuloException("Got unexpected exception", ae);
       }
@@ -98,6 +102,6 @@ public class ChangePass extends Test {
     } else
       SecurityHelper.setTabUserPass(state, newPass);
     if (!hasPerm)
-      throw new AccumuloException("Password change succeeded when it should have failed.");
+      throw new AccumuloException("Password change succeeded when it should have failed for " + source + " changing the password for " + target + ".");
   }
 }

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/DropTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/DropTable.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/DropTable.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/DropTable.java Wed Jul 18 18:45:16 2012
@@ -70,6 +70,9 @@ public class DropTable extends Test {
               SecurityHelper.setTabPerm(state, user, tp, false);
           return;
         }
+      } else if (ae.getErrorCode().equals(SecurityErrorCode.BAD_CREDENTIALS)) {
+        if (SecurityHelper.sysUserPassTransient(state))
+          return;
       }
       throw new AccumuloException("Got unexpected ae error code", ae);
     } catch (TableNotFoundException tnfe) {

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/SecurityHelper.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/SecurityHelper.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/SecurityHelper.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/SecurityHelper.java Wed Jul 18 18:45:16 2012
@@ -81,8 +81,14 @@ public class SecurityHelper {
   public static void setSysUserPass(State state, byte[] sysUserPass) {
     log.debug("Setting system user pass to " + new String(sysUserPass));
     state.set(masterPass, sysUserPass);
+    state.set(masterPass + "time", System.currentTimeMillis());
+
   }
   
+  public static boolean sysUserPassTransient(State state) {
+    return System.currentTimeMillis() - state.getInteger(masterPass + "time") < 1000;
+  }
+
   public static byte[] getTabUserPass(State state) {
     return (byte[]) state.get(tUserPass);
   }
@@ -90,8 +96,13 @@ public class SecurityHelper {
   public static void setTabUserPass(State state, byte[] tabUserPass) {
     log.debug("Setting table user pass to " + new String(tabUserPass));
     state.set(tUserPass, tabUserPass);
+    state.set(tUserPass + "time", System.currentTimeMillis());
   }
   
+  public static boolean tabUserPassTransient(State state) {
+    return System.currentTimeMillis() - state.getInteger(tUserPass + "time") < 1000;
+  }
+
   public static boolean getTabUserExists(State state) {
     return Boolean.parseBoolean(state.getString(tUserExists));
   }

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/TableOp.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/TableOp.java?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/TableOp.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/TableOp.java Wed Jul 18 18:45:16 2012
@@ -195,6 +195,9 @@ public class TableOp extends Test {
             if (hasPerm)
               throw new AccumuloException("Bulk Import failed when it should have worked: " + tableName);
             return;
+          } else if (ae.getErrorCode().equals(SecurityErrorCode.BAD_CREDENTIALS)) {
+            if (SecurityHelper.sysUserPassTransient(state))
+              return;
           }
           throw new AccumuloException("Unexpected exception!", ae);
         }

Propchange: accumulo/branches/ACCUMULO-259/src/
------------------------------------------------------------------------------
  Merged /accumulo/trunk/src:r1361355-1362560

Modified: accumulo/branches/ACCUMULO-259/test/system/auto/simple/shell.py
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/test/system/auto/simple/shell.py?rev=1363041&r1=1363040&r2=1363041&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/test/system/auto/simple/shell.py (original)
+++ accumulo/branches/ACCUMULO-259/test/system/auto/simple/shell.py Wed Jul 18 18:45:16 2012
@@ -19,6 +19,7 @@ import logging
 import unittest
 import time
 from TestUtils import TestUtilsMixin, ROOT, ROOT_PASSWORD, ACCUMULO_HOME
+from subprocess import Popen as BasePopen, PIPE
 
 log = logging.getLogger('test.shell')
       
@@ -36,6 +37,7 @@ class ShellTest(TestUtilsMixin,unittest.
         TestUtilsMixin.setUp(self)
         
     def runTest(self):
+        self.badLoginTest()
         self.setIterTest()
         self.setScanIterTest()
         self.iteratorsTest()
@@ -55,6 +57,12 @@ class ShellTest(TestUtilsMixin,unittest.
         self.getauthsTest()
         
         
+    def badLoginTest(self, **opts):
+      log.debug("Running shell with bad password")
+      handle = self.runOn(self.masterHost(), [self.accumulo_sh(), 'shell', '-u', ROOT, '-p', "ThisWouldBeATerriblePasswordToHave"], stdin=PIPE, **opts)
+      handle.communicate("quit\n")
+      self.failUnless(handle.returncode != 0, "Was able to create a shell with bad credentials")
+
     def setIterTest(self):
         input = 'setiter -t setitertest -n mymax -scan -p 10 -class org.apache.accumulo.core.iterators.user.MaxCombiner\n\ncf\n\nSTRING\n'
         out,err,code = self.rootShell(self.masterHost(),input)