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 2011/10/27 17:25:17 UTC

svn commit: r1189806 [34/46] - in /incubator/accumulo: branches/1.3/contrib/ branches/1.3/src/core/src/main/java/org/apache/accumulo/core/client/ branches/1.3/src/core/src/main/java/org/apache/accumulo/core/client/admin/ branches/1.3/src/core/src/main/...

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiLevelIndex.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiLevelIndex.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiLevelIndex.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiLevelIndex.java Thu Oct 27 15:24:51 2011
@@ -132,8 +132,10 @@ public class MultiLevelIndex {
     @Override
     public IndexEntry get(int index) {
       int len;
-      if (index == offsets.length - 1) len = data.length - offsets[index];
-      else len = offsets[index + 1] - offsets[index];
+      if (index == offsets.length - 1)
+        len = data.length - offsets[index];
+      else
+        len = offsets[index + 1] - offsets[index];
       
       ByteArrayInputStream bais = new ByteArrayInputStream(data, offsets[index], len);
       DataInputStream dis = new DataInputStream(bais);
@@ -172,8 +174,10 @@ public class MultiLevelIndex {
     @Override
     public Key get(int index) {
       int len;
-      if (index == offsets.length - 1) len = data.length - offsets[index];
-      else len = offsets[index + 1] - offsets[index];
+      if (index == offsets.length - 1)
+        len = data.length - offsets[index];
+      else
+        len = offsets[index + 1] - offsets[index];
       
       ByteArrayInputStream bais = new ByteArrayInputStream(data, offsets[index], len);
       DataInputStream dis = new DataInputStream(bais);
@@ -371,7 +375,8 @@ public class MultiLevelIndex {
     
     private void flush(int level, Key lastKey, boolean last) throws IOException {
       
-      if (last && level == levels.size() - 1) return;
+      if (last && level == levels.size() - 1)
+        return;
       
       IndexBlock iblock = levels.get(level);
       if ((iblock.getSize() > threshold && iblock.offsets.size() > 1) || last) {
@@ -383,8 +388,10 @@ public class MultiLevelIndex {
         add(level + 1, lastKey, 0, out.getStartPos(), out.getCompressedSize(), out.getRawSize());
         flush(level + 1, lastKey, last);
         
-        if (last) levels.set(level, null);
-        else levels.set(level, new IndexBlock(level, totalAdded));
+        if (last)
+          levels.set(level, null);
+        else
+          levels.set(level, new IndexBlock(level, totalAdded));
       }
     }
     
@@ -395,7 +402,8 @@ public class MultiLevelIndex {
     }
     
     public void addLast(Key key, int data, long offset, long compressedSize, long rawSize) throws IOException {
-      if (addedLast) throw new IllegalStateException("already added last");
+      if (addedLast)
+        throw new IllegalStateException("already added last");
       
       totalAdded++;
       add(0, key, data, offset, compressedSize, rawSize);
@@ -405,7 +413,8 @@ public class MultiLevelIndex {
     }
     
     public void close(DataOutput out) throws IOException {
-      if (totalAdded > 0 && !addedLast) throw new IllegalStateException("did not call addLast");
+      if (totalAdded > 0 && !addedLast)
+        throw new IllegalStateException("did not call addLast");
       
       out.writeInt(totalAdded);
       // save root node
@@ -448,10 +457,12 @@ public class MultiLevelIndex {
           }
         });
         
-        if (pos < 0) pos = (pos * -1) - 1;
+        if (pos < 0)
+          pos = (pos * -1) - 1;
         
         if (pos == indexBlock.getIndex().size()) {
-          if (parent != null) throw new IllegalStateException();
+          if (parent != null)
+            throw new IllegalStateException();
           this.currentPos = pos;
           return this;
         }
@@ -469,7 +480,8 @@ public class MultiLevelIndex {
       
       private Node getLast() throws IOException {
         currentPos = indexBlock.getIndex().size() - 1;
-        if (indexBlock.getLevel() == 0) return this;
+        if (indexBlock.getLevel() == 0)
+          return this;
         
         IndexEntry ie = indexBlock.getIndex().get(currentPos);
         Node child = new Node(this, getIndexBlock(ie));
@@ -478,7 +490,8 @@ public class MultiLevelIndex {
       
       private Node getFirst() throws IOException {
         currentPos = 0;
-        if (indexBlock.getLevel() == 0) return this;
+        if (indexBlock.getLevel() == 0)
+          return this;
         
         IndexEntry ie = indexBlock.getIndex().get(currentPos);
         Node child = new Node(this, getIndexBlock(ie));
@@ -486,7 +499,8 @@ public class MultiLevelIndex {
       }
       
       private Node getPrevious() throws IOException {
-        if (currentPos == 0) return parent.getPrevious();
+        if (currentPos == 0)
+          return parent.getPrevious();
         
         currentPos--;
         
@@ -497,7 +511,8 @@ public class MultiLevelIndex {
       }
       
       private Node getNext() throws IOException {
-        if (currentPos == indexBlock.getIndex().size() - 1) return parent.getNext();
+        if (currentPos == indexBlock.getIndex().size() - 1)
+          return parent.getNext();
         
         currentPos++;
         
@@ -548,7 +563,8 @@ public class MultiLevelIndex {
       
       @Override
       public boolean hasNext() {
-        if (node == null) return false;
+        if (node == null)
+          return false;
         
         if (!liter.hasNext()) {
           return node.indexBlock.hasNext();
@@ -582,7 +598,8 @@ public class MultiLevelIndex {
       
       @Override
       public boolean hasPrevious() {
-        if (node == null) return false;
+        if (node == null)
+          return false;
         
         if (!liter.hasPrevious()) {
           return node.indexBlock.getOffset() > 0;
@@ -670,10 +687,12 @@ public class MultiLevelIndex {
     
     private void getIndexInfo(IndexBlock ib, Map<Integer,Long> sizesByLevel, Map<Integer,Long> countsByLevel) throws IOException {
       Long size = sizesByLevel.get(ib.getLevel());
-      if (size == null) size = 0l;
+      if (size == null)
+        size = 0l;
       
       Long count = countsByLevel.get(ib.getLevel());
-      if (count == null) count = 0l;
+      if (count == null)
+        count = 0l;
       
       size += ib.index.sizeInBytes();
       count++;

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java Thu Oct 27 15:24:51 2011
@@ -138,7 +138,8 @@ public class RFile {
     }
     
     private void setFirstKey(Key key) {
-      if (firstKey != null) throw new IllegalStateException();
+      if (firstKey != null)
+        throw new IllegalStateException();
       this.firstKey = new Key(key);
     }
     
@@ -196,12 +197,15 @@ public class RFile {
       int size = in.readInt();
       
       if (size == -1) {
-        if (!isDefaultLG) throw new IllegalStateException("Non default LG " + name + " does not have column families");
+        if (!isDefaultLG)
+          throw new IllegalStateException("Non default LG " + name + " does not have column families");
         
         columnFamilies = null;
       } else {
-        if (columnFamilies == null) columnFamilies = new HashMap<ByteSequence,Count>();
-        else columnFamilies.clear();
+        if (columnFamilies == null)
+          columnFamilies = new HashMap<ByteSequence,Count>();
+        else
+          columnFamilies.clear();
         
         for (int i = 0; i < size; i++) {
           int len = in.readInt();
@@ -247,7 +251,8 @@ public class RFile {
       }
       
       out.writeBoolean(firstKey != null);
-      if (firstKey != null) firstKey.write(out);
+      if (firstKey != null)
+        firstKey.write(out);
       
       indexWriter.close(out);
     }
@@ -335,7 +340,8 @@ public class RFile {
       mba.writeInt(RINDEX_MAGIC);
       mba.writeInt(RINDEX_VER_6);
       
-      if (currentLocalityGroup != null) localityGroups.add(currentLocalityGroup);
+      if (currentLocalityGroup != null)
+        localityGroups.add(currentLocalityGroup);
       
       mba.writeInt(localityGroups.size());
       
@@ -400,9 +406,10 @@ public class RFile {
     private void closeBlock(Key key, boolean lastBlock) throws IOException {
       blockWriter.close();
       
-      if (lastBlock) currentLocalityGroup.indexWriter.addLast(key, entries, blockWriter.getStartPos(), blockWriter.getCompressedSize(),
-          blockWriter.getRawSize());
-      else currentLocalityGroup.indexWriter.add(key, entries, blockWriter.getStartPos(), blockWriter.getCompressedSize(), blockWriter.getRawSize());
+      if (lastBlock)
+        currentLocalityGroup.indexWriter.addLast(key, entries, blockWriter.getStartPos(), blockWriter.getCompressedSize(), blockWriter.getRawSize());
+      else
+        currentLocalityGroup.indexWriter.add(key, entries, blockWriter.getStartPos(), blockWriter.getCompressedSize(), blockWriter.getRawSize());
       
       blockWriter = null;
       lastKeyInBlock = null;
@@ -452,7 +459,8 @@ public class RFile {
     
     @Override
     public void startNewLocalityGroup(String name, Set<ByteSequence> columnFamilies) throws IOException {
-      if (columnFamilies == null) throw new NullPointerException();
+      if (columnFamilies == null)
+        throw new NullPointerException();
       
       _startNewLocalityGroup(name, columnFamilies);
     }
@@ -512,7 +520,8 @@ public class RFile {
     public void close() throws IOException {
       closed = true;
       hasTop = false;
-      if (currBlock != null) currBlock.close();
+      if (currBlock != null)
+        currBlock.close();
       
     }
     
@@ -553,7 +562,8 @@ public class RFile {
     
     private void _next() throws IOException {
       
-      if (!hasTop) throw new IllegalStateException();
+      if (!hasTop)
+        throw new IllegalStateException();
       
       if (entriesLeft == 0) {
         currBlock.close();
@@ -578,21 +588,27 @@ public class RFile {
     }
     
     private ABlockReader getDataBlock(IndexEntry indexEntry) throws IOException {
-      if (interruptFlag != null && interruptFlag.get()) throw new IterationInterruptedException();
+      if (interruptFlag != null && interruptFlag.get())
+        throw new IterationInterruptedException();
       
-      if (version == RINDEX_VER_3 || version == RINDEX_VER_4) return reader.getDataBlock(startBlock + iiter.previousIndex());
-      else return reader.getDataBlock(indexEntry.getOffset(), indexEntry.getCompressedSize(), indexEntry.getRawSize());
+      if (version == RINDEX_VER_3 || version == RINDEX_VER_4)
+        return reader.getDataBlock(startBlock + iiter.previousIndex());
+      else
+        return reader.getDataBlock(indexEntry.getOffset(), indexEntry.getCompressedSize(), indexEntry.getRawSize());
       
     }
     
     @Override
     public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
       
-      if (closed) throw new IllegalStateException("Locality group reader closed");
+      if (closed)
+        throw new IllegalStateException("Locality group reader closed");
       
-      if (columnFamilies.size() != 0 || inclusive) throw new IllegalArgumentException("I do not know how to filter column families");
+      if (columnFamilies.size() != 0 || inclusive)
+        throw new IllegalArgumentException("I do not know how to filter column families");
       
-      if (interruptFlag != null && interruptFlag.get()) throw new IterationInterruptedException();
+      if (interruptFlag != null && interruptFlag.get())
+        throw new IterationInterruptedException();
       
       try {
         _seek(range);
@@ -629,7 +645,8 @@ public class RFile {
       }
       
       Key startKey = range.getStartKey();
-      if (startKey == null) startKey = new Key();
+      if (startKey == null)
+        startKey = new Key();
       
       boolean reseek = true;
       
@@ -692,9 +709,11 @@ public class RFile {
             iiter.previous();
           }
           
-          if (iiter.hasPrevious()) prevKey = new Key(iiter.peekPrevious().getKey()); // initially prevKey is the last key of the prev block
-          else prevKey = new Key(); // first block in the file, so set prev key to minimal key
-          
+          if (iiter.hasPrevious())
+            prevKey = new Key(iiter.peekPrevious().getKey()); // initially prevKey is the last key of the prev block
+          else
+            prevKey = new Key(); // first block in the file, so set prev key to minimal key
+            
           IndexEntry indexEntry = iiter.next();
           entriesLeft = indexEntry.getNumEntries();
           currBlock = getDataBlock(indexEntry);
@@ -726,7 +745,8 @@ public class RFile {
     
     @Override
     public Key getLastKey() throws IOException {
-      if (index.size() == 0) return null;
+      if (index.size() == 0)
+        return null;
       return index.getLastKey();
     }
     
@@ -780,8 +800,10 @@ public class RFile {
       int magic = mb.readInt();
       int ver = mb.readInt();
       
-      if (magic != RINDEX_MAGIC) throw new IOException("Did not see expected magic number, saw " + magic);
-      if (ver != RINDEX_VER_6 && ver != RINDEX_VER_4 && ver != RINDEX_VER_3) throw new IOException("Did not see expected version, saw " + ver);
+      if (magic != RINDEX_MAGIC)
+        throw new IOException("Did not see expected magic number, saw " + magic);
+      if (ver != RINDEX_VER_6 && ver != RINDEX_VER_4 && ver != RINDEX_VER_3)
+        throw new IOException("Did not see expected version, saw " + ver);
       
       int size = mb.readInt();
       lgReaders = new LocalityGroupReader[size];
@@ -800,7 +822,8 @@ public class RFile {
       
       nonDefaultColumnFamilies = new HashSet<ByteSequence>();
       for (LocalityGroupMetadata lgm : localityGroups) {
-        if (!lgm.isDefaultLG) nonDefaultColumnFamilies.addAll(lgm.columnFamilies.keySet());
+        if (!lgm.isDefaultLG)
+          nonDefaultColumnFamilies.addAll(lgm.columnFamilies.keySet());
       }
       
       createHeap(lgReaders.length);
@@ -831,7 +854,8 @@ public class RFile {
     
     @Override
     public void closeDeepCopies() {
-      if (deepCopy) throw new RuntimeException("Calling closeDeepCopies on a deep copy is not supported");
+      if (deepCopy)
+        throw new RuntimeException("Calling closeDeepCopies on a deep copy is not supported");
       
       for (Reader deepCopy : deepCopies)
         deepCopy.closeLocalityGroupReaders();
@@ -841,7 +865,8 @@ public class RFile {
     
     @Override
     public void close() throws IOException {
-      if (deepCopy) throw new RuntimeException("Calling close on a deep copy is not supported");
+      if (deepCopy)
+        throw new RuntimeException("Calling close on a deep copy is not supported");
       
       closeDeepCopies();
       closeLocalityGroupReaders();
@@ -868,7 +893,8 @@ public class RFile {
           minKey = lgReaders[i].getFirstKey();
         } else {
           Key firstKey = lgReaders[i].getFirstKey();
-          if (firstKey != null && firstKey.compareTo(minKey) < 0) minKey = firstKey;
+          if (firstKey != null && firstKey.compareTo(minKey) < 0)
+            minKey = firstKey;
         }
       }
       
@@ -888,7 +914,8 @@ public class RFile {
           maxKey = lgReaders[i].getLastKey();
         } else {
           Key lastKey = lgReaders[i].getLastKey();
-          if (lastKey != null && lastKey.compareTo(maxKey) > 0) maxKey = lastKey;
+          if (lastKey != null && lastKey.compareTo(maxKey) > 0)
+            maxKey = lastKey;
         }
       }
       
@@ -928,13 +955,15 @@ public class RFile {
       numLGSeeked = 0;
       
       Set<ByteSequence> cfSet;
-      if (columnFamilies.size() > 0) if (columnFamilies instanceof Set<?>) {
-        cfSet = (Set<ByteSequence>) columnFamilies;
-      } else {
-        cfSet = new HashSet<ByteSequence>();
-        cfSet.addAll(columnFamilies);
-      }
-      else cfSet = Collections.emptySet();
+      if (columnFamilies.size() > 0)
+        if (columnFamilies instanceof Set<?>) {
+          cfSet = (Set<ByteSequence>) columnFamilies;
+        } else {
+          cfSet = new HashSet<ByteSequence>();
+          cfSet.addAll(columnFamilies);
+        }
+      else
+        cfSet = Collections.emptySet();
       
       for (LocalityGroupReader lgr : lgReaders) {
         
@@ -972,10 +1001,16 @@ public class RFile {
            */
           
           for (Entry<ByteSequence,Count> entry : lgr.columnFamilies.entrySet())
-            if (entry.getValue().count > 0) if (cfSet.contains(entry.getKey())) if (inclusive) include = true;
-            else exclude = true;
-            else if (inclusive) exclude = true;
-            else include = true;
+            if (entry.getValue().count > 0)
+              if (cfSet.contains(entry.getKey()))
+                if (inclusive)
+                  include = true;
+                else
+                  exclude = true;
+              else if (inclusive)
+                exclude = true;
+              else
+                include = true;
         }
         
         if (include) {
@@ -1018,9 +1053,11 @@ public class RFile {
     
     @Override
     public void setInterruptFlag(AtomicBoolean flag) {
-      if (deepCopy) throw new RuntimeException("Calling setInterruptFlag on a deep copy is not supported");
+      if (deepCopy)
+        throw new RuntimeException("Calling setInterruptFlag on a deep copy is not supported");
       
-      if (deepCopies.size() != 0) throw new RuntimeException("Setting interrupt flag after calling deep copy not supported");
+      if (deepCopies.size() != 0)
+        throw new RuntimeException("Setting interrupt flag after calling deep copy not supported");
       
       setInterruptFlagInternal(flag);
     }

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java Thu Oct 27 15:24:51 2011
@@ -112,7 +112,8 @@ public class RFileOperations extends Fil
     long hblock = conf.getLong("dfs.block.size", 1 << 26);
     long tblock = acuconf.getMemoryInBytes(Property.TABLE_FILE_BLOCK_SIZE);
     long block = hblock;
-    if (tblock > 0) block = tblock;
+    if (tblock > 0)
+      block = tblock;
     int bufferSize = conf.getInt("io.file.buffer.size", 4096);
     
     long blockSize = acuconf.getMemoryInBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE);

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java Thu Oct 27 15:24:51 2011
@@ -89,20 +89,26 @@ public class RelativeKey implements Writ
     fieldsSame = 0;
     
     if (prevKey != null) {
-      if (prevKey.getRowData().equals(key.getRowData())) fieldsSame |= ROW_SAME;
+      if (prevKey.getRowData().equals(key.getRowData()))
+        fieldsSame |= ROW_SAME;
       
-      if (prevKey.getColumnFamilyData().equals(key.getColumnFamilyData())) fieldsSame |= CF_SAME;
+      if (prevKey.getColumnFamilyData().equals(key.getColumnFamilyData()))
+        fieldsSame |= CF_SAME;
       
-      if (prevKey.getColumnQualifierData().equals(key.getColumnQualifierData())) fieldsSame |= CQ_SAME;
+      if (prevKey.getColumnQualifierData().equals(key.getColumnQualifierData()))
+        fieldsSame |= CQ_SAME;
       
-      if (prevKey.getColumnVisibilityData().equals(key.getColumnVisibilityData())) fieldsSame |= CV_SAME;
+      if (prevKey.getColumnVisibilityData().equals(key.getColumnVisibilityData()))
+        fieldsSame |= CV_SAME;
       
-      if (prevKey.getTimestamp() == key.getTimestamp()) fieldsSame |= TS_SAME;
+      if (prevKey.getTimestamp() == key.getTimestamp())
+        fieldsSame |= TS_SAME;
       
     }
     
     // stored deleted information in bit vector instead of its own byte
-    if (key.isDeleted()) fieldsSame |= DELETED;
+    if (key.isDeleted())
+      fieldsSame |= DELETED;
   }
   
   @Override
@@ -209,12 +215,15 @@ public class RelativeKey implements Writ
       cqCmp = cq.compareTo(stopCQ);
       
       if (rowCmp >= 0) {
-        if (rowCmp > 0) return 0;
+        if (rowCmp > 0)
+          return 0;
         
         if (cfCmp >= 0) {
-          if (cfCmp > 0) return 0;
+          if (cfCmp > 0)
+            return 0;
           
-          if (cqCmp >= 0) return 0;
+          if (cqCmp >= 0)
+            return 0;
         }
       }
       
@@ -297,12 +306,15 @@ public class RelativeKey implements Writ
       count++;
       
       if (changed && rowCmp >= 0) {
-        if (rowCmp > 0) break;
+        if (rowCmp > 0)
+          break;
         
         if (cfCmp >= 0) {
-          if (cfCmp > 0) break;
+          if (cfCmp > 0)
+            break;
           
-          if (cqCmp >= 0) break;
+          if (cqCmp >= 0)
+            break;
         }
       }
       

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BCFile.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BCFile.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BCFile.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BCFile.java Thu Oct 27 15:24:51 2011
@@ -882,7 +882,8 @@ public final class BCFile {
     }
     
     public void addBlockRegion(BlockRegion region) {
-      if (trackBlocks) listRegions.add(region);
+      if (trackBlocks)
+        listRegions.add(region);
     }
     
     public void write(DataOutput out) throws IOException {

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BoundedRangeFileInputStream.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BoundedRangeFileInputStream.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BoundedRangeFileInputStream.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BoundedRangeFileInputStream.java Thu Oct 27 15:24:51 2011
@@ -74,7 +74,8 @@ class BoundedRangeFileInputStream extend
   @Override
   public int read() throws IOException {
     int ret = read(oneByte);
-    if (ret == 1) return oneByte[0] & 0xff;
+    if (ret == 1)
+      return oneByte[0] & 0xff;
     return -1;
   }
   
@@ -90,7 +91,8 @@ class BoundedRangeFileInputStream extend
     }
     
     final int n = (int) Math.min(Integer.MAX_VALUE, Math.min(len, (end - pos)));
-    if (n == 0) return -1;
+    if (n == 0)
+      return -1;
     Integer ret = 0;
     synchronized (in) {
       in.seek(pos);
@@ -132,7 +134,8 @@ class BoundedRangeFileInputStream extend
   
   @Override
   public void reset() throws IOException {
-    if (mark < 0) throw new IOException("Resetting to invalid mark");
+    if (mark < 0)
+      throw new IOException("Resetting to invalid mark");
     pos = mark;
   }
   

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/Chunk.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/Chunk.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/Chunk.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/Chunk.java Thu Oct 27 15:24:51 2011
@@ -113,10 +113,13 @@ final class Chunk {
      *           on I/O errors.
      */
     private boolean checkEOF() throws IOException {
-      if (isClosed()) return true;
+      if (isClosed())
+        return true;
       while (true) {
-        if (remain > 0) return false;
-        if (lastChunk) return true;
+        if (remain > 0)
+          return false;
+        if (lastChunk)
+          return true;
         readLength();
       }
     }
@@ -131,9 +134,11 @@ final class Chunk {
     
     @Override
     public int read() throws IOException {
-      if (checkEOF()) return -1;
+      if (checkEOF())
+        return -1;
       int ret = in.read();
-      if (ret < 0) throw new IOException("Corrupted chunk encoding stream");
+      if (ret < 0)
+        throw new IOException("Corrupted chunk encoding stream");
       --remain;
       return ret;
     }
@@ -152,7 +157,8 @@ final class Chunk {
       if (!checkEOF()) {
         int n = Math.min(remain, len);
         int ret = in.read(b, off, n);
-        if (ret < 0) throw new IOException("Corrupted chunk encoding stream");
+        if (ret < 0)
+          throw new IOException("Corrupted chunk encoding stream");
         remain -= ret;
         return ret;
       }

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/CompareUtils.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/CompareUtils.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/CompareUtils.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/CompareUtils.java Thu Oct 27 15:24:51 2011
@@ -72,8 +72,10 @@ class CompareUtils {
     @Override
     public int compare(Scalar o1, Scalar o2) {
       long diff = o1.magnitude() - o2.magnitude();
-      if (diff < 0) return -1;
-      if (diff > 0) return 1;
+      if (diff < 0)
+        return -1;
+      if (diff > 0)
+        return 1;
       return 0;
     }
   }

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/TFile.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/TFile.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/TFile.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/TFile.java Thu Oct 27 15:24:51 2011
@@ -652,8 +652,10 @@ public class TFile {
       int compareTo(int bid, long rid) {
         if (this.blockIndex == bid) {
           long ret = this.recordIndex - rid;
-          if (ret > 0) return 1;
-          if (ret < 0) return -1;
+          if (ret > 0)
+            return 1;
+          if (ret < 0)
+            return -1;
           return 0;
         }
         return this.blockIndex - bid;
@@ -683,12 +685,17 @@ public class TFile {
        */
       @Override
       public boolean equals(Object obj) {
-        if (this == obj) return true;
-        if (obj == null) return false;
-        if (getClass() != obj.getClass()) return false;
+        if (this == obj)
+          return true;
+        if (obj == null)
+          return false;
+        if (getClass() != obj.getClass())
+          return false;
         Location other = (Location) obj;
-        if (blockIndex != other.blockIndex) return false;
-        if (recordIndex != other.recordIndex) return false;
+        if (blockIndex != other.blockIndex)
+          return false;
+        if (recordIndex != other.recordIndex)
+          return false;
         return true;
       }
     }
@@ -875,7 +882,8 @@ public class TFile {
       }
       checkTFileDataIndex();
       int blkIndex = (greater) ? tfileIndex.upperBound(key) : tfileIndex.lowerBound(key);
-      if (blkIndex < 0) return end;
+      if (blkIndex < 0)
+        return end;
       return new Location(blkIndex, 0);
     }
     
@@ -903,7 +911,8 @@ public class TFile {
      */
     Location getLocationNear(long offset) {
       int blockIndex = readerBCF.getBlockIndexNear(offset);
-      if (blockIndex == -1) return end;
+      if (blockIndex == -1)
+        return end;
       return new Location(blockIndex, 0);
     }
     
@@ -917,7 +926,8 @@ public class TFile {
      */
     public RawComparable getKeyNear(long offset) throws IOException {
       int blockIndex = readerBCF.getBlockIndexNear(offset);
-      if (blockIndex == -1) return null;
+      if (blockIndex == -1)
+        return null;
       checkTFileDataIndex();
       return new ByteArray(tfileIndex.getEntry(blockIndex).key);
     }
@@ -1332,7 +1342,8 @@ public class TFile {
        * check whether we have already successfully obtained the key. It also initializes the valueInputStream.
        */
       void checkKey() throws IOException {
-        if (klen >= 0) return;
+        if (klen >= 0)
+          return;
         if (atEnd()) {
           throw new EOFException("No key-value to read");
         }
@@ -1651,8 +1662,10 @@ public class TFile {
          */
         @Override
         public boolean equals(Object other) {
-          if (this == other) return true;
-          if (!(other instanceof Entry)) return false;
+          if (this == other)
+            return true;
+          if (!(other instanceof Entry))
+            return false;
           return ((Entry) other).compareTo(keyBuffer, 0, getKeyLength()) == 0;
         }
         
@@ -1699,8 +1712,10 @@ public class TFile {
         
         while (currentLocation.getRecordIndex() < entryInBlock) {
           int cmp = compareCursorKeyTo(key);
-          if (cmp > 0) return false;
-          if (cmp == 0 && !greater) return true;
+          if (cmp > 0)
+            return false;
+          if (cmp == 0 && !greater)
+            return true;
           if (!valueBufferInputStream.isClosed()) {
             valueBufferInputStream.close();
           }

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/TFileDumper.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/TFileDumper.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/TFileDumper.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/TFileDumper.java Thu Oct 27 15:24:51 2011
@@ -49,7 +49,8 @@ class TFileDumper {
   private enum Align {
     LEFT, CENTER, RIGHT, ZERO_PADDED;
     static String format(String s, int width, Align align) {
-      if (s.length() >= width) return s;
+      if (s.length() >= width)
+        return s;
       int room = width - s.length();
       Align alignAdjusted = align;
       if (room == 1) {

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/Utils.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/Utils.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/Utils.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/Utils.java Thu Oct 27 15:24:51 2011
@@ -245,7 +245,8 @@ public final class Utils {
    */
   public static String readString(DataInput in) throws IOException {
     int length = readVInt(in);
-    if (length == -1) return null;
+    if (length == -1)
+      return null;
     byte[] buffer = new byte[length];
     in.readFully(buffer);
     return Text.decode(buffer);
@@ -357,8 +358,10 @@ public final class Utils {
     
     @Override
     public boolean equals(Object other) {
-      if (this == other) return true;
-      if (!(other instanceof Version)) return false;
+      if (this == other)
+        return true;
+      if (!(other instanceof Version))
+        return false;
       return compareTo((Version) other) == 0;
     }
     
@@ -389,8 +392,10 @@ public final class Utils {
       int mid = (low + high) >>> 1;
       T midVal = list.get(mid);
       int ret = cmp.compare(midVal, key);
-      if (ret < 0) low = mid + 1;
-      else high = mid;
+      if (ret < 0)
+        low = mid + 1;
+      else
+        high = mid;
     }
     return low;
   }
@@ -416,8 +421,10 @@ public final class Utils {
       int mid = (low + high) >>> 1;
       T midVal = list.get(mid);
       int ret = cmp.compare(midVal, key);
-      if (ret <= 0) low = mid + 1;
-      else high = mid;
+      if (ret <= 0)
+        low = mid + 1;
+      else
+        high = mid;
     }
     return low;
   }
@@ -441,8 +448,10 @@ public final class Utils {
       int mid = (low + high) >>> 1;
       Comparable<? super T> midVal = list.get(mid);
       int ret = midVal.compareTo(key);
-      if (ret < 0) low = mid + 1;
-      else high = mid;
+      if (ret < 0)
+        low = mid + 1;
+      else
+        high = mid;
     }
     return low;
   }
@@ -466,8 +475,10 @@ public final class Utils {
       int mid = (low + high) >>> 1;
       Comparable<? super T> midVal = list.get(mid);
       int ret = midVal.compareTo(key);
-      if (ret <= 0) low = mid + 1;
-      else high = mid;
+      if (ret <= 0)
+        low = mid + 1;
+      else
+        high = mid;
     }
     return low;
   }

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/AggregatingIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/AggregatingIterator.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/AggregatingIterator.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/AggregatingIterator.java Thu Oct 27 15:24:51 2011
@@ -65,7 +65,8 @@ public class AggregatingIterator impleme
   private void aggregateRowColumn(Aggregator aggr) throws IOException {
     // this function assumes that first value is not delete
     
-    if (iterator.getTopKey().isDeleted()) return;
+    if (iterator.getTopKey().isDeleted())
+      return;
     
     workKey.set(iterator.getTopKey());
     
@@ -189,7 +190,8 @@ public class AggregatingIterator impleme
   public boolean validateOptions(Map<String,String> options) {
     for (Entry<String,String> entry : options.entrySet()) {
       String classname = entry.getValue();
-      if (classname == null) return false;
+      if (classname == null)
+        return false;
       Class<? extends Aggregator> clazz;
       try {
         clazz = AccumuloClassLoader.loadClass(classname, Aggregator.class);

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/Combiner.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/Combiner.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/Combiner.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/Combiner.java Thu Oct 27 15:24:51 2011
@@ -64,7 +64,8 @@ public abstract class Combiner extends W
     
     @Override
     public Value next() {
-      if (!hasNext) throw new NoSuchElementException();
+      if (!hasNext)
+        throw new NoSuchElementException();
       Value topValue = source.getTopValue();
       try {
         source.next();
@@ -88,13 +89,15 @@ public abstract class Combiner extends W
   
   @Override
   public Key getTopKey() {
-    if (topKey == null) return super.getTopKey();
+    if (topKey == null)
+      return super.getTopKey();
     return topKey;
   }
   
   @Override
   public Value getTopValue() {
-    if (topKey == null) return super.getTopValue();
+    if (topKey == null)
+      return super.getTopValue();
     return topValue;
   }
   
@@ -122,7 +125,8 @@ public abstract class Combiner extends W
     if (super.hasTop()) {
       workKey.set(super.getTopKey());
       if (combiners.isEmpty() || combiners.contains(workKey)) {
-        if (workKey.isDeleted()) return;
+        if (workKey.isDeleted())
+          return;
         topKey = workKey;
         Iterator<Value> viter = new ValueIterator(getSource());
         topValue = reduce(topKey, viter);

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FamilyIntersectingIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FamilyIntersectingIterator.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FamilyIntersectingIterator.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FamilyIntersectingIterator.java Thu Oct 27 15:24:51 2011
@@ -102,8 +102,10 @@ public class FamilyIntersectingIterator 
   @Override
   public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
     super.init(source, options, env);
-    if (options.containsKey(indexFamilyOptionName)) indexColf = new Text(options.get(indexFamilyOptionName));
-    if (options.containsKey(docFamilyOptionName)) docColf = new Text(options.get(docFamilyOptionName));
+    if (options.containsKey(indexFamilyOptionName))
+      indexColf = new Text(options.get(indexFamilyOptionName));
+    if (options.containsKey(docFamilyOptionName))
+      docColf = new Text(options.get(docFamilyOptionName));
     docSource = source.deepCopy(env);
     indexColfSet = Collections.singleton((ByteSequence) new ArrayByteSequence(indexColf.getBytes(), 0, indexColf.getLength()));
   }
@@ -122,8 +124,10 @@ public class FamilyIntersectingIterator 
   @Override
   protected void advanceToIntersection() throws IOException {
     super.advanceToIntersection();
-    if (topKey == null) return;
-    if (log.isTraceEnabled()) log.trace("using top key to seek for doc: " + topKey.toString());
+    if (topKey == null)
+      return;
+    if (log.isTraceEnabled())
+      log.trace("using top key to seek for doc: " + topKey.toString());
     Key docKey = buildDocKey();
     docSource.seek(new Range(docKey, true, null, false), docColfSet, true);
     log.debug("got doc key: " + docSource.getTopKey().toString());
@@ -134,18 +138,22 @@ public class FamilyIntersectingIterator 
   }
   
   protected Key buildDocKey() {
-    if (log.isTraceEnabled()) log.trace("building doc key for " + currentPartition + " " + currentDocID);
+    if (log.isTraceEnabled())
+      log.trace("building doc key for " + currentPartition + " " + currentDocID);
     int zeroIndex = currentDocID.find("\0");
-    if (zeroIndex < 0) throw new IllegalArgumentException("bad current docID");
+    if (zeroIndex < 0)
+      throw new IllegalArgumentException("bad current docID");
     Text colf = new Text(docColf);
     colf.append(nullByte, 0, 1);
     colf.append(currentDocID.getBytes(), 0, zeroIndex);
     docColfSet = Collections.singleton((ByteSequence) new ArrayByteSequence(colf.getBytes(), 0, colf.getLength()));
-    if (log.isTraceEnabled()) log.trace(zeroIndex + " " + currentDocID.getLength());
+    if (log.isTraceEnabled())
+      log.trace(zeroIndex + " " + currentDocID.getLength());
     Text colq = new Text();
     colq.set(currentDocID.getBytes(), zeroIndex + 1, currentDocID.getLength() - zeroIndex - 2);
     Key k = new Key(currentPartition, colf, colq);
-    if (log.isTraceEnabled()) log.trace("built doc key for seek: " + k.toString());
+    if (log.isTraceEnabled())
+      log.trace("built doc key for seek: " + k.toString());
     return k;
   }
 }

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FilteringIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FilteringIterator.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FilteringIterator.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FilteringIterator.java Thu Oct 27 15:24:51 2011
@@ -80,7 +80,8 @@ public class FilteringIterator extends W
           break;
         }
       }
-      if (goodKey == true) return;
+      if (goodKey == true)
+        return;
       getSource().next();
     }
   }
@@ -137,10 +138,12 @@ public class FilteringIterator extends W
     Collection<Entry<String,String>> entries = options.entrySet();
     for (Entry<String,String> e : entries) {
       name = e.getKey();
-      if ((index = name.indexOf(".")) < 0) namesToClasses.put(name, e.getValue());
+      if ((index = name.indexOf(".")) < 0)
+        namesToClasses.put(name, e.getValue());
       else {
         subName = name.substring(0, index);
-        if (!namesToOptions.containsKey(subName)) namesToOptions.put(subName, new HashMap<String,String>());
+        if (!namesToOptions.containsKey(subName))
+          namesToOptions.put(subName, new HashMap<String,String>());
         namesToOptions.get(subName).put(name.substring(index + 1), e.getValue());
       }
     }

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FirstEntryInRowIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FirstEntryInRowIterator.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FirstEntryInRowIterator.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FirstEntryInRowIterator.java Thu Oct 27 15:24:51 2011
@@ -99,8 +99,8 @@ public class FirstEntryInRowIterator ext
         
         // determine where to seek to, but don't go beyond the user-specified range
         Key nextKey = getSource().getTopKey().followingKey(PartialKey.ROW);
-        if (!latestRange.afterEndKey(nextKey)) getSource().seek(new Range(nextKey, true, latestRange.getEndKey(), latestRange.isEndKeyInclusive()),
-            latestColumnFamilies, latestInclusive);
+        if (!latestRange.afterEndKey(nextKey))
+          getSource().seek(new Range(nextKey, true, latestRange.getEndKey(), latestRange.isEndKeyInclusive()), latestColumnFamilies, latestInclusive);
       }
     }
     lastRowFound = getSource().hasTop() ? getSource().getTopKey().getRow(lastRowFound) : null;

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IntersectingIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IntersectingIterator.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IntersectingIterator.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IntersectingIterator.java Thu Oct 27 15:24:51 2011
@@ -354,7 +354,8 @@ public class IntersectingIterator implem
   }
   
   public static String stringTopKey(SortedKeyValueIterator<Key,Value> iter) {
-    if (iter.hasTop()) return iter.getTopKey().toString();
+    if (iter.hasTop())
+      return iter.getTopKey().toString();
     return "";
   }
   
@@ -375,8 +376,10 @@ public class IntersectingIterator implem
   public static String encodeBooleans(boolean[] flags) {
     byte[] bytes = new byte[flags.length];
     for (int i = 0; i < flags.length; i++) {
-      if (flags[i]) bytes[i] = 1;
-      else bytes[i] = 0;
+      if (flags[i])
+        bytes[i] = 1;
+      else
+        bytes[i] = 0;
     }
     return new String(Base64.encodeBase64(bytes));
   }
@@ -392,13 +395,16 @@ public class IntersectingIterator implem
   
   public static boolean[] decodeBooleans(String flags) {
     // return null of there were no flags
-    if (flags == null) return null;
+    if (flags == null)
+      return null;
     
     byte[] bytes = Base64.decodeBase64(flags.getBytes());
     boolean[] bFlags = new boolean[bytes.length];
     for (int i = 0; i < bytes.length; i++) {
-      if (bytes[i] == 1) bFlags[i] = true;
-      else bFlags[i] = false;
+      if (bytes[i] == 1)
+        bFlags[i] = true;
+      else
+        bFlags[i] = false;
     }
     return bFlags;
   }

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java Thu Oct 27 15:24:51 2011
@@ -93,7 +93,8 @@ public class IteratorUtil {
     int max = 0;
     
     for (IterInfo iterInfo : iters) {
-      if (iterInfo.priority > max) max = iterInfo.priority;
+      if (iterInfo.priority > max)
+        max = iterInfo.priority;
     }
     
     return max;
@@ -182,7 +183,8 @@ public class IteratorUtil {
     parseIterConf(scope, iters, allOptions, conf);
     
     for (Entry<String,Map<String,String>> entry : ssio.entrySet()) {
-      if (entry.getValue() == null) continue;
+      if (entry.getValue() == null)
+        continue;
       Map<String,String> options = allOptions.get(entry.getKey());
       if (options == null) {
         allOptions.put(entry.getKey(), entry.getValue());
@@ -207,7 +209,8 @@ public class IteratorUtil {
         
         Map<String,String> options = iterOpts.get(iterInfo.iterName);
         
-        if (options == null) options = Collections.emptyMap();
+        if (options == null)
+          options = Collections.emptyMap();
         
         skvi.init(prev, options, env);
         prev = skvi;

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/LargeRowFilter.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/LargeRowFilter.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/LargeRowFilter.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/LargeRowFilter.java Thu Oct 27 15:24:51 2011
@@ -219,7 +219,8 @@ public class LargeRowFilter implements S
       while (currentPosition < keys.size() && range.beforeStartKey(keys.get(currentPosition)))
         currentPosition++;
       
-      if (currentPosition == keys.size()) readNextRow();
+      if (currentPosition == keys.size())
+        readNextRow();
       
     } else {
       source.seek(range, columnFamilies, inclusive);

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/LongCombiner.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/LongCombiner.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/LongCombiner.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/LongCombiner.java Thu Oct 27 15:24:51 2011
@@ -37,7 +37,8 @@ public abstract class LongCombiner exten
   @Override
   public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
     super.init(source, options, env);
-    if (options.get(TYPE) == null) throw new IOException("no type specified");
+    if (options.get(TYPE) == null)
+      throw new IOException("no type specified");
     switch (Type.valueOf(options.get(TYPE))) {
       case VARNUM:
         encoder = new VarNumEncoder();
@@ -64,7 +65,8 @@ public abstract class LongCombiner exten
   
   @Override
   public boolean validateOptions(Map<String,String> options) {
-    if (options.get(TYPE) == null) return false;
+    if (options.get(TYPE) == null)
+      return false;
     try {
       Type.valueOf(options.get(TYPE));
     } catch (Exception e) {
@@ -120,8 +122,8 @@ public abstract class LongCombiner exten
     }
     
     public static long decode(byte[] b, int offset) {
-      if (b.length < offset + 8) throw new NumberFormatException("trying to convert to long, but byte array isn't long enough, wanted " + (offset + 8)
-          + " found " + b.length);
+      if (b.length < offset + 8)
+        throw new NumberFormatException("trying to convert to long, but byte array isn't long enough, wanted " + (offset + 8) + " found " + b.length);
       return (((long) b[offset + 0] << 56) + ((long) (b[offset + 1] & 255) << 48) + ((long) (b[offset + 2] & 255) << 40) + ((long) (b[offset + 3] & 255) << 32)
           + ((long) (b[offset + 4] & 255) << 24) + ((b[offset + 5] & 255) << 16) + ((b[offset + 6] & 255) << 8) + ((b[offset + 7] & 255) << 0));
     }
@@ -144,9 +146,11 @@ public abstract class LongCombiner exten
     long bSign = Long.signum(b);
     if ((aSign != 0) && (bSign != 0) && (aSign == bSign)) {
       if (aSign > 0) {
-        if (Long.MAX_VALUE - a < b) return Long.MAX_VALUE;
+        if (Long.MAX_VALUE - a < b)
+          return Long.MAX_VALUE;
       } else {
-        if (Long.MIN_VALUE - a > b) return Long.MIN_VALUE;
+        if (Long.MIN_VALUE - a > b)
+          return Long.MIN_VALUE;
       }
     }
     return a + b;

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/OptionDescriber.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/OptionDescriber.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/OptionDescriber.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/OptionDescriber.java Thu Oct 27 15:24:51 2011
@@ -51,9 +51,11 @@ public interface OptionDescriber {
     public IteratorOptions(String name, String description, Map<String,String> namedOptions, List<String> unnamedOptionDescriptions) {
       this.name = name;
       this.namedOptions = null;
-      if (namedOptions != null) this.namedOptions = new LinkedHashMap<String,String>(namedOptions);
+      if (namedOptions != null)
+        this.namedOptions = new LinkedHashMap<String,String>(namedOptions);
       this.unnamedOptionDescriptions = null;
-      if (unnamedOptionDescriptions != null) this.unnamedOptionDescriptions = new ArrayList<String>(unnamedOptionDescriptions);
+      if (unnamedOptionDescriptions != null)
+        this.unnamedOptionDescriptions = new ArrayList<String>(unnamedOptionDescriptions);
       this.description = description;
     }
     
@@ -90,12 +92,14 @@ public interface OptionDescriber {
     }
     
     public void addNamedOption(String name, String description) {
-      if (namedOptions == null) namedOptions = new LinkedHashMap<String,String>();
+      if (namedOptions == null)
+        namedOptions = new LinkedHashMap<String,String>();
       namedOptions.put(name, description);
     }
     
     public void addUnnamedOption(String description) {
-      if (unnamedOptionDescriptions == null) unnamedOptionDescriptions = new ArrayList<String>();
+      if (unnamedOptionDescriptions == null)
+        unnamedOptionDescriptions = new ArrayList<String>();
       unnamedOptionDescriptions.add(description);
     }
   }

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/OrIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/OrIterator.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/OrIterator.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/OrIterator.java Thu Oct 27 15:24:51 2011
@@ -90,13 +90,15 @@ public class OrIterator implements Sorte
   @Override
   final public void next() throws IOException {
     
-    if (currentTerm == null) return;
+    if (currentTerm == null)
+      return;
     
     // Advance currentTerm
     currentTerm.iter.next();
     
     // See if currentTerm is still valid, remove if not
-    if (!(currentTerm.iter.hasTop()) || ((currentTerm.term != null) && (currentTerm.term.compareTo(currentTerm.iter.getTopKey().getColumnFamily()) != 0))) currentTerm = null;
+    if (!(currentTerm.iter.hasTop()) || ((currentTerm.term != null) && (currentTerm.term.compareTo(currentTerm.iter.getTopKey().getColumnFamily()) != 0)))
+      currentTerm = null;
     
     // optimization.
     // if size == 0, currentTerm is the only item left,
@@ -104,7 +106,8 @@ public class OrIterator implements Sorte
     // In either case, we don't need to use the PriorityQueue
     if (sorted.size() > 0) {
       // sort the term back in
-      if (currentTerm != null) sorted.add(currentTerm);
+      if (currentTerm != null)
+        sorted.add(currentTerm);
       // and get the current top item out.
       currentTerm = sorted.poll();
     }
@@ -124,17 +127,19 @@ public class OrIterator implements Sorte
     // and we don't have a priority queue of size 0 or 1.
     if (sources.size() == 1) {
       
-      if (currentTerm == null) currentTerm = sources.get(0);
+      if (currentTerm == null)
+        currentTerm = sources.get(0);
       Range newRange = null;
       
       if (range != null) {
-        if ((range.getStartKey() == null) || (range.getStartKey().getRow() == null)) newRange = range;
+        if ((range.getStartKey() == null) || (range.getStartKey().getRow() == null))
+          newRange = range;
         else {
           Key newKey = null;
-          if (range.getStartKey().getColumnQualifier() == null) newKey = new Key(range.getStartKey().getRow(), (currentTerm.term == null) ? nullText
-              : currentTerm.term);
-          else newKey = new Key(range.getStartKey().getRow(), (currentTerm.term == null) ? nullText : currentTerm.term, range.getStartKey()
-              .getColumnQualifier());
+          if (range.getStartKey().getColumnQualifier() == null)
+            newKey = new Key(range.getStartKey().getRow(), (currentTerm.term == null) ? nullText : currentTerm.term);
+          else
+            newKey = new Key(range.getStartKey().getRow(), (currentTerm.term == null) ? nullText : currentTerm.term, range.getStartKey().getColumnQualifier());
           newRange = new Range((newKey == null) ? nullKey : newKey, true, range.getEndKey(), false);
         }
       }
@@ -145,7 +150,8 @@ public class OrIterator implements Sorte
       // 1) NOT an iterator
       // 2) we have seeked into the next term (ie: seek man, get man001)
       // then ignore it as a valid source
-      if (!(currentTerm.iter.hasTop()) || ((currentTerm.term != null) && (currentTerm.term.compareTo(currentTerm.iter.getTopKey().getColumnFamily()) != 0))) currentTerm = null;
+      if (!(currentTerm.iter.hasTop()) || ((currentTerm.term != null) && (currentTerm.term.compareTo(currentTerm.iter.getTopKey().getColumnFamily()) != 0)))
+        currentTerm = null;
       
       // Otherwise, source is valid.
       return;
@@ -162,7 +168,8 @@ public class OrIterator implements Sorte
       for (TermSource TS : sources) {
         TS.iter.seek(range, columnFamilies, inclusive);
         
-        if ((TS.iter.hasTop()) && ((TS.term != null) && (TS.term.compareTo(TS.iter.getTopKey().getColumnFamily()) == 0))) sorted.add(TS);
+        if ((TS.iter.hasTop()) && ((TS.term != null) && (TS.term.compareTo(TS.iter.getTopKey().getColumnFamily()) == 0)))
+          sorted.add(TS);
       }
       currentTerm = sorted.poll();
       return;
@@ -177,11 +184,14 @@ public class OrIterator implements Sorte
       Range newRange = null;
       
       if (range != null) {
-        if ((range.getStartKey() == null) || (range.getStartKey().getRow() == null)) newRange = range;
+        if ((range.getStartKey() == null) || (range.getStartKey().getRow() == null))
+          newRange = range;
         else {
           Key newKey = null;
-          if (range.getStartKey().getColumnQualifier() == null) newKey = new Key(range.getStartKey().getRow(), (TS.term == null) ? nullText : TS.term);
-          else newKey = new Key(range.getStartKey().getRow(), (TS.term == null) ? nullText : TS.term, range.getStartKey().getColumnQualifier());
+          if (range.getStartKey().getColumnQualifier() == null)
+            newKey = new Key(range.getStartKey().getRow(), (TS.term == null) ? nullText : TS.term);
+          else
+            newKey = new Key(range.getStartKey().getRow(), (TS.term == null) ? nullText : TS.term, range.getStartKey().getColumnQualifier());
           newRange = new Range((newKey == null) ? nullKey : newKey, true, range.getEndKey(), false);
         }
       }
@@ -193,7 +203,8 @@ public class OrIterator implements Sorte
       // 1) NOT an iterator
       // 2) we have seeked into the next term (ie: seek man, get man001)
       // then ignore it as a valid source
-      if (!(TS.iter.hasTop()) || ((TS.term != null) && (TS.term.compareTo(TS.iter.getTopKey().getColumnFamily()) != 0))) iter.remove();
+      if (!(TS.iter.hasTop()) || ((TS.term != null) && (TS.term.compareTo(TS.iter.getTopKey().getColumnFamily()) != 0)))
+        iter.remove();
       
       // Otherwise, source is valid. Add it to the sources.
       sorted.add(TS);

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/RegExIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/RegExIterator.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/RegExIterator.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/RegExIterator.java Thu Oct 27 15:24:51 2011
@@ -85,10 +85,14 @@ public class RegExIterator extends Skipp
    *          if true, any of the non-null terms can match to return the entry
    */
   public static void setRegexs(IteratorSetting si, String rowTerm, String cfTerm, String cqTerm, String valueTerm, boolean orFields) {
-    if (rowTerm != null) si.addOption(RegExFilter.ROW_REGEX, rowTerm);
-    if (cfTerm != null) si.addOption(RegExFilter.COLF_REGEX, cfTerm);
-    if (cqTerm != null) si.addOption(RegExFilter.COLQ_REGEX, cqTerm);
-    if (valueTerm != null) si.addOption(RegExFilter.VALUE_REGEX, valueTerm);
+    if (rowTerm != null)
+      si.addOption(RegExFilter.ROW_REGEX, rowTerm);
+    if (cfTerm != null)
+      si.addOption(RegExFilter.COLF_REGEX, cfTerm);
+    if (cqTerm != null)
+      si.addOption(RegExFilter.COLQ_REGEX, cqTerm);
+    if (valueTerm != null)
+      si.addOption(RegExFilter.VALUE_REGEX, valueTerm);
     if (orFields) {
       si.addOption(RegExFilter.OR_FIELDS, "true");
     }

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/RowDeletingIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/RowDeletingIterator.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/RowDeletingIterator.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/RowDeletingIterator.java Thu Oct 27 15:24:51 2011
@@ -127,7 +127,8 @@ public class RowDeletingIterator impleme
         currentRowDeleted = true;
         deleteTS = source.getTopKey().getTimestamp();
         
-        if (propogateDeletes) break;
+        if (propogateDeletes)
+          break;
       } else {
         break;
       }

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/SortedMapIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/SortedMapIterator.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/SortedMapIterator.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/SortedMapIterator.java Thu Oct 27 15:24:51 2011
@@ -81,16 +81,19 @@ public class SortedMapIterator implement
   @Override
   public void next() throws IOException {
     
-    if (entry == null) throw new IllegalStateException();
+    if (entry == null)
+      throw new IllegalStateException();
     
-    if (interruptFlag != null && interruptCheckCount++ % 100 == 0 && interruptFlag.get()) throw new IterationInterruptedException();
+    if (interruptFlag != null && interruptCheckCount++ % 100 == 0 && interruptFlag.get())
+      throw new IterationInterruptedException();
     
     if (iter.hasNext()) {
       entry = iter.next();
       if (range.afterEndKey((Key) entry.getKey())) {
         entry = null;
       }
-    } else entry = null;
+    } else
+      entry = null;
     
   }
   
@@ -101,7 +104,8 @@ public class SortedMapIterator implement
       throw new IllegalArgumentException("I do not know how to filter column families");
     }
     
-    if (interruptFlag != null && interruptFlag.get()) throw new IterationInterruptedException();
+    if (interruptFlag != null && interruptFlag.get())
+      throw new IterationInterruptedException();
     
     this.range = range;
     
@@ -116,7 +120,8 @@ public class SortedMapIterator implement
       if (range.afterEndKey(entry.getKey())) {
         entry = null;
       }
-    } else entry = null;
+    } else
+      entry = null;
     
     while (hasTop() && range.beforeStartKey(getTopKey())) {
       next();

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/TypedValueCombiner.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/TypedValueCombiner.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/TypedValueCombiner.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/TypedValueCombiner.java Thu Oct 27 15:24:51 2011
@@ -45,7 +45,8 @@ public abstract class TypedValueCombiner
     
     @Override
     public V next() {
-      if (!source.hasNext()) throw new NoSuchElementException();
+      if (!source.hasNext())
+        throw new NoSuchElementException();
       return encoder.decode(source.next().get());
     }
     

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/VersioningIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/VersioningIterator.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/VersioningIterator.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/VersioningIterator.java Thu Oct 27 15:24:51 2011
@@ -47,7 +47,8 @@ public class VersioningIterator extends 
   public VersioningIterator() {}
   
   public VersioningIterator(SortedKeyValueIterator<Key,Value> iterator, int maxVersions) {
-    if (maxVersions < 1) throw new IllegalArgumentException("maxVersions for versioning iterator must be >= 1");
+    if (maxVersions < 1)
+      throw new IllegalArgumentException("maxVersions for versioning iterator must be >= 1");
     this.setSource(iterator);
     this.maxVersions = maxVersions;
   }
@@ -98,7 +99,8 @@ public class VersioningIterator extends 
   }
   
   private void resetVersionCount() {
-    if (super.hasTop()) currentKey.set(getSource().getTopKey());
+    if (super.hasTop())
+      currentKey.set(getSource().getTopKey());
     numVersions = 1;
   }
   
@@ -117,10 +119,13 @@ public class VersioningIterator extends 
     this.numVersions = 0;
     
     String maxVerString = options.get("maxVersions");
-    if (maxVerString != null) this.maxVersions = Integer.parseInt(maxVerString);
-    else this.maxVersions = 1;
+    if (maxVerString != null)
+      this.maxVersions = Integer.parseInt(maxVerString);
+    else
+      this.maxVersions = 1;
     
-    if (maxVersions < 1) throw new IllegalArgumentException("maxVersions for versioning iterator must be >= 1");
+    if (maxVersions < 1)
+      throw new IllegalArgumentException("maxVersions for versioning iterator must be >= 1");
   }
   
   @Override
@@ -134,7 +139,8 @@ public class VersioningIterator extends 
   @Override
   public boolean validateOptions(Map<String,String> options) {
     int i = Integer.parseInt(options.get(MAXVERSIONS_OPT));
-    if (i < 1) throw new IllegalArgumentException(MAXVERSIONS_OPT + " for versioning iterator must be >= 1");
+    if (i < 1)
+      throw new IllegalArgumentException(MAXVERSIONS_OPT + " for versioning iterator must be >= 1");
     return true;
   }
   

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/WholeRowIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/WholeRowIterator.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/WholeRowIterator.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/WholeRowIterator.java Thu Oct 27 15:24:51 2011
@@ -148,10 +148,12 @@ public class WholeRowIterator implements
   List<Value> values = new ArrayList<Value>();
   
   private void prepKeys() throws IOException {
-    if (topKey != null) return;
+    if (topKey != null)
+      return;
     Text currentRow;
     do {
-      if (sourceIter.hasTop() == false) return;
+      if (sourceIter.hasTop() == false)
+        return;
       currentRow = new Text(sourceIter.getTopKey().getRow());
       keys.clear();
       values.clear();
@@ -183,7 +185,8 @@ public class WholeRowIterator implements
   
   @Override
   public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
-    if (sourceIter != null) return new WholeRowIterator(sourceIter.deepCopy(env));
+    if (sourceIter != null)
+      return new WholeRowIterator(sourceIter.deepCopy(env));
     return new WholeRowIterator();
   }
   
@@ -226,7 +229,8 @@ public class WholeRowIterator implements
       // assuming that we are seeking using a key previously returned by this iterator
       // therefore go to the next row
       Key followingRowKey = sk.followingKey(PartialKey.ROW);
-      if (range.getEndKey() != null && followingRowKey.compareTo(range.getEndKey()) > 0) return;
+      if (range.getEndKey() != null && followingRowKey.compareTo(range.getEndKey()) > 0)
+        return;
       
       range = new Range(sk.followingKey(PartialKey.ROW), true, range.getEndKey(), range.isEndKeyInclusive());
     }

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java Thu Oct 27 15:24:51 2011
@@ -35,7 +35,8 @@ public abstract class WrappingIterator i
   }
   
   protected SortedKeyValueIterator<Key,Value> getSource() {
-    if (source == null) throw new IllegalStateException("getting null source");
+    if (source == null)
+      throw new IllegalStateException("getting null source");
     return source;
   }
   
@@ -46,22 +47,28 @@ public abstract class WrappingIterator i
   
   @Override
   public Key getTopKey() {
-    if (source == null) throw new IllegalStateException("no source set");
-    if (seenSeek == false) throw new IllegalStateException("never been seeked");
+    if (source == null)
+      throw new IllegalStateException("no source set");
+    if (seenSeek == false)
+      throw new IllegalStateException("never been seeked");
     return getSource().getTopKey();
   }
   
   @Override
   public Value getTopValue() {
-    if (source == null) throw new IllegalStateException("no source set");
-    if (seenSeek == false) throw new IllegalStateException("never been seeked");
+    if (source == null)
+      throw new IllegalStateException("no source set");
+    if (seenSeek == false)
+      throw new IllegalStateException("never been seeked");
     return getSource().getTopValue();
   }
   
   @Override
   public boolean hasTop() {
-    if (source == null) throw new IllegalStateException("no source set");
-    if (seenSeek == false) throw new IllegalStateException("never been seeked");
+    if (source == null)
+      throw new IllegalStateException("no source set");
+    if (seenSeek == false)
+      throw new IllegalStateException("never been seeked");
     return getSource().hasTop();
   }
   
@@ -73,8 +80,10 @@ public abstract class WrappingIterator i
   
   @Override
   public void next() throws IOException {
-    if (source == null) throw new IllegalStateException("no source set");
-    if (seenSeek == false) throw new IllegalStateException("never been seeked");
+    if (source == null)
+      throw new IllegalStateException("no source set");
+    if (seenSeek == false)
+      throw new IllegalStateException("never been seeked");
     getSource().next();
   }
   

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/LongSummation.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/LongSummation.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/LongSummation.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/LongSummation.java Thu Oct 27 15:24:51 2011
@@ -46,8 +46,8 @@ public class LongSummation implements Ag
   }
   
   public static long bytesToLong(byte[] b, int offset) throws IOException {
-    if (b.length < offset + 8) throw new IOException("trying to convert to long, but byte array isn't long enough, wanted " + (offset + 8) + " found "
-        + b.length);
+    if (b.length < offset + 8)
+      throw new IOException("trying to convert to long, but byte array isn't long enough, wanted " + (offset + 8) + " found " + b.length);
     return (((long) b[offset + 0] << 56) + ((long) (b[offset + 1] & 255) << 48) + ((long) (b[offset + 2] & 255) << 40) + ((long) (b[offset + 3] & 255) << 32)
         + ((long) (b[offset + 4] & 255) << 24) + ((b[offset + 5] & 255) << 16) + ((b[offset + 6] & 255) << 8) + ((b[offset + 7] & 255) << 0));
   }

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/NumSummation.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/NumSummation.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/NumSummation.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/NumSummation.java Thu Oct 27 15:24:51 2011
@@ -66,9 +66,11 @@ public class NumSummation implements Agg
     long bSign = Long.signum(b);
     if ((aSign != 0) && (bSign != 0) && (aSign == bSign)) {
       if (aSign > 0) {
-        if (Long.MAX_VALUE - a < b) return Long.MAX_VALUE;
+        if (Long.MAX_VALUE - a < b)
+          return Long.MAX_VALUE;
       } else {
-        if (Long.MIN_VALUE - a > b) return Long.MIN_VALUE;
+        if (Long.MIN_VALUE - a > b)
+          return Long.MIN_VALUE;
       }
     }
     return a + b;

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnSet.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnSet.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnSet.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnSet.java Thu Oct 27 15:24:51 2011
@@ -64,7 +64,8 @@ public class ColumnSet {
     // lookup column family and column qualifier
     if (objectsCol.size() > 0) {
       lookupCol.set(key);
-      if (objectsCol.contains(lookupCol)) return true;
+      if (objectsCol.contains(lookupCol))
+        return true;
     }
     
     // lookup just column family

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnUtil.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnUtil.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnUtil.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnUtil.java Thu Oct 27 15:24:51 2011
@@ -61,12 +61,14 @@ public class ColumnUtil {
     }
     
     public boolean equals(Object o) {
-      if (o instanceof ColFamHashKey) return equals((ColFamHashKey) o);
+      if (o instanceof ColFamHashKey)
+        return equals((ColFamHashKey) o);
       return false;
     }
     
     public boolean equals(ColFamHashKey ohk) {
-      if (columnFamily == null) return key.compareColumnFamily(ohk.columnFamily) == 0;
+      if (columnFamily == null)
+        return key.compareColumnFamily(ohk.columnFamily) == 0;
       return ohk.key.compareColumnFamily(columnFamily) == 0;
     }
   }
@@ -100,12 +102,14 @@ public class ColumnUtil {
     }
     
     public boolean equals(Object o) {
-      if (o instanceof ColHashKey) return equals((ColHashKey) o);
+      if (o instanceof ColHashKey)
+        return equals((ColHashKey) o);
       return false;
     }
     
     public boolean equals(ColHashKey ohk) {
-      if (columnFamily == null) return key.compareColumnFamily(ohk.columnFamily) == 0 && key.compareColumnQualifier(ohk.columnQualifier) == 0;
+      if (columnFamily == null)
+        return key.compareColumnFamily(ohk.columnFamily) == 0 && key.compareColumnQualifier(ohk.columnQualifier) == 0;
       return ohk.key.compareColumnFamily(columnFamily) == 0 && ohk.key.compareColumnQualifier(columnQualifier) == 0;
     }
   }

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/filter/AgeOffFilter.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/filter/AgeOffFilter.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/filter/AgeOffFilter.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/filter/AgeOffFilter.java Thu Oct 27 15:24:51 2011
@@ -30,23 +30,28 @@ public class AgeOffFilter implements Fil
   
   @Override
   public boolean accept(Key k, Value v) {
-    if (currentTime - k.getTimestamp() > threshold) return false;
+    if (currentTime - k.getTimestamp() > threshold)
+      return false;
     return true;
   }
   
   @Override
   public void init(Map<String,String> options) {
     threshold = -1;
-    if (options == null) throw new IllegalArgumentException("ttl must be set for AgeOffFilter");
+    if (options == null)
+      throw new IllegalArgumentException("ttl must be set for AgeOffFilter");
     
     String ttl = options.get("ttl");
-    if (ttl == null) throw new IllegalArgumentException("ttl must be set for AgeOffFilter");
+    if (ttl == null)
+      throw new IllegalArgumentException("ttl must be set for AgeOffFilter");
     
     threshold = Long.parseLong(ttl);
     
     String time = options.get("currentTime");
-    if (time != null) currentTime = Long.parseLong(time);
-    else currentTime = System.currentTimeMillis();
+    if (time != null)
+      currentTime = Long.parseLong(time);
+    else
+      currentTime = System.currentTimeMillis();
     
     // add sanity checks for threshold and currentTime?
   }

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/filter/ColumnAgeOffFilter.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/filter/ColumnAgeOffFilter.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/filter/ColumnAgeOffFilter.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/filter/ColumnAgeOffFilter.java Thu Oct 27 15:24:51 2011
@@ -55,8 +55,10 @@ public class ColumnAgeOffFilter implemen
   @Override
   public boolean accept(Key k, Value v) {
     Long threshold = ttls.getObject(k);
-    if (threshold == null) return true;
-    if (currentTime - k.getTimestamp() > threshold) return false;
+    if (threshold == null)
+      return true;
+    if (currentTime - k.getTimestamp() > threshold)
+      return false;
     return true;
   }
   

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/filter/ColumnQualifierFilter.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/filter/ColumnQualifierFilter.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/filter/ColumnQualifierFilter.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/filter/ColumnQualifierFilter.java Thu Oct 27 15:24:51 2011
@@ -38,9 +38,11 @@ public class ColumnQualifierFilter imple
   }
   
   public boolean accept(Key key, Value v) {
-    if (!scanColumns) return true;
+    if (!scanColumns)
+      return true;
     
-    if (columnFamilies.contains(key.getColumnFamilyData())) return true;
+    if (columnFamilies.contains(key.getColumnFamilyData()))
+      return true;
     
     HashSet<ByteSequence> cfset = columnsQualifiers.get(key.getColumnQualifierData());
     // ensure the columm qualifier goes with a paired column family,

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/filter/RegExFilter.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/filter/RegExFilter.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/filter/RegExFilter.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/filter/RegExFilter.java Thu Oct 27 15:24:51 2011
@@ -67,8 +67,9 @@ public class RegExFilter implements Filt
   
   @Override
   public boolean accept(Key key, Value value) {
-    if (orFields) return matches(rowMatcher, key.getRowData()) || matches(colfMatcher, key.getColumnFamilyData())
-        || matches(colqMatcher, key.getColumnQualifierData()) || matches(valueMatcher, value.get(), 0, value.get().length);
+    if (orFields)
+      return matches(rowMatcher, key.getRowData()) || matches(colfMatcher, key.getColumnFamilyData()) || matches(colqMatcher, key.getColumnQualifierData())
+          || matches(valueMatcher, value.get(), 0, value.get().length);
     return matches(rowMatcher, key.getRowData()) && matches(colfMatcher, key.getColumnFamilyData()) && matches(colqMatcher, key.getColumnQualifierData())
         && matches(valueMatcher, value.get(), 0, value.get().length);
   }
@@ -120,13 +121,17 @@ public class RegExFilter implements Filt
   
   @Override
   public boolean validateOptions(Map<String,String> options) {
-    if (options.containsKey(ROW_REGEX)) Pattern.compile(options.get(ROW_REGEX)).matcher("");
+    if (options.containsKey(ROW_REGEX))
+      Pattern.compile(options.get(ROW_REGEX)).matcher("");
     
-    if (options.containsKey(COLF_REGEX)) Pattern.compile(options.get(COLF_REGEX)).matcher("");
+    if (options.containsKey(COLF_REGEX))
+      Pattern.compile(options.get(COLF_REGEX)).matcher("");
     
-    if (options.containsKey(COLQ_REGEX)) Pattern.compile(options.get(COLQ_REGEX)).matcher("");
+    if (options.containsKey(COLQ_REGEX))
+      Pattern.compile(options.get(COLQ_REGEX)).matcher("");
     
-    if (options.containsKey(VALUE_REGEX)) Pattern.compile(options.get(VALUE_REGEX)).matcher("");
+    if (options.containsKey(VALUE_REGEX))
+      Pattern.compile(options.get(VALUE_REGEX)).matcher("");
     
     return true;
   }

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/filter/VisibilityFilter.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/filter/VisibilityFilter.java?rev=1189806&r1=1189805&r2=1189806&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/filter/VisibilityFilter.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/filter/VisibilityFilter.java Thu Oct 27 15:24:51 2011
@@ -47,11 +47,14 @@ public class VisibilityFilter implements
   public boolean accept(Key k, Value v) {
     Text testVis = k.getColumnVisibility(tmpVis);
     
-    if (testVis.getLength() == 0 && defaultVisibility.getLength() == 0) return true;
-    else if (testVis.getLength() == 0) testVis = defaultVisibility;
+    if (testVis.getLength() == 0 && defaultVisibility.getLength() == 0)
+      return true;
+    else if (testVis.getLength() == 0)
+      testVis = defaultVisibility;
     
     Boolean b = (Boolean) cache.get(testVis);
-    if (b != null) return b;
+    if (b != null)
+      return b;
     
     try {
       Boolean bb = ve.evaluate(new ColumnVisibility(testVis));