You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by bi...@apache.org on 2011/10/27 21:33:49 UTC

svn commit: r1189962 - in /incubator/accumulo: branches/1.3/src/core/src/main/java/org/apache/accumulo/core/iterators/ trunk/contrib/accumulo_sample/query/src/main/java/iterator/ trunk/docs/ trunk/docs/src/user_manual/ trunk/src/core/src/main/java/org/...

Author: billie
Date: Thu Oct 27 19:33:48 2011
New Revision: 1189962

URL: http://svn.apache.org/viewvc?rev=1189962&view=rev
Log:
ACCUMULO-15 documented and defined iterator behavior in SortedKeyValueIterator interface, changed a number of iterators to conform to the IllegalArgumentException specification

Modified:
    incubator/accumulo/branches/1.3/src/core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java
    incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/AbstractEvaluatingIterator.java
    incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/AndIterator.java
    incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/BooleanLogicIterator.java
    incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/OptimizedQueryIterator.java
    incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/UniqFieldNameValueIterator.java
    incubator/accumulo/trunk/docs/   (props changed)
    incubator/accumulo/trunk/docs/src/user_manual/   (props changed)
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/AggregatingIterator.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FilteringIterator.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IntersectingIterator.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/LongCombiner.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingArrayCombiner.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/TimestampFilter.java
    incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/filedata/ChunkCombiner.java

Modified: incubator/accumulo/branches/1.3/src/core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.3/src/core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java?rev=1189962&r1=1189961&r2=1189962&view=diff
==============================================================================
--- incubator/accumulo/branches/1.3/src/core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java (original)
+++ incubator/accumulo/branches/1.3/src/core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java Thu Oct 27 19:33:48 2011
@@ -26,32 +26,80 @@ import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 
 /**
- * An iterator that support iterating over key and value pairs. Anything implementing this interface should return keys in sorted order.
- * 
- * 
+ * An iterator that supports iterating over key and value pairs. Anything implementing this interface should return keys in sorted order.
  */
 
 public interface SortedKeyValueIterator<K extends WritableComparable<?>,V extends Writable> {
-  
+  /**
+   * Initializes the iterator.  Data should not be read from the source in this method.
+   * 
+   * @param source <tt>SortedKeyValueIterator</tt> source to read data from.
+   * @param options <tt>Map</tt> map of string option names to option values.
+   * @param env <tt>IteratorEnvironment</tt> environment in which iterator is being run.
+   * @throws IOException TODO
+   * @exception IllegalArgumentException if there are problems with the options.
+   * @exception UnsupportedOperationException if not supported.
+   * @exception IOException
+   */
   void init(SortedKeyValueIterator<K,V> source, Map<String,String> options, IteratorEnvironment env) throws IOException;
   
-  // we should add method to get a continue key that appropriately translates
-  
+  /**
+   * Returns true if the iterator has more elements.
+   * 
+   * @return <tt>true</tt> if the iterator has more elements.
+   * @exception IllegalStateException if called before seek.
+   */
   boolean hasTop();
   
+  /**
+   * Advances to the next K,V pair.
+   * 
+   * @throws IOException if an I/O error occurs.
+   * @throws IllegalStateException if called before seek.
+   * @exception NoSuchElementException if next element doesn't exist.
+   */
   void next() throws IOException;
   
   /**
    * An iterator must seek to the first key in the range taking inclusiveness into account. However, an iterator does not have to stop at the end of the range.
    * The whole range is provided so that iterators can make optimizations.
+   * 
+   * @param range <tt>Range</tt> of keys to iterate over.
+   * @param columnFamilies <tt>Collection</tt> of column families to include or exclude.
+   * @param inclusive <tt>boolean</tt> that indicates whether to include (true) or exclude (false) column families.
+   * @throws IOException if an I/O error occurs.
+   * @exception IllegalArgumentException if there are problems with the parameters.
    */
   void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException;
   
+  /**
+   * Returns top key.  Can be called 0 or more times without affecting behavior of next() or hasTop().
+   * 
+   * @return <tt>K</tt>
+   * @exception IllegalStateException if called before seek.
+   * @exception NoSuchElementException if top element doesn't exist.
+   */
   K getTopKey();
   
+  /**
+   * Returns top value.  Can be called 0 or more times without affecting behavior of next() or hasTop().
+   * 
+   * @return <tt>V</tt>
+   * @exception IllegalStateException if called before seek.
+   * @exception NoSuchElementException if top element doesn't exist.
+   */
   V getTopValue();
   
-  // create a deep copy of this iterator as though seek had not yet been called
-  // init must not be called after clone, on either of the instances
+  /**
+   * Creates a deep copy of this iterator as though seek had not yet been called.
+   * init should be called on an iterator before deepCopy is called. 
+   * init should not need to be called on the copy that is returned by deepCopy; that is,
+   * when necessary init should be called in the deepCopy method on the iterator it returns.
+   * The behavior is unspecified if init is called after deepCopy either on the original or the copy.
+   * 
+   * @param env <tt>IteratorEnvironment</tt> environment in which iterator is being run.
+   * @return <tt>SortedKeyValueIterator</tt> a copy of this iterator (with the same source and settings)
+   * @exception UnsupportedOperationException if not supported.
+   */
   SortedKeyValueIterator<K,V> deepCopy(IteratorEnvironment env);
 }

