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/11/17 16:29:08 UTC

svn commit: r1203222 - in /incubator/accumulo/trunk: ./ docs/examples/ src/core/src/main/java/org/apache/accumulo/core/client/ src/core/src/main/java/org/apache/accumulo/core/data/ src/core/src/main/java/org/apache/accumulo/core/iterators/ src/core/src...

Author: billie
Date: Thu Nov 17 15:29:07 2011
New Revision: 1203222

URL: http://svn.apache.org/viewvc?rev=1203222&view=rev
Log:
ACCUMULO-148 merged changes to trunk

Modified:
    incubator/accumulo/trunk/   (props changed)
    incubator/accumulo/trunk/docs/examples/README.constraints
    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/data/Range.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/Filter.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/OptionDescriber.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/TypedValueCombiner.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/AgeOffFilter.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/ColumnAgeOffFilter.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/MaxCombiner.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/MinCombiner.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/NoVisFilter.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/RegExFilter.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/SummingCombiner.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/TimestampFilter.java

Propchange: incubator/accumulo/trunk/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Nov 17 15:29:07 2011
@@ -1,2 +1,2 @@
 /incubator/accumulo/branches/1.3:1190280,1190413,1190420,1190427,1190500,1195622,1195625,1195629,1195635,1196044,1196054,1196057,1196071-1196072,1196106,1197066,1198935,1199383
-/incubator/accumulo/branches/1.4:1202767-1202802,1202815
+/incubator/accumulo/branches/1.4:1201902-1203219

Modified: incubator/accumulo/trunk/docs/examples/README.constraints
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/docs/examples/README.constraints?rev=1203222&r1=1203221&r2=1203222&view=diff
==============================================================================
--- incubator/accumulo/trunk/docs/examples/README.constraints (original)
+++ incubator/accumulo/trunk/docs/examples/README.constraints Thu Nov 17 15:29:07 2011
@@ -22,7 +22,7 @@ numeric keys.  The other constraint does
 inserts that violate these constraints are attempted and denied.  The scan at
 the end shows the inserts were not allowed. 
 
-    $ ./bin/accumulo shell -u username -p pass
+    $ ./bin/accumulo shell -u username -p password
     
     Shell - Accumulo Interactive Shell
     - 
@@ -34,7 +34,7 @@ the end shows the inserts were not allow
     - 
     username@instance> createtable testConstraints
     username@instance testConstraints> config -t testConstraints -s table.constraint.1=org.apache.accumulo.examples.constraints.NumericValueConstraint
-    username@instance testConstraints> config -t testConstraints -s table.constraint.2=org.apache.accumulo.examples.constraints.AlphaNumKeyConstrain                                                                                                    
+    username@instance testConstraints> config -t testConstraints -s table.constraint.2=org.apache.accumulo.examples.constraints.AlphaNumKeyConstraint
     username@instance testConstraints> insert r1 cf1 cq1 1111
     username@instance testConstraints> insert r1 cf1 cq1 ABC
       Constraint Failures:

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=1203222&r1=1203221&r2=1203222&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 Nov 17 15:29:07 2011
@@ -38,17 +38,16 @@ import org.apache.accumulo.core.data.Val
 import org.apache.accumulo.core.data.thrift.IterInfo;
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.IteratorUtil;
-import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.hadoop.io.Text;
 
-/*
- * A scanner that instantiates iterators on the client side instead of on the tablet server.  This can be useful for testing iterators or in cases where you don't want iterators affecting the performance of tablet servers.
+/**
+ * A scanner that instantiates iterators on the client side instead of on the tablet server. This can be useful for testing iterators or in cases where you
+ * don't want iterators affecting the performance of tablet servers.
  * 
- * Suggested usage:
- * Scanner scanner = new ClientSideIteratorScanner(connector.createScanner(tableName, authorizations))
+ * Suggested usage: Scanner scanner = new ClientSideIteratorScanner(connector.createScanner(tableName, authorizations))
  */