Modified: incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/AbstractEvaluatingIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/AbstractEvaluatingIterator.java?rev=1189962&r1=1189961&r2=1189962&view=diff
==============================================================================
--- incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/AbstractEvaluatingIterator.java (original)
+++ incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/AbstractEvaluatingIterator.java Thu Oct 27 19:33:48 2011
@@ -293,7 +293,7 @@ public abstract class AbstractEvaluating
         	}
             this.evaluator = new QueryEvaluator(this.expression);
         } catch (ParseException e) {
-            throw new IOException("Failed to parse query", e);
+            throw new IllegalArgumentException("Failed to parse query", e);
         }
         EventFields.initializeKryo(kryo);
     }

Modified: incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/AndIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/AndIterator.java?rev=1189962&r1=1189961&r2=1189962&view=diff
==============================================================================
--- incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/AndIterator.java (original)
+++ incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/AndIterator.java Thu Oct 27 19:33:48 2011
@@ -20,10 +20,6 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.Map;
 
-import org.apache.commons.codec.binary.Base64;
-import org.apache.hadoop.io.Text;
-import org.apache.log4j.Logger;
-
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.PartialKey;
@@ -32,6 +28,9 @@ import org.apache.accumulo.core.data.Val
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.util.TextUtil;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.io.Text;
+import org.apache.log4j.Logger;
 
 public class AndIterator implements SortedKeyValueIterator<Key, Value> {
 
@@ -690,7 +689,7 @@ public class AndIterator implements Sort
         boolean[] notFlags = decodeBooleans(options.get(notFlagsOptionName));
 
         if (terms.length < 2) {
-            throw new IOException("AndIterator requires two or more columns families");
+            throw new IllegalArgumentException("AndIterator requires two or more columns families");
         }
 
         // Scan the not flags.
@@ -725,7 +724,7 @@ public class AndIterator implements Sort
             }
 
             if (notFlags[0]) {
-                throw new IOException("AndIterator requires at least one column family without not");
+                throw new IllegalArgumentException("AndIterator requires at least one column family without not");
             }
         }
 

Modified: incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/BooleanLogicIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/BooleanLogicIterator.java?rev=1189962&r1=1189961&r2=1189962&view=diff
==============================================================================
--- incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/BooleanLogicIterator.java (original)
+++ incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/BooleanLogicIterator.java Thu Oct 27 19:33:48 2011
@@ -31,6 +31,14 @@ import java.util.Map.Entry;
 import java.util.PriorityQueue;
 import java.util.Set;
 
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.IteratorEnvironment;
+import org.apache.accumulo.core.iterators.OptionDescriber;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.commons.jexl2.parser.ASTAndNode;
 import org.apache.commons.jexl2.parser.ASTEQNode;
 import org.apache.commons.jexl2.parser.ASTERNode;
@@ -55,14 +63,6 @@ import parser.QueryParser.QueryTerm;
 import parser.RangeCalculator.RangeBounds;
 import parser.TreeNode;
 import util.FieldIndexKeyParser;
-import org.apache.accumulo.core.data.ByteSequence;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.PartialKey;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.iterators.IteratorEnvironment;
-import org.apache.accumulo.core.iterators.OptionDescriber;
-import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 
 import com.google.common.collect.Multimap;
 
@@ -185,9 +185,9 @@ public class BooleanLogicIterator implem
 
         } catch (ParseException ex) {
             log.error("ParseException in init: " + ex);
-            throw new IOException("Failed to parse query", ex);
+            throw new IllegalArgumentException("Failed to parse query", ex);
         } catch (Exception ex) {
-            throw new IOException("probably had no indexed terms", ex);
+            throw new IllegalArgumentException("probably had no indexed terms", ex);
         }
 
     }

Modified: incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/OptimizedQueryIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/OptimizedQueryIterator.java?rev=1189962&r1=1189961&r2=1189962&view=diff
==============================================================================
--- incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/OptimizedQueryIterator.java (original)
+++ incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/OptimizedQueryIterator.java Thu Oct 27 19:33:48 2011
@@ -67,7 +67,7 @@ public class OptimizedQueryIterator impl
 
     public void init(SortedKeyValueIterator<Key, Value> source, Map<String, String> options, IteratorEnvironment env) throws IOException {
         if (!validateOptions(options)) {
-            throw new IOException("Invalid options");
+            throw new IllegalArgumentException("Invalid options");
         }
 
         //Setup the EvaluatingIterator

Modified: incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/UniqFieldNameValueIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/UniqFieldNameValueIterator.java?rev=1189962&r1=1189961&r2=1189962&view=diff
==============================================================================
--- incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/UniqFieldNameValueIterator.java (original)
+++ incubator/accumulo/trunk/contrib/accumulo_sample/query/src/main/java/iterator/UniqFieldNameValueIterator.java Thu Oct 27 19:33:48 2011
@@ -21,12 +21,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Map;
 
-import org.apache.commons.jexl2.Expression;
-import org.apache.hadoop.io.Text;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-import util.FieldIndexKeyParser;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.PartialKey;
@@ -35,6 +29,12 @@ import org.apache.accumulo.core.data.Val
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.WrappingIterator;
+import org.apache.commons.jexl2.Expression;
+import org.apache.hadoop.io.Text;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+import util.FieldIndexKeyParser;
 
 public class UniqFieldNameValueIterator extends WrappingIterator {
 

Propchange: incubator/accumulo/trunk/docs/
------------------------------------------------------------------------------
--- svn:ignore (added)
+++ svn:ignore Thu Oct 27 19:33:48 2011
@@ -0,0 +1 @@
+accumulo_user_manual.pdf

Propchange: incubator/accumulo/trunk/docs/src/user_manual/
------------------------------------------------------------------------------
--- svn:ignore (added)
+++ svn:ignore Thu Oct 27 19:33:48 2011
@@ -0,0 +1 @@
+accumulo_user_manual.toc

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java?rev=1189962&r1=1189961&r2=1189962&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java Thu Oct 27 19:33:48 2011
@@ -86,7 +86,7 @@ public class ClientSideIteratorScanner e
     @Override
     public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
       if (!inclusive && columnFamilies.size() > 0)
-        throw new UnsupportedOperationException();
+        throw new IllegalArgumentException();
       scanner.setRange(range);
       scanner.clearColumns();
       for (ByteSequence colf : columnFamilies) {

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=1189962&r1=1189961&r2=1189962&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 19:33:48 2011
@@ -170,13 +170,13 @@ public class AggregatingIterator impleme
       this.aggregators = new ColumnToClassMapping<Aggregator>(options, Aggregator.class);
     } catch (ClassNotFoundException e) {
       log.error(e.toString());
-      throw new IOException(e);
+      throw new IllegalArgumentException(e);
     } catch (InstantiationException e) {
       log.error(e.toString());
-      throw new IOException(e);
+      throw new IllegalArgumentException(e);
     } catch (IllegalAccessException e) {
       log.error(e.toString());
-      throw new IOException(e);
+      throw new IllegalArgumentException(e);
     }
   }
   

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=1189962&r1=1189961&r2=1189962&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 19:33:48 2011
@@ -107,13 +107,13 @@ public class FilteringIterator extends W
       
     } catch (ClassNotFoundException e) {
       log.error(e.toString());
-      throw new IOException(e);
+      throw new IllegalArgumentException(e);
     } catch (InstantiationException e) {
       log.error(e.toString());
-      throw new IOException(e);
+      throw new IllegalArgumentException(e);
     } catch (IllegalAccessException e) {
       log.error(e.toString());
-      throw new IOException(e);
+      throw new IllegalArgumentException(e);
     }
   }
   

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=1189962&r1=1189961&r2=1189962&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 19:33:48 2011
@@ -415,7 +415,7 @@ public class IntersectingIterator implem
     boolean[] notFlag = decodeBooleans(options.get(notFlagOptionName));
     
     if (terms.length < 2) {
-      throw new IOException("IntersectionIterator requires two or more columns families");
+      throw new IllegalArgumentException("IntersectionIterator requires two or more columns families");
     }
     
     // Scan the not flags.