-
 public class ClientSideIteratorScanner extends ScannerOptions implements Scanner {
   private int size;
   private int timeOut;

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/data/Range.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/data/Range.java?rev=1203222&r1=1203221&r2=1203222&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/data/Range.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/data/Range.java Thu Nov 17 15:29:07 2011
@@ -512,76 +512,91 @@ public class Range implements WritableCo
     return infiniteStopKey;
   }
   
-  /*
+  /**
    * Creates a range that covers an exact row Returns the same Range as new Range(row)
    * 
-   * @param row all keys in the range will have this row
+   * @param row
+   *          all keys in the range will have this row
    */
   public static Range exact(Text row) {
     return new Range(row);
   }
   
-  /*
+  /**
    * Creates a range that covers an exact row and column family
    * 
-   * @param row all keys in the range will have this row
+   * @param row
+   *          all keys in the range will have this row
    * 
-   * @param cf all keys in the range will have this column family
+   * @param cf
+   *          all keys in the range will have this column family
    */
   public static Range exact(Text row, Text cf) {
     Key startKey = new Key(row, cf);
     return new Range(startKey, true, startKey.followingKey(PartialKey.ROW_COLFAM), false);
   }
   
-  /*
+  /**
    * Creates a range that covers an exact row, column family, and column qualifier
    * 
-   * @param row all keys in the range will have this row
+   * @param row
+   *          all keys in the range will have this row
    * 
-   * @param cf all keys in the range will have this column family
+   * @param cf
+   *          all keys in the range will have this column family
    * 
-   * @param cq all keys in the range will have this column qualifier
+   * @param cq
+   *          all keys in the range will have this column qualifier
    */
   public static Range exact(Text row, Text cf, Text cq) {
     Key startKey = new Key(row, cf, cq);
     return new Range(startKey, true, startKey.followingKey(PartialKey.ROW_COLFAM_COLQUAL), false);
   }
   
-  /*
+  /**
    * Creates a range that covers an exact row, column family, column qualifier, and visibility
    * 
-   * @param row all keys in the range will have this row
+   * @param row
+   *          all keys in the range will have this row
    * 
-   * @param cf all keys in the range will have this column family
+   * @param cf
+   *          all keys in the range will have this column family
    * 
-   * @param cq all keys in the range will have this column qualifier
+   * @param cq
+   *          all keys in the range will have this column qualifier
    * 
-   * @param cv all keys in the range will have this column visibility
+   * @param cv
+   *          all keys in the range will have this column visibility
    */
   public static Range exact(Text row, Text cf, Text cq, Text cv) {
     Key startKey = new Key(row, cf, cq, cv);
     return new Range(startKey, true, startKey.followingKey(PartialKey.ROW_COLFAM_COLQUAL_COLVIS), false);
   }
   
-  /*
+  /**
    * Creates a range that covers an exact row, column family, column qualifier, visibility, and timestamp
    * 
-   * @param row all keys in the range will have this row
+   * @param row
+   *          all keys in the range will have this row
    * 
-   * @param cf all keys in the range will have this column family
+   * @param cf
+   *          all keys in the range will have this column family
    * 
-   * @param cq all keys in the range will have this column qualifier
+   * @param cq
+   *          all keys in the range will have this column qualifier
    * 
-   * @param cv all keys in the range will have this column visibility
+   * @param cv
+   *          all keys in the range will have this column visibility
    * 
-   * @param ts all keys in the range will have this timestamp
+   * @param ts
+   *          all keys in the range will have this timestamp
    */
   public static Range exact(Text row, Text cf, Text cq, Text cv, long ts) {
     Key startKey = new Key(row, cf, cq, cv, ts);
     return new Range(startKey, true, startKey.followingKey(PartialKey.ROW_COLFAM_COLQUAL_COLVIS_TIME), false);
   }
   
-  /*
+  /**
    * Returns a Text that sorts just after all Texts beginning with a prefix
    * 
    * @param prefix
@@ -605,52 +620,62 @@ public class Range implements WritableCo
     return new Text(newBytes);
   }
   
-  /*
+  /**
    * Returns a Range that covers all rows beginning with a prefix
    * 
-   * @param rowPrefix all keys in the range will have rows that begin with this prefix
+   * @param rowPrefix
+   *          all keys in the range will have rows that begin with this prefix
    */
   public static Range prefix(Text rowPrefix) {
     Text fp = followingPrefix(rowPrefix);
     return new Range(new Key(rowPrefix), true, fp == null ? null : new Key(fp), false);
   }
   
-  /*
+  /**
    * Returns a Range that covers all column families beginning with a prefix within a given row
    * 
-   * @param row all keys in the range will have this row
+   * @param row
+   *          all keys in the range will have this row
    * 
-   * @param cfPrefix all keys in the range will have column families that begin with this prefix
+   * @param cfPrefix
+   *          all keys in the range will have column families that begin with this prefix
    */
   public static Range prefix(Text row, Text cfPrefix) {
     Text fp = followingPrefix(cfPrefix);
     return new Range(new Key(row, cfPrefix), true, fp == null ? new Key(row).followingKey(PartialKey.ROW) : new Key(row, fp), false);
   }
   
-  /*
+  /**
    * Returns a Range that covers all column qualifiers beginning with a prefix within a given row and column family
    * 
-   * @param row all keys in the range will have this row
+   * @param row
+   *          all keys in the range will have this row
    * 
-   * @param cf all keys in the range will have this column family
+   * @param cf
+   *          all keys in the range will have this column family
    * 
-   * @param cqPrefix all keys in the range will have column qualifiers that begin with this prefix
+   * @param cqPrefix
+   *          all keys in the range will have column qualifiers that begin with this prefix
    */
   public static Range prefix(Text row, Text cf, Text cqPrefix) {
     Text fp = followingPrefix(cqPrefix);
     return new Range(new Key(row, cf, cqPrefix), true, fp == null ? new Key(row, cf).followingKey(PartialKey.ROW_COLFAM) : new Key(row, cf, fp), false);
   }
   
-  /*
+  /**
    * Returns a Range that covers all column visibilities beginning with a prefix within a given row, column family, and column qualifier
    * 
-   * @param row all keys in the range will have this row
+   * @param row
+   *          all keys in the range will have this row
    * 
-   * @param cf all keys in the range will have this column family
+   * @param cf
+   *          all keys in the range will have this column family
    * 
-   * @param cq all keys in the range will have this column qualifier
+   * @param cq
+   *          all keys in the range will have this column qualifier
    * 
-   * @param cvPrefix all keys in the range will have column visibilities that begin with this prefix
+   * @param cvPrefix
+   *          all keys in the range will have column visibilities that begin with this prefix
    */
   public static Range prefix(Text row, Text cf, Text cq, Text cvPrefix) {
     Text fp = followingPrefix(cvPrefix);
@@ -658,7 +683,7 @@ public class Range implements WritableCo
         fp), false);
   }
   