@@ -438,7 +438,7 @@ public class IntersectingIterator implem
         }
       }
       if (notFlag[0]) {
-        throw new IOException("IntersectionIterator requires at lest one column family without not");
+        throw new IllegalArgumentException("IntersectionIterator requires at lest one column family without not");
       }
     }
     

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=1189962&r1=1189961&r2=1189962&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 19:33:48 2011
@@ -38,7 +38,7 @@ public abstract class LongCombiner exten
   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");
+      throw new IllegalArgumentException("no type specified");
     switch (Type.valueOf(options.get(TYPE))) {
       case VARNUM:
         encoder = new VarNumEncoder();
@@ -50,7 +50,7 @@ public abstract class LongCombiner exten
         encoder = new StringEncoder();
         return;
       default:
-        throw new UnsupportedOperationException();
+        throw new IllegalArgumentException();
     }
   }
   

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java?rev=1189962&r1=1189961&r2=1189962&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java Thu Oct 27 19:33:48 2011
@@ -26,32 +26,80 @@ import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 
 /**
- * An iterator that support iterating over key and value pairs. Anything implementing this interface should return keys in sorted order.
- * 
- * 
+ * An iterator that supports iterating over key and value pairs. Anything implementing this interface should return keys in sorted order.
  */
 
 public interface SortedKeyValueIterator<K extends WritableComparable<?>,V extends Writable> {
-  
+  /**
+   * Initializes the iterator.  Data should not be read from the source in this method.
+   * 
+   * @param source <tt>SortedKeyValueIterator</tt> source to read data from.
+   * @param options <tt>Map</tt> map of string option names to option values.
+   * @param env <tt>IteratorEnvironment</tt> environment in which iterator is being run.
+   * @throws IOException TODO
+   * @exception IllegalArgumentException if there are problems with the options.
+   * @exception UnsupportedOperationException if not supported.
+   * @exception IOException
+   */
   void init(SortedKeyValueIterator<K,V> source, Map<String,String> options, IteratorEnvironment env) throws IOException;
   
-  // we should add method to get a continue key that appropriately translates
-  
+  /**
+   * Returns true if the iterator has more elements.
+   * 
+   * @return <tt>true</tt> if the iterator has more elements.
+   * @exception IllegalStateException if called before seek.
+   */
   boolean hasTop();
   
+  /**
+   * Advances to the next K,V pair.
+   * 
+   * @throws IOException if an I/O error occurs.
+   * @throws IllegalStateException if called before seek.
+   * @exception NoSuchElementException if next element doesn't exist.
+   */
   void next() throws IOException;
   
   /**
    * An iterator must seek to the first key in the range taking inclusiveness into account. However, an iterator does not have to stop at the end of the range.
    * The whole range is provided so that iterators can make optimizations.
+   * 
+   * @param range <tt>Range</tt> of keys to iterate over.
+   * @param columnFamilies <tt>Collection</tt> of column families to include or exclude.
+   * @param inclusive <tt>boolean</tt> that indicates whether to include (true) or exclude (false) column families.
+   * @throws IOException if an I/O error occurs.
+   * @exception IllegalArgumentException if there are problems with the parameters.
    */
   void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException;
   
+  /**
+   * Returns top key.  Can be called 0 or more times without affecting behavior of next() or hasTop().
+   * 
+   * @return <tt>K</tt>
+   * @exception IllegalStateException if called before seek.
+   * @exception NoSuchElementException if top element doesn't exist.
+   */
   K getTopKey();
   
+  /**
+   * Returns top value.  Can be called 0 or more times without affecting behavior of next() or hasTop().
+   * 
+   * @return <tt>V</tt>
+   * @exception IllegalStateException if called before seek.
+   * @exception NoSuchElementException if top element doesn't exist.
+   */
   V getTopValue();
   
-  // create a deep copy of this iterator as though seek had not yet been called
-  // init must not be called after clone, on either of the instances
+  /**
+   * Creates a deep copy of this iterator as though seek had not yet been called.
+   * init should be called on an iterator before deepCopy is called. 
+   * init should not need to be called on the copy that is returned by deepCopy; that is,
+   * when necessary init should be called in the deepCopy method on the iterator it returns.
+   * The behavior is unspecified if init is called after deepCopy either on the original or the copy.
+   * 
+   * @param env <tt>IteratorEnvironment</tt> environment in which iterator is being run.
+   * @return <tt>SortedKeyValueIterator</tt> a copy of this iterator (with the same source and settings)
+   * @exception UnsupportedOperationException if not supported.
+   */
   SortedKeyValueIterator<K,V> deepCopy(IteratorEnvironment env);
 }

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingArrayCombiner.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingArrayCombiner.java?rev=1189962&r1=1189961&r2=1189962&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingArrayCombiner.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingArrayCombiner.java Thu Oct 27 19:33:48 2011
@@ -63,7 +63,7 @@ public class SummingArrayCombiner extend
   public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
     super.init(source, options, env);
     if (options.get(LongCombiner.TYPE) == null)