-  /*
+  /**
    * Creates a range that covers an exact row
    * 
    * @see Range#exact(Text)
@@ -667,7 +692,7 @@ public class Range implements WritableCo
     return Range.exact(new Text(row.toString()));
   }
   
-  /*
+  /**
    * Creates a range that covers an exact row and column family
    * 
    * @see Range#exact(Text, Text)
@@ -676,7 +701,7 @@ public class Range implements WritableCo
     return Range.exact(new Text(row.toString()), new Text(cf.toString()));
   }
   
-  /*
+  /**
    * Creates a range that covers an exact row, column family, and column qualifier
    * 
    * @see Range#exact(Text, Text, Text)
@@ -685,7 +710,7 @@ public class Range implements WritableCo
     return Range.exact(new Text(row.toString()), new Text(cf.toString()), new Text(cq.toString()));
   }
   
-  /*
+  /**
    * Creates a range that covers an exact row, column family, column qualifier, and visibility
    * 
    * @see Range#exact(Text, Text, Text, Text)
@@ -694,7 +719,7 @@ public class Range implements WritableCo
     return Range.exact(new Text(row.toString()), new Text(cf.toString()), new Text(cq.toString()), new Text(cv.toString()));
   }
   
-  /*
+  /**
    * Creates a range that covers an exact row, column family, column qualifier, visibility, and timestamp
    * 
    * @see Range#exact(Text, Text, Text, Text, long)
@@ -703,7 +728,7 @@ public class Range implements WritableCo
     return Range.exact(new Text(row.toString()), new Text(cf.toString()), new Text(cq.toString()), new Text(cv.toString()), ts);
   }
   
-  /*
+  /**
    * Returns a Range that covers all rows beginning with a prefix
    * 
    * @see Range#prefix(Text)
@@ -712,7 +737,7 @@ public class Range implements WritableCo
     return Range.prefix(new Text(rowPrefix.toString()));
   }
   
-  /*
+  /**
    * Returns a Range that covers all column families beginning with a prefix within a given row
    * 
    * @see Range#prefix(Text, Text)
@@ -721,7 +746,7 @@ public class Range implements WritableCo
     return Range.prefix(new Text(row.toString()), new Text(cfPrefix.toString()));
   }
   
-  /*
+  /**
    * Returns a Range that covers all column qualifiers beginning with a prefix within a given row and column family
    * 
    * @see Range#prefix(Text, Text, Text)
@@ -730,7 +755,7 @@ public class Range implements WritableCo
     return Range.prefix(new Text(row.toString()), new Text(cf.toString()), new Text(cqPrefix.toString()));
   }
   
-  /*
+  /**
    * Returns a Range that covers all column visibilities beginning with a prefix within a given row, column family, and column qualifier
    * 
    * @see Range#prefix(Text, Text, Text, Text)

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/Filter.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/Filter.java?rev=1203222&r1=1203221&r2=1203222&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/Filter.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/Filter.java Thu Nov 17 15:29:07 2011
@@ -26,6 +26,15 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 
+/**
+ * A SortedKeyValueIterator that filters entries from its source iterator.
+ * 
+ * Subclasses must implement an accept method: public boolean accept(Key k, Value v);
+ * 
+ * Key/Value pairs for which the accept method returns true are said to match the filter. By default, this class iterates over entries that match its filter.
+ * This iterator takes an optional "negate" boolean parameter that defaults to false. If negate is set to true, this class instead omits entries that match its
+ * filter, thus iterating over entries that do not match its filter.
+ */
 public abstract class Filter extends WrappingIterator implements OptionDescriber {
   @Override
   public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
@@ -62,6 +71,9 @@ public abstract class Filter extends Wra
     findTop();
   }
   
+  /**
+   * Iterates over the source until an acceptable key/value pair is found.
+   */
   protected void findTop() {
     while (getSource().hasTop() && (negate == accept(getSource().getTopKey(), getSource().getTopValue()))) {
       try {
@@ -72,6 +84,9 @@ public abstract class Filter extends Wra
     }
   }
   
+  /**
+   * @return <tt>true</tt> if the key/value pair is accepted by the filter.
+   */
   public abstract boolean accept(Key k, Value v);
   
   @Override
@@ -96,5 +111,4 @@ public abstract class Filter extends Wra
     }
     return true;
   }
-  
 }

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=1203222&r1=1203221&r2=1203222&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 Nov 17 15:29:07 2011
@@ -27,15 +27,14 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Value;
 import org.apache.hadoop.io.WritableUtils;
 
-/*
+/**
  * A TypedValueCombiner that translates each Value to a Long before reducing, then encodes the reduced Long back to a Value.
  * 
- * Subclasses must implement a typedReduce method:
- *   public Long typedReduce(Key key, Iterator<Long> iter);
+ * Subclasses must implement a typedReduce method: public Long typedReduce(Key key, Iterator<Long> iter);
  * 
  * This typedReduce method will be passed the most recent Key and an iterator over the Values (translated to Longs) for all non-deleted versions of that Key.
  * 
- * A required option for this Combiner is "type" which indicates which type of Encoder to use to encode and decode Longs into Values.  Supported types are
+ * A required option for this Combiner is "type" which indicates which type of Encoder to use to encode and decode Longs into Values. Supported types are
  * VARNUM, LONG, and STRING which indicate the VarNumEncoder, LongEncoder, and StringEncoder respectively.
  */
 public abstract class LongCombiner extends TypedValueCombiner<Long> {
@@ -87,7 +86,7 @@ public abstract class LongCombiner exten
     return true;
   }
   