-      throw new IOException("no type specified");
+      throw new IllegalArgumentException("no type specified");
     switch (Type.valueOf(options.get(LongCombiner.TYPE))) {
       case VARNUM:
         encoder = new VarNumArrayEncoder();
@@ -75,7 +75,7 @@ public class SummingArrayCombiner extend
         encoder = new StringArrayEncoder();
         return;
       default:
-        throw new UnsupportedOperationException();
+        throw new IllegalArgumentException();
     }
   }
   

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/TimestampFilter.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/TimestampFilter.java?rev=1189962&r1=1189961&r2=1189962&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/TimestampFilter.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/TimestampFilter.java Thu Oct 27 19:33:48 2011
@@ -74,7 +74,7 @@ public class TimestampFilter extends Fil
       start = dateParser.parse(options.get(START)).getTime();
       end = dateParser.parse(options.get(END)).getTime();
     } catch (Exception e) {
-      throw new IOException(e);
+      throw new IllegalArgumentException(e);
     }
     if (options.get(START_INCL) != null)
       startInclusive = Boolean.parseBoolean(options.get(START_INCL));

Modified: incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/filedata/ChunkCombiner.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/filedata/ChunkCombiner.java?rev=1189962&r1=1189961&r2=1189962&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/filedata/ChunkCombiner.java (original)
+++ incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/filedata/ChunkCombiner.java Thu Oct 27 19:33:48 2011
@@ -173,7 +173,7 @@ public class ChunkCombiner implements So
     try {
       cc.init(source.deepCopy(env), null, env);
     } catch (IOException e) {
-      throw new RuntimeException(e);
+      throw new IllegalArgumentException(e);
     }
     return cc;
   }