-  /*
+  /**
    * An Encoder that uses a variable-length encoding for Longs. It uses WritableUtils.writeVLong and WritableUtils.readVLong for encoding and decoding.
    */
   public static class VarNumEncoder implements Encoder<Long> {
@@ -116,7 +115,7 @@ public abstract class LongCombiner exten
     }
   }
   
-  /*
+  /**
    * An Encoder that uses an 8-byte encoding for Longs.
    */
   public static class LongEncoder implements Encoder<Long> {
@@ -147,7 +146,7 @@ public abstract class LongCombiner exten
     }
   }
   
-  /*
+  /**
    * An Encoder that uses a String representation of Longs. It uses Long.toString and Long.parseLong for encoding and decoding.
    */
   public static class StringEncoder implements Encoder<Long> {

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=1203222&r1=1203221&r2=1203222&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 Nov 17 15:29:07 2011
@@ -17,20 +17,17 @@
 package org.apache.accumulo.core.iterators;
 
 import java.util.ArrayList;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.LinkedHashMap;
 
-/*
- * The OptionDescriber interface allows you to set up iterator properties interactively in the accumulo shell.
- * If your iterator and/or filter must implement this interface for the interactive part.  The alternative would
- * be to manually set configuration options with the config -t tableName property=value.  If you go the manual
- * route, be careful to use the correct structure for the property and to set all the properties required for 
- * the iterator.
+/**
+ * The OptionDescriber interface allows you to set up iterator properties interactively in the accumulo shell. If your iterator and/or filter must implement
+ * this interface for the interactive part. The alternative would be to manually set configuration options with the config -t tableName property=value. If you
+ * go the manual route, be careful to use the correct structure for the property and to set all the properties required for the iterator.
  * 
- * OptionDescribers will need to implement two methods:
- *      describeOptions() which returns an instance of IteratorOptions
- * 	and validateOptions(Map<String,String> options) which is intended to throw an exception or return false if the options are not acceptable.
+ * OptionDescribers will need to implement two methods: describeOptions() which returns an instance of IteratorOptions and validateOptions(Map<String,String>
+ * options) which is intended to throw an exception or return false if the options are not acceptable.
  * 
  */
 public interface OptionDescriber {
@@ -40,13 +37,21 @@ public interface OptionDescriber {
     public String name;
     public String description;
     
-    /*
-     * IteratorOptions requires the following: name is the distinguishing name for the iterator or filter description is a description of the iterator or filter
-     * namedOptions is a map from specifically named options to their descriptions (null if unused) e.g., the AgeOffFilter requires a parameter called "ttl", so
-     * its namedOptions = Collections.singletonMap("ttl", "time to live (milliseconds)") unnamedOptionDescriptions is a list of descriptions of additional
-     * options that don't have fixed names (null if unused) the descriptions are intended to describe a category, and the user will provide parameter names and
-     * values in that category e.g., the FilteringIterator needs a list of Filters intended to be named by their priority numbers, so its
-     * unnamedOptionDescriptions = Collections.singletonList("<filterPriorityNumber> <ageoff|regex|filterClass>")
+    /**
+     * IteratorOptions requires the following:
+     * 
+     * @param name
+     *          is the distinguishing name for the iterator or filter
+     * @param description
+     *          is a description of the iterator or filter
+     * @param namedOptions
+     *          is a map from specifically named options to their descriptions (null if unused) e.g., the AgeOffFilter requires a parameter called "ttl", so its
+     *          namedOptions = Collections.singletonMap("ttl", "time to live (milliseconds)")
+     * @param unnamedOptionDescriptions
+     *          is a list of descriptions of additional options that don't have fixed names (null if unused). The descriptions are intended to describe a
+     *          category, and the user will provide parameter names and values in that category; e.g., the FilteringIterator needs a list of Filters intended to
+     *          be named by their priority numbers, so its unnamedOptionDescriptions =
+     *          Collections.singletonList("<filterPriorityNumber> <ageoff|regex|filterClass>")
      */
     public IteratorOptions(String name, String description, Map<String,String> namedOptions, List<String> unnamedOptionDescriptions) {
       this.name = name;

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=1203222&r1=1203221&r2=1203222&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 Nov 17 15:29:07 2011
@@ -22,32 +22,33 @@ import java.util.NoSuchElementException;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 
-/*
+/**
  * A Combiner that decodes each Value to type V before reducing, then encodes the result of typedReduce back to Value.
  * 
- * Subclasses must implement a typedReduce method:
- *   public V typedReduce(Key key, Iterator<V> iter);
+ * Subclasses must implement a typedReduce method: public V typedReduce(Key key, Iterator<V> iter);
  * 
  * This typedReduce method will be passed the most recent Key and an iterator over the Values (translated to Vs) for all non-deleted versions of that Key.
- *
+ * 
  * Subclasses may implement a switch on the "type" variable to choose an Encoder in their init method.
  */
 public abstract class TypedValueCombiner<V> extends Combiner {
   protected Encoder<V> encoder;
   
-  /*
+  /**
    * A Java Iterator that translates an Iterator<Value> to an Iterator<V> using the decode method of an Encoder.
    */
   private static class VIterator<V> implements Iterator<V> {
     private Iterator<Value> source;
     private Encoder<V> encoder;
     
-    /*
+    /**
      * Constructs an Iterator<V> from an Iterator<Value>
      * 
-     * @param iter The source iterator
+     * @param iter
+     *          The source iterator
      * 
-     * @param encoder The Encoder whose decode method is used to translate from Value to V
+     * @param encoder
+     *          The Encoder whose decode method is used to translate from Value to V
      */
     VIterator(Iterator<Value> iter, Encoder<V> encoder) {
       this.source = iter;
@@ -72,7 +73,7 @@ public abstract class TypedValueCombiner
     }
   }
   
-  /*
+  /**
    * An interface for translating from byte[] to V and back.
    */
   public static interface Encoder<V> {
@@ -86,12 +87,14 @@ public abstract class TypedValueCombiner
     return new Value(encoder.encode(typedReduce(key, new VIterator<V>(iter, encoder))));
   }
   
-  /*
+  /**
    * Reduces a list of V into a single V.
    * 
-   * @param key The most recent version of the Key being reduced.
+   * @param key
+   *          The most recent version of the Key being reduced.
    * 
-   * @param iter An iterator over the V for different versions of the key.
+   * @param iter
+   *          An iterator over the V for different versions of the key.
    * 
    * @return The combined V.
    */

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/AgeOffFilter.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/AgeOffFilter.java?rev=1203222&r1=1203221&r2=1203222&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/AgeOffFilter.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/AgeOffFilter.java Thu Nov 17 15:29:07 2011
@@ -25,6 +25,11 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 
+/**
+ * A filter that ages off key/value pairs based on the Key's timestamp. It removes an entry if its timestamp is less than currentTime - threshold.
+ * 
+ * This filter requires a "ttl" option, in milliseconds, to determine the age off threshold.
+ */
 public class AgeOffFilter extends Filter {
   private static final String TTL = "ttl";
   private long threshold;
@@ -32,12 +37,29 @@ public class AgeOffFilter extends Filter
   
   public AgeOffFilter() {}
   
+  /**
+   * Constructs a filter that omits entries read from a source iterator if the Key's timestamp is less than currentTime - threshold.
+   * 
+   * @param iterator
+   *          The source iterator.
+   * 
+   * @param threshold
+   *          Maximum age in milliseconds of data to keep.
+   * 
+   * @param threshold
+   *          Current time in milliseconds.
+   */
   public AgeOffFilter(SortedKeyValueIterator<Key,Value> iterator, long threshold, long currentTime) {
     super(iterator);
     this.threshold = threshold;
     this.currentTime = currentTime;
   }
   
+  /**
+   * Accepts entries whose timestamps are less than currentTime - threshold.
+   * 
+   * @see org.apache.accumulo.core.iterators.Filter#accept(org.apache.accumulo.core.data.Key, org.apache.accumulo.core.data.Value)
+   */
   @Override
   public boolean accept(Key k, Value v) {
     if (currentTime - k.getTimestamp() > threshold)

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/ColumnAgeOffFilter.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/ColumnAgeOffFilter.java?rev=1203222&r1=1203221&r2=1203222&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/ColumnAgeOffFilter.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/ColumnAgeOffFilter.java Thu Nov 17 15:29:07 2011
@@ -28,6 +28,10 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.iterators.conf.ColumnToClassMapping;
 import org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig;
 
+/**
+ * A filter that ages off key/value pairs based on the Key's column and timestamp. It removes an entry if its timestamp is less than currentTime - threshold.
+ * Different thresholds are set for each column.
+ */
 public class ColumnAgeOffFilter extends Filter {
   
   public ColumnAgeOffFilter() {}

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/MaxCombiner.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/MaxCombiner.java?rev=1203222&r1=1203221&r2=1203222&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/MaxCombiner.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/MaxCombiner.java Thu Nov 17 15:29:07 2011
@@ -21,6 +21,9 @@ import java.util.Iterator;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.iterators.LongCombiner;
 
+/**
+ * A Combiner that interprets Values as Longs and returns the largest Long among them.
+ */
 public class MaxCombiner extends LongCombiner {
   @Override
   public Long typedReduce(Key key, Iterator<Long> iter) {

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/MinCombiner.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/MinCombiner.java?rev=1203222&r1=1203221&r2=1203222&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/MinCombiner.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/MinCombiner.java Thu Nov 17 15:29:07 2011
@@ -21,6 +21,9 @@ import java.util.Iterator;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.iterators.LongCombiner;
 
+/**
+ * A Combiner that interprets Values as Longs and returns the smallest Long among them.
+ */
 public class MinCombiner extends LongCombiner {
   @Override
   public Long typedReduce(Key key, Iterator<Long> iter) {

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/NoVisFilter.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/NoVisFilter.java?rev=1203222&r1=1203221&r2=1203222&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/NoVisFilter.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/NoVisFilter.java Thu Nov 17 15:29:07 2011
@@ -22,6 +22,9 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.security.ColumnVisibility;
 
+/**
+ * A Filter that matches entries with a non-empty ColumnVisibility.
+ */
 public class NoVisFilter extends Filter {
   
   public NoVisFilter() {}

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/RegExFilter.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/RegExFilter.java?rev=1203222&r1=1203221&r2=1203222&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/RegExFilter.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/RegExFilter.java Thu Nov 17 15:29:07 2011
@@ -30,6 +30,9 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.util.ByteArrayBackedCharSequence;
 
+/**
+ * A Filter that matches entries based on Java regular expressions.
+ */
 public class RegExFilter extends Filter {
   
   @Override

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=1203222&r1=1203221&r2=1203222&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 Nov 17 15:29:07 2011
@@ -35,6 +35,9 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.iterators.TypedValueCombiner;
 import org.apache.hadoop.io.WritableUtils;
 
+/**
+ * A Combiner that interprets Values as arrays of Longs and returns an array of element-wise sums.
+ */
 public class SummingArrayCombiner extends TypedValueCombiner<List<Long>> {
   @Override
   public List<Long> typedReduce(Key key, Iterator<List<Long>> iter) {

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingCombiner.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingCombiner.java?rev=1203222&r1=1203221&r2=1203222&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingCombiner.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingCombiner.java Thu Nov 17 15:29:07 2011
@@ -21,6 +21,9 @@ import java.util.Iterator;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.iterators.LongCombiner;
 
+/**
+ * A Combiner that interprets Values as Longs and returns their sum.
+ */
 public class SummingCombiner extends LongCombiner {
   @Override
   public Long typedReduce(Key key, Iterator<Long> iter) {

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=1203222&r1=1203221&r2=1203222&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 Nov 17 15:29:07 2011
@@ -26,6 +26,9 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 
+/**
+ * A Filter that matches entries whose timestamps fall within a range.
+ */
 public class TimestampFilter extends Filter {
   private final SimpleDateFormat dateParser = new SimpleDateFormat("yyyyMMddHHmmssz");