hadoop-common-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From cutt...@apache.org
Subject svn commit: r533233 - in /lucene/hadoop/trunk: ./ src/contrib/abacus/src/java/org/apache/hadoop/abacus/ src/java/org/apache/hadoop/mapred/lib/aggregate/ src/test/org/apache/hadoop/mapred/ src/test/org/apache/hadoop/mapred/lib/ src/test/org/apache/hadoo...
Date Fri, 27 Apr 2007 21:36:47 GMT
Author: cutting
Date: Fri Apr 27 14:36:43 2007
New Revision: 533233

URL: http://svn.apache.org/viewvc?view=rev&rev=533233
Log:
HADOOP-1290.  Move contrib/abacus into mapred/lib/aggregate.  Contributed by Runping.

Added:
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/
      - copied from r533222, lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/LongValueMax.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/LongValueMin.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/StringValueMax.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/StringValueMin.java
    lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestAggregates.java
    lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/lib/
    lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/lib/aggregate/
    lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/lib/aggregate/AggregatorTests.java
Removed:
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/JobBase.java
Modified:
    lucene/hadoop/trunk/CHANGES.txt
    lucene/hadoop/trunk/build.xml
    lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/DoubleValueSum.java
    lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/JobBase.java
    lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/LongValueSum.java
    lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/UniqValueCount.java
    lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/UserDefinedValueAggregatorDescriptor.java
    lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregator.java
    lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorBaseDescriptor.java
    lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorCombiner.java
    lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorDescriptor.java
    lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorJob.java
    lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorJobBase.java
    lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorMapper.java
    lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorReducer.java
    lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueHistogram.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/DoubleValueSum.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/LongValueSum.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/UniqValueCount.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/UserDefinedValueAggregatorDescriptor.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregator.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorBaseDescriptor.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorCombiner.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorDescriptor.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorJob.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorJobBase.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorMapper.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorReducer.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueHistogram.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/package.html

Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?view=diff&rev=533233&r1=533232&r2=533233
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Fri Apr 27 14:36:43 2007
@@ -291,6 +291,9 @@
 86. HADOOP-1278.  Improve blacklisting of TaskTrackers by JobTracker,
     to reduce false positives.  (Arun C Murthy via cutting)
 
+87. HADOOP-1290.  Move contrib/abacus into mapred/lib/aggregate.
+    (Runping Qi via cutting)
+
 
 Release 0.12.3 - 2007-04-06
 

Modified: lucene/hadoop/trunk/build.xml
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/build.xml?view=diff&rev=533233&r1=533232&r2=533233
==============================================================================
--- lucene/hadoop/trunk/build.xml (original)
+++ lucene/hadoop/trunk/build.xml Fri Apr 27 14:36:43 2007
@@ -553,7 +553,6 @@
     	<packageset dir="${examples.dir}"/>
 
     	<packageset dir="src/contrib/streaming/src/java"/>
-    	<packageset dir="src/contrib/abacus/src/java"/>
     	<packageset dir="src/contrib/data_join/src/java"/>
 
         <link href="${javadoc.link.java}"/>
@@ -563,7 +562,6 @@
     	<group title="Examples" packages="org.apache.hadoop.examples*"/>
 
        <group title="contrib: Streaming" packages="org.apache.hadoop.streaming*"/>
-       <group title="contrib: Abacus" packages="org.apache.hadoop.abacus*"/>
        <group title="contrib: DataJoin" packages="org.apache.hadoop.contrib/join*"/>
 
     </javadoc>

Modified: lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/DoubleValueSum.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/DoubleValueSum.java?view=diff&rev=533233&r1=533232&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/DoubleValueSum.java (original)
+++ lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/DoubleValueSum.java Fri Apr 27 14:36:43 2007
@@ -22,6 +22,8 @@
 
 
 /**
+ * @deprecated
+ * 
  * This class implements a value aggregator that sums up a sequence of double
  * values.
  * 

Modified: lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/JobBase.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/JobBase.java?view=diff&rev=533233&r1=533232&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/JobBase.java (original)
+++ lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/JobBase.java Fri Apr 27 14:36:43 2007
@@ -30,6 +30,8 @@
 import org.apache.hadoop.mapred.Reducer;
 
 /**
+ * * @deprecated
+ * 
  * A common base implementing some statics collecting mechanisms that are
  * commonly used in a typical map/reduce job.
  * 

Modified: lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/LongValueSum.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/LongValueSum.java?view=diff&rev=533233&r1=533232&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/LongValueSum.java (original)
+++ lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/LongValueSum.java Fri Apr 27 14:36:43 2007
@@ -21,6 +21,8 @@
 import java.util.ArrayList;
 
 /**
+ * * @deprecated
+ * 
  * This class implements a value aggregator that sums up 
  * a sequence of long values.
  * 

Modified: lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/UniqValueCount.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/UniqValueCount.java?view=diff&rev=533233&r1=533232&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/UniqValueCount.java (original)
+++ lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/UniqValueCount.java Fri Apr 27 14:36:43 2007
@@ -24,6 +24,8 @@
 import java.util.TreeMap;
 
 /**
+ * * @deprecated
+ * 
  * This class implements a value aggregator that dedupes a sequence of objects.
  * 
  */

Modified: lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/UserDefinedValueAggregatorDescriptor.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/UserDefinedValueAggregatorDescriptor.java?view=diff&rev=533233&r1=533232&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/UserDefinedValueAggregatorDescriptor.java (original)
+++ lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/UserDefinedValueAggregatorDescriptor.java Fri Apr 27 14:36:43 2007
@@ -25,6 +25,8 @@
 import org.apache.hadoop.mapred.JobConf;
 
 /**
+ * @deprecated
+ * 
  * This class implements a wrapper for a user defined value aggregator descriptor.
  * It servs two functions: One is to create an object of ValueAggregatorDescriptor from the
  * name of a user defined class that may be dynamically loaded. The other is to

Modified: lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregator.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregator.java?view=diff&rev=533233&r1=533232&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregator.java (original)
+++ lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregator.java Fri Apr 27 14:36:43 2007
@@ -21,6 +21,8 @@
 import java.util.ArrayList;
 
 /**
+ * @deprecated
+ * 
  * This interface defines the minimal protocol for value aggregators.
  * 
  */

Modified: lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorBaseDescriptor.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorBaseDescriptor.java?view=diff&rev=533233&r1=533232&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorBaseDescriptor.java (original)
+++ lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorBaseDescriptor.java Fri Apr 27 14:36:43 2007
@@ -24,6 +24,8 @@
 import org.apache.hadoop.mapred.JobConf;
 
 /** 
+ * @deprecated
+ * 
  * This class implements the common functionalities of 
  * the subclasses of ValueAggregatorDescriptor class.
  *

Modified: lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorCombiner.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorCombiner.java?view=diff&rev=533233&r1=533232&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorCombiner.java (original)
+++ lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorCombiner.java Fri Apr 27 14:36:43 2007
@@ -29,6 +29,8 @@
 import org.apache.hadoop.mapred.Reporter;
 
 /**
+ * @deprecated
+ * 
  * This class implements the generic combiner of Abacus.
  */
 public class ValueAggregatorCombiner extends ValueAggregatorJobBase {

Modified: lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorDescriptor.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorDescriptor.java?view=diff&rev=533233&r1=533232&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorDescriptor.java (original)
+++ lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorDescriptor.java Fri Apr 27 14:36:43 2007
@@ -25,6 +25,8 @@
 import org.apache.hadoop.mapred.JobConf;
 
 /**
+ * @deprecated
+ * 
  * This interface defines the contract a value aggregator descriptor must
  * support. Such a descriptor can be configured with a JobConf object. Its main
  * function is to generate a list of aggregation-id/value pairs. An aggregation

Modified: lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorJob.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorJob.java?view=diff&rev=533233&r1=533232&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorJob.java (original)
+++ lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorJob.java Fri Apr 27 14:36:43 2007
@@ -34,6 +34,9 @@
 import org.apache.hadoop.mapred.RunningJob;
 
 /**
+ * 
+ * @deprecated
+ * 
  * This is the main class for creating a map/reduce job using Abacus framework.
  * The Abacus is a specialization of map/reduce framework, specilizing for
  * performing various simple aggregations.

Modified: lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorJobBase.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorJobBase.java?view=diff&rev=533233&r1=533232&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorJobBase.java (original)
+++ lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorJobBase.java Fri Apr 27 14:36:43 2007
@@ -25,6 +25,8 @@
 
 
 /**
+ * @deprecated
+ * 
  * This abstract class implements some common functionalities of the
  * the generic mapper, reducer and combiner classes of Abacus.
  *

Modified: lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorMapper.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorMapper.java?view=diff&rev=533233&r1=533232&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorMapper.java (original)
+++ lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorMapper.java Fri Apr 27 14:36:43 2007
@@ -28,6 +28,8 @@
 import org.apache.hadoop.mapred.Reporter;
 
 /**
+ * @deprecated
+ * 
  * This class implements the generic mapper of Abacus.
  */
 public class ValueAggregatorMapper extends ValueAggregatorJobBase {

Modified: lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorReducer.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorReducer.java?view=diff&rev=533233&r1=533232&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorReducer.java (original)
+++ lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueAggregatorReducer.java Fri Apr 27 14:36:43 2007
@@ -28,9 +28,9 @@
 import org.apache.hadoop.mapred.Reporter;
 
 /**
- * This class implements the generic reducer of Abacus.
+ * @deprecated
  * 
- * @author runping
+ * This class implements the generic reducer of Abacus.
  * 
  */
 public class ValueAggregatorReducer extends ValueAggregatorJobBase {

Modified: lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueHistogram.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueHistogram.java?view=diff&rev=533233&r1=533232&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueHistogram.java (original)
+++ lucene/hadoop/trunk/src/contrib/abacus/src/java/org/apache/hadoop/abacus/ValueHistogram.java Fri Apr 27 14:36:43 2007
@@ -27,6 +27,8 @@
 import org.apache.hadoop.io.Text;
 
 /**
+ * @deprecated
+ * 
  * This class implements a value aggregator that computes the 
  * histogram of a sequence of strings.
  * 

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/DoubleValueSum.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/DoubleValueSum.java?view=diff&rev=533233&r1=533222&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/DoubleValueSum.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/DoubleValueSum.java Fri Apr 27 14:36:43 2007
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.abacus;
+package org.apache.hadoop.mapred.lib.aggregate;
 
 import java.util.ArrayList;
 
@@ -86,9 +86,9 @@
    *         representation of the aggregated value. The return value is
    *         expected to be used by the a combiner.
    */
-  public ArrayList getCombinerOutput() {
-    ArrayList retv = new ArrayList(1);
-    retv.add(getReport());
+  public ArrayList<String> getCombinerOutput() {
+    ArrayList<String> retv = new ArrayList<String>(1);
+    retv.add("" + sum);
     return retv;
   }
 

Added: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/LongValueMax.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/LongValueMax.java?view=auto&rev=533233
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/LongValueMax.java (added)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/LongValueMax.java Fri Apr 27 14:36:43 2007
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred.lib.aggregate;
+
+import java.util.ArrayList;
+
+/**
+ * This class implements a value aggregator that maintain the maximum of 
+ * a sequence of long values.
+ * 
+ */
+public class LongValueMax implements ValueAggregator {
+
+  long maxVal = Long.MIN_VALUE;
+    
+  /**
+   *  the default constructor
+   *
+   */
+  public LongValueMax() {
+    reset();
+  }
+
+  /**
+   * add a value to the aggregator
+   * 
+   * @param val
+   *          an object whose string representation represents a long value.
+   * 
+   */
+  public void addNextValue(Object val) {
+    long newVal = Long.parseLong(val.toString());
+    if (this.maxVal < newVal) {
+      this.maxVal = newVal;
+    }
+  }
+    
+  /**
+   * add a value to the aggregator
+   * 
+   * @param newVal
+   *          a long value.
+   * 
+   */
+  public void addNextValue(long newVal) {
+    if (this.maxVal < newVal) {
+      this.maxVal = newVal;
+    };
+  }
+    
+  /**
+   * @return the aggregated value
+   */
+  public long getVal() {
+    return this.maxVal;
+  }
+    
+  /**
+   * @return the string representation of the aggregated value
+   */
+  public String getReport() {
+    return ""+maxVal;
+  }
+
+  /**
+   * reset the aggregator
+   */
+  public void reset() {
+    maxVal = Long.MIN_VALUE;
+  }
+
+  /**
+   * @return return an array of one element. The element is a string
+   *         representation of the aggregated value. The return value is
+   *         expected to be used by the a combiner.
+   */
+  public ArrayList<String> getCombinerOutput() {
+    ArrayList<String> retv = new ArrayList<String>(1);;
+    retv.add(""+maxVal);
+    return retv;
+  }
+}

Added: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/LongValueMin.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/LongValueMin.java?view=auto&rev=533233
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/LongValueMin.java (added)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/LongValueMin.java Fri Apr 27 14:36:43 2007
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred.lib.aggregate;
+
+import java.util.ArrayList;
+
+/**
+ * This class implements a value aggregator that maintain the minimum of 
+ * a sequence of long values.
+ * 
+ */
+public class LongValueMin implements ValueAggregator {
+
+  long minVal = Long.MAX_VALUE;
+    
+  /**
+   *  the default constructor
+   *
+   */
+  public LongValueMin() {
+    reset();
+  }
+
+  /**
+   * add a value to the aggregator
+   * 
+   * @param val
+   *          an object whose string representation represents a long value.
+   * 
+   */
+  public void addNextValue(Object val) {
+    long newVal = Long.parseLong(val.toString());
+    if (this.minVal > newVal) {
+      this.minVal = newVal;
+    }
+  }
+    
+  /**
+   * add a value to the aggregator
+   * 
+   * @param newVal
+   *          a long value.
+   * 
+   */
+  public void addNextValue(long newVal) {
+    if (this.minVal > newVal) {
+      this.minVal = newVal;
+    };
+  }
+    
+  /**
+   * @return the aggregated value
+   */
+  public long getVal() {
+    return this.minVal;
+  }
+    
+  /**
+   * @return the string representation of the aggregated value
+   */
+  public String getReport() {
+    return ""+minVal;
+  }
+
+  /**
+   * reset the aggregator
+   */
+  public void reset() {
+    minVal = Long.MAX_VALUE;
+  }
+
+  /**
+   * @return return an array of one element. The element is a string
+   *         representation of the aggregated value. The return value is
+   *         expected to be used by the a combiner.
+   */
+  public ArrayList<String> getCombinerOutput() {
+    ArrayList<String> retv = new ArrayList<String>(1);
+    retv.add(""+minVal);
+    return retv;
+  }
+}

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/LongValueSum.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/LongValueSum.java?view=diff&rev=533233&r1=533222&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/LongValueSum.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/LongValueSum.java Fri Apr 27 14:36:43 2007
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.abacus;
+package org.apache.hadoop.mapred.lib.aggregate;
 
 import java.util.ArrayList;
 
@@ -85,9 +85,9 @@
    *         representation of the aggregated value. The return value is
    *         expected to be used by the a combiner.
    */
-  public ArrayList getCombinerOutput() {
-    ArrayList retv = new ArrayList(1);
-    retv.add(getReport());
+  public ArrayList<String> getCombinerOutput() {
+    ArrayList<String> retv = new ArrayList<String>(1);
+    retv.add(""+sum);
     return retv;
   }
 }

Added: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/StringValueMax.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/StringValueMax.java?view=auto&rev=533233
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/StringValueMax.java (added)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/StringValueMax.java Fri Apr 27 14:36:43 2007
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred.lib.aggregate;
+
+import java.util.ArrayList;
+
+/**
+ * This class implements a value aggregator that maintain the biggest of 
+ * a sequence of strings.
+ * 
+ */
+public class StringValueMax implements ValueAggregator {
+
+  String maxVal = null;
+    
+  /**
+   *  the default constructor
+   *
+   */
+  public StringValueMax() {
+    reset();
+  }
+
+  /**
+   * add a value to the aggregator
+   * 
+   * @param val
+   *          a string.
+   * 
+   */
+  public void addNextValue(Object val) {
+    String newVal = val.toString();
+    if (this.maxVal == null || this.maxVal.compareTo(newVal) < 0) {
+      this.maxVal = newVal;
+    }
+  }
+    
+    
+  /**
+   * @return the aggregated value
+   */
+  public String getVal() {
+    return this.maxVal;
+  }
+    
+  /**
+   * @return the string representation of the aggregated value
+   */
+  public String getReport() {
+    return maxVal;
+  }
+
+  /**
+   * reset the aggregator
+   */
+  public void reset() {
+    maxVal = null;
+  }
+
+  /**
+   * @return return an array of one element. The element is a string
+   *         representation of the aggregated value. The return value is
+   *         expected to be used by the a combiner.
+   */
+  public ArrayList<String> getCombinerOutput() {
+    ArrayList<String> retv = new ArrayList<String>(1);
+    retv.add(maxVal);
+    return retv;
+  }
+}

Added: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/StringValueMin.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/StringValueMin.java?view=auto&rev=533233
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/StringValueMin.java (added)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/StringValueMin.java Fri Apr 27 14:36:43 2007
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred.lib.aggregate;
+
+import java.util.ArrayList;
+
+/**
+ * This class implements a value aggregator that maintain the smallest of 
+ * a sequence of strings.
+ * 
+ */
+public class StringValueMin implements ValueAggregator {
+
+  String minVal = null;
+    
+  /**
+   *  the default constructor
+   *
+   */
+  public StringValueMin() {
+    reset();
+  }
+
+  /**
+   * add a value to the aggregator
+   * 
+   * @param val
+   *          a string.
+   * 
+   */
+  public void addNextValue(Object val) {
+    String newVal = val.toString();
+    if (this.minVal == null || this.minVal.compareTo(newVal) > 0) {
+      this.minVal = newVal;
+    }
+  }
+    
+    
+  /**
+   * @return the aggregated value
+   */
+  public String getVal() {
+    return this.minVal;
+  }
+    
+  /**
+   * @return the string representation of the aggregated value
+   */
+  public String getReport() {
+    return minVal;
+  }
+
+  /**
+   * reset the aggregator
+   */
+  public void reset() {
+    minVal = null;
+  }
+
+  /**
+   * @return return an array of one element. The element is a string
+   *         representation of the aggregated value. The return value is
+   *         expected to be used by the a combiner.
+   */
+  public ArrayList<String> getCombinerOutput() {
+    ArrayList<String> retv = new ArrayList<String>(1);
+    retv.add(minVal);
+    return retv;
+  }
+}

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/UniqValueCount.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/UniqValueCount.java?view=diff&rev=533233&r1=533222&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/UniqValueCount.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/UniqValueCount.java Fri Apr 27 14:36:43 2007
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.abacus;
+package org.apache.hadoop.mapred.lib.aggregate;
 
 import java.util.ArrayList;
 import java.util.Iterator;

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/UserDefinedValueAggregatorDescriptor.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/UserDefinedValueAggregatorDescriptor.java?view=diff&rev=533233&r1=533222&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/UserDefinedValueAggregatorDescriptor.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/UserDefinedValueAggregatorDescriptor.java Fri Apr 27 14:36:43 2007
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.abacus;
+package org.apache.hadoop.mapred.lib.aggregate;
 
 import java.lang.reflect.Constructor;
 import java.util.ArrayList;

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregator.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregator.java?view=diff&rev=533233&r1=533222&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregator.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregator.java Fri Apr 27 14:36:43 2007
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.abacus;
+package org.apache.hadoop.mapred.lib.aggregate;
 
 import java.util.ArrayList;
 

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorBaseDescriptor.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorBaseDescriptor.java?view=diff&rev=533233&r1=533222&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorBaseDescriptor.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorBaseDescriptor.java Fri Apr 27 14:36:43 2007
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.abacus;
+package org.apache.hadoop.mapred.lib.aggregate;
 
 import java.util.ArrayList;
 import java.util.Map.Entry;
@@ -37,6 +37,15 @@
   static public final String DOUBLE_VALUE_SUM = "DoubleValueSum";
 
   static public final String VALUE_HISTOGRAM = "ValueHistogram";
+  
+  static public final String LONG_VALUE_MAX = "LongValueMax";
+  
+  static public final String LONG_VALUE_MIN = "LongValueMin";
+  
+  static public final String STRING_VALUE_MAX = "StringValueMax";
+  
+  static public final String STRING_VALUE_MIN = "StringValueMin";
+  
 
   public String inputFile = null;
 
@@ -84,12 +93,20 @@
    */
   static public ValueAggregator generateValueAggregator(String type) {
     ValueAggregator retv = null;
-    if (type.compareToIgnoreCase(UNIQ_VALUE_COUNT) == 0) {
-      retv = new UniqValueCount();
-    } else if (type.compareToIgnoreCase(LONG_VALUE_SUM) == 0) {
+    if (type.compareToIgnoreCase(LONG_VALUE_SUM) == 0) {
       retv = new LongValueSum();
+    } if (type.compareToIgnoreCase(LONG_VALUE_MAX) == 0) {
+      retv = new LongValueMax();
+    } else if (type.compareToIgnoreCase(LONG_VALUE_MIN) == 0) {
+      retv = new LongValueMin();
+    } else if (type.compareToIgnoreCase(STRING_VALUE_MAX) == 0) {
+      retv = new StringValueMax();
+    } else if (type.compareToIgnoreCase(STRING_VALUE_MIN) == 0) {
+      retv = new StringValueMin();
     } else if (type.compareToIgnoreCase(DOUBLE_VALUE_SUM) == 0) {
       retv = new DoubleValueSum();
+    } else if (type.compareToIgnoreCase(UNIQ_VALUE_COUNT) == 0) {
+      retv = new UniqValueCount();
     } else if (type.compareToIgnoreCase(VALUE_HISTOGRAM) == 0) {
       retv = new ValueHistogram();
     }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorCombiner.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorCombiner.java?view=diff&rev=533233&r1=533222&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorCombiner.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorCombiner.java Fri Apr 27 14:36:43 2007
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.abacus;
+package org.apache.hadoop.mapred.lib.aggregate;
 
 import java.io.IOException;
 import java.util.Iterator;
@@ -53,9 +53,6 @@
     String type = keyStr.substring(0, pos);
     ValueAggregator aggregator = ValueAggregatorBaseDescriptor
       .generateValueAggregator(type);
-    if (aggregator == null) {
-      LOG.info(key.toString());
-    }
     while (values.hasNext()) {
       aggregator.addNextValue(values.next());
     }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorDescriptor.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorDescriptor.java?view=diff&rev=533233&r1=533222&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorDescriptor.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorDescriptor.java Fri Apr 27 14:36:43 2007
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.abacus;
+package org.apache.hadoop.mapred.lib.aggregate;
 
 import java.util.ArrayList;
 import java.util.Map.Entry;

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorJob.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorJob.java?view=diff&rev=533233&r1=533222&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorJob.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorJob.java Fri Apr 27 14:36:43 2007
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.abacus;
+package org.apache.hadoop.mapred.lib.aggregate;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -153,9 +153,6 @@
     theJob.setCombinerClass(ValueAggregatorCombiner.class);
     theJob.setNumMapTasks(1);
     theJob.setNumReduceTasks(numOfReducers);
-    theJob.set("mapred.sds.data.serialization.format", "csv");
-    theJob.set("mapred.child.java.opts", "-Xmx1024m");
-    // aggregator.setKeepFailedTaskFiles(true);
     return theJob;
   }
 

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorJobBase.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorJobBase.java?view=diff&rev=533233&r1=533222&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorJobBase.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorJobBase.java Fri Apr 27 14:36:43 2007
@@ -16,37 +16,31 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.abacus;
+package org.apache.hadoop.mapred.lib.aggregate;
 
 import java.io.IOException;
 import java.util.ArrayList;
 
 import org.apache.hadoop.mapred.JobConf;
-
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.Reducer;
 
 /**
  * This abstract class implements some common functionalities of the
  * the generic mapper, reducer and combiner classes of Abacus.
  *
  */
-public abstract class ValueAggregatorJobBase extends JobBase {
- 
-  protected ArrayList aggregatorDescriptorList = null;
-        
+public abstract class ValueAggregatorJobBase implements Mapper, Reducer {
+
+  protected ArrayList<ValueAggregatorDescriptor> aggregatorDescriptorList = null;
+
   public void configure(JobConf job) {
-    super.configure(job);
-        
-    setLongValue("totalCount", 0);
-    setLongValue("errorCount", 0);
-    setLongValue("collectedCount", 0);
-    setLongValue("groupCount", 0);
-        
     this.initializeMySpec(job);
     this.logSpec();
   }
 
   private static ValueAggregatorDescriptor getValueAggregatorDescriptor(
-                                                                        String spec, JobConf job) {
+      String spec, JobConf job) {
     if (spec == null)
       return null;
     String[] segments = spec.split(",", -1);
@@ -54,14 +48,14 @@
     if (type.compareToIgnoreCase("UserDefined") == 0) {
       String className = segments[1];
       return new UserDefinedValueAggregatorDescriptor(className, job);
-    } 
+    }
     return null;
   }
 
-  private static ArrayList getAggregatorDescriptors(JobConf job) {
+  private static ArrayList<ValueAggregatorDescriptor> getAggregatorDescriptors(JobConf job) {
     String advn = "aggregator.descriptor";
     int num = job.getInt(advn + ".num", 0);
-    ArrayList retv = new ArrayList(num);
+    ArrayList<ValueAggregatorDescriptor> retv = new ArrayList<ValueAggregatorDescriptor>(num);
     for (int i = 0; i < num; i++) {
       String spec = job.get(advn + "." + i);
       ValueAggregatorDescriptor ad = getValueAggregatorDescriptor(spec, job);
@@ -71,15 +65,16 @@
     }
     return retv;
   }
-    
+
   private void initializeMySpec(JobConf job) {
     this.aggregatorDescriptorList = getAggregatorDescriptors(job);
     if (this.aggregatorDescriptorList.size() == 0) {
-      this.aggregatorDescriptorList.add(new UserDefinedValueAggregatorDescriptor(
-                                                                                 ValueAggregatorBaseDescriptor.class.getCanonicalName(), job));
+      this.aggregatorDescriptorList
+          .add(new UserDefinedValueAggregatorDescriptor(
+              ValueAggregatorBaseDescriptor.class.getCanonicalName(), job));
     }
   }
-    
+
   protected void logSpec() {
     StringBuffer sb = new StringBuffer();
     sb.append("\n");
@@ -90,11 +85,9 @@
       for (int i = 0; i < aggregatorDescriptorList.size(); i++) {
         sb.append(" ").append(aggregatorDescriptorList.get(i).toString());
       }
-    }      
-    LOG.info(sb.toString());
+    }
   }
 
   public void close() throws IOException {
-    report();
   }
 }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorMapper.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorMapper.java?view=diff&rev=533233&r1=533222&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorMapper.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorMapper.java Fri Apr 27 14:36:43 2007
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.abacus;
+package org.apache.hadoop.mapred.lib.aggregate;
 
 import java.io.IOException;
 import java.util.Iterator;
@@ -39,7 +39,6 @@
   public void map(WritableComparable key, Writable value,
                   OutputCollector output, Reporter reporter) throws IOException {
 
-    addLongValue("groupCount", 1);
     Iterator iter = this.aggregatorDescriptorList.iterator();
     while (iter.hasNext()) {
       ValueAggregatorDescriptor ad = (ValueAggregatorDescriptor) iter.next();
@@ -48,12 +47,7 @@
         Entry en = ens.next();
         output.collect((WritableComparable) en.getKey(), (Writable) en
                        .getValue());
-        addLongValue("collectedCount", 1);
       }
-    }
-
-    if (getLongValue("groupCount").longValue() % 10000 == 0) {
-      report();
     }
   }
 

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorReducer.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorReducer.java?view=diff&rev=533233&r1=533222&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorReducer.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorReducer.java Fri Apr 27 14:36:43 2007
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.abacus;
+package org.apache.hadoop.mapred.lib.aggregate;
 
 import java.io.IOException;
 import java.util.Iterator;
@@ -30,7 +30,6 @@
 /**
  * This class implements the generic reducer of Abacus.
  * 
- * @author runping
  * 
  */
 public class ValueAggregatorReducer extends ValueAggregatorJobBase {
@@ -46,7 +45,6 @@
    */
   public void reduce(WritableComparable key, Iterator values,
                      OutputCollector output, Reporter reporter) throws IOException {
-    addLongValue("groupCount", 1);
     String keyStr = key.toString();
     int pos = keyStr.indexOf(ValueAggregatorDescriptor.TYPE_SEPARATOR);
     String type = keyStr.substring(0, pos);
@@ -56,17 +54,12 @@
     ValueAggregator aggregator = ValueAggregatorBaseDescriptor
       .generateValueAggregator(type);
     while (values.hasNext()) {
-      addLongValue("totalCount", 1);
       aggregator.addNextValue(values.next());
     }
 
     String val = aggregator.getReport();
     key = new Text(keyStr);
     output.collect(key, new Text(val));
-    addLongValue("collectedCount", 1);
-    if (getLongValue("collectedCount").longValue() % 10000 == 0) {
-      report();
-    }
   }
 
   /**

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueHistogram.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueHistogram.java?view=diff&rev=533233&r1=533222&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueHistogram.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/ValueHistogram.java Fri Apr 27 14:36:43 2007
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.abacus;
+package org.apache.hadoop.mapred.lib.aggregate;
 
 import java.util.ArrayList;
 import java.util.Iterator;
@@ -48,9 +48,13 @@
   public void addNextValue(Object val) {
     String valCountStr = val.toString();
     int pos = valCountStr.lastIndexOf("\t");
-    String valStr = valCountStr.substring(0, pos);
-    String countStr = valCountStr.substring(pos + 1);
-
+    String valStr = valCountStr;
+    String countStr = "1";
+    if (pos >= 0) {
+      valCountStr.substring(0, pos);
+      countStr = valCountStr.substring(pos + 1);
+    }
+    
     Long count = (Long) this.items.get(valStr);
     long inc = Long.parseLong(countStr);
 
@@ -85,7 +89,6 @@
     }
     Arrays.sort(counts);
     sb.append(counts.length);
-    StringBuffer sbVal = new StringBuffer();
     i = 0;
     long acc = 0;
     while (i < counts.length) {
@@ -154,7 +157,7 @@
       Entry en = (Entry) iter.next();
       Object val = en.getKey();
       Long count = (Long) en.getValue();
-      retv.add(new Text(val.toString() + "\t" + count.longValue()));
+      retv.add(val.toString() + "\t" + count.longValue());
     }
     return retv;
   }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/package.html
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/package.html?view=diff&rev=533233&r1=533222&r2=533233
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/package.html (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate/package.html Fri Apr 27 14:36:43 2007
@@ -1,20 +1,19 @@
 <html>
 <body>
 
-Hadoop based Abacus is a specialization of map/reduce framework, specilizing for
-performing various counting and aggregations. It offers similar functionalities to Google's SawZall.
+Classes for performing various counting and aggregations.
 <p />
-<h2><a name="Abacus_framework"></a> Abacus framework </h2>
+<h2><a name="Aggregate"></a>Aggregate framework </h2>
 <p />
 Generally speaking, in order to implement an application using Map/Reduce
 model, the developer needs to implement Map and Reduce functions (and possibly
 Combine function). However, for a lot of applications related to counting and
-statistics computing, these functions have very similar characteristics. Abacus abstracts out
-the general patterns and provides a package implementing 
+statistics computing, these functions have very similar
+characteristics. This provides a package implementing 
 those patterns. In particular, the package provides a generic mapper class,
 a reducer class and a combiner class, and a set of built-in value aggregators.
 It also provides a generic utility class, ValueAggregatorJob, that offers a static function that 
-creates Abacus map/reduce jobs:
+creates map/reduce jobs:
 <blockquote>
 <pre>
 public static JobConf createValueAggregatorJob(String args&#91;]) throws IOException;
@@ -65,7 +64,7 @@
 &#60;/property&#62;
 &#60;property&#62;
    &#60;name&#62;aggregator.descriptor.0&#60;/name&#62;
-   &#60;value&#62;UserDefined,org.apache.hadoop.abacus.examples.WordCountAggregatorDescriptor&#60;/value&#62;
+   &#60;value&#62;UserDefined,org.apache.hadoop.mapred.lib.aggregate.examples.WordCountAggregatorDescriptor&#60;/value&#62;
 &#60;/property&#62; 
 </pre>
 </blockquote>
@@ -123,21 +122,21 @@
 </pre>
 </blockquote>
 In order to be able to use combiner, all the aggregation type be aggregators must be associative and communitive.
-The following are the types Abacus supports now: <ul>
+The following are the types supported: <ul>
 <li> LongValueSum: sum over long values 
 </li> <li> DoubleValueSum: sum over float/double values 
 </li> <li> uniqValueCount: count the number of distinct values 
 </li> <li> ValueHistogram: compute the histogram of values compute the minimum, maximum, media,average, standard deviation of numeric values
 </li></ul> 
 <p />
-<h2><a name="Create_and_run_an_Abacus_based_a"></a> Create and run an Abacus based application </h2>
+<h2><a name="Create_and_run"></a> Create and run an application </h2>
 <p />
-To create an Abacus based application, the user needs to do the following things:
+To create an application, the user needs to do the following things:
 <p />
 1. Implement a user plugin:
 <blockquote>
 <pre>
-import org.apache.hadoop.abacus.ValueAggregatorBaseDescriptor;
+import org.apache.hadoop.mapred.lib.aggregate.ValueAggregatorBaseDescriptor;
 import org.apache.hadoop.mapred.JobConf;
 
 public class WordCountAggregatorDescriptor extends ValueAggregatorBaseDescriptor {
@@ -150,8 +149,6 @@
 }
 </pre>
 </blockquote>
-Examples can be found in src/contrib/abacus/examples directory.
-<p />
 
 2. Create an xml file specifying the user plugin.
 <p />
@@ -161,11 +158,11 @@
 Finally, run the job:
 <blockquote>
 <pre>
-        hadoop jar wc.jar org.apache.hadoop.abacus.examples.ValueAggregatorJob indirs outdir numofreducers textinputformat|sequencefileinputformat spec_file
+        hadoop jar wc.jar org.apache.hadoop.mapred.lib.aggregate..ValueAggregatorJob indirs outdir numofreducers textinputformat|sequencefileinputformat spec_file
 </pre>
 </blockquote>
 <p />
 
 
 </body>
-</html>
\ No newline at end of file
+</html>

Added: lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestAggregates.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestAggregates.java?view=auto&rev=533233
==============================================================================
--- lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestAggregates.java (added)
+++ lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestAggregates.java Fri Apr 27 14:36:43 2007
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapred;
+
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.mapred.lib.*;
+import org.apache.hadoop.mapred.lib.aggregate.*;
+import junit.framework.TestCase;
+import java.io.*;
+import java.util.*;
+import java.text.NumberFormat;
+
+public class TestAggregates extends TestCase {
+
+  private static NumberFormat idFormat = NumberFormat.getInstance();
+    static {
+      idFormat.setMinimumIntegerDigits(4);
+      idFormat.setGroupingUsed(false);
+  }
+
+
+  public void testAggregates() throws Exception {
+    launch();
+  }
+
+  public static void launch() throws Exception {
+    JobConf conf = new JobConf(TestAggregates.class);
+    FileSystem fs = FileSystem.get(conf);
+    int numOfInputLines = 20;
+
+    Path OUTPUT_DIR = new Path("build/test/output_for_aggregates_test");
+    Path INPUT_DIR = new Path("build/test/input_for_aggregates_test");
+    String inputFile = "input.txt";
+    fs.delete(INPUT_DIR);
+    fs.mkdirs(INPUT_DIR);
+    fs.delete(OUTPUT_DIR);
+
+    StringBuffer inputData = new StringBuffer();
+    StringBuffer expectedOutput = new StringBuffer();
+    expectedOutput.append("max\t19\n");
+    expectedOutput.append("min\t1\n"); 
+
+    FSDataOutputStream fileOut = fs.create(new Path(INPUT_DIR, inputFile));
+    for (int i = 1; i < numOfInputLines; i++) {
+      expectedOutput.append("count_").append(idFormat.format(i));
+      expectedOutput.append("\t").append(i).append("\n");
+
+      inputData.append(idFormat.format(i));
+      for (int j = 1; j < i; j++) {
+        inputData.append(" ").append(idFormat.format(i));
+      }
+      inputData.append("\n");
+    }
+    expectedOutput.append("value_as_string_max\t9\n");
+    expectedOutput.append("value_as_string_min\t1\n");
+    expectedOutput.append("uniq_count\t19\n");
+
+
+    fileOut.write(inputData.toString().getBytes("utf-8"));
+    fileOut.close();
+
+    System.out.println("inputData:");
+    System.out.println(inputData.toString());
+    JobConf job = new JobConf(conf, TestAggregates.class);
+    job.setInputPath(INPUT_DIR);
+    job.setInputFormat(TextInputFormat.class);
+
+    job.setOutputPath(OUTPUT_DIR);
+    job.setOutputFormat(TextOutputFormat.class);
+    job.setMapOutputKeyClass(Text.class);
+    job.setMapOutputValueClass(Text.class);
+    job.setOutputKeyClass(Text.class);
+    job.setOutputValueClass(Text.class);
+    job.setNumReduceTasks(1);
+
+    job.setMapperClass(ValueAggregatorMapper.class);
+    job.setReducerClass(ValueAggregatorReducer.class);
+    job.setCombinerClass(ValueAggregatorCombiner.class);
+
+    job.set("aggregator.descriptor.num", 1);
+    job.set("aggregator.descriptor.0", "UserDefined,org.apache.hadoop.mapred.lib.aggregate.AggregatorTests");
+
+    JobClient.runJob(job);
+
+    //
+    // Finally, we compare the reconstructed answer key with the
+    // original one.  Remember, we need to ignore zero-count items
+    // in the original key.
+    //
+    boolean success = true;
+    Path outPath = new Path(OUTPUT_DIR, "part-00000");
+    String outdata = TestMiniMRWithDFS.readOutput(outPath,job);
+    outdata = outdata.substring(0, expectedOutput.toString().length());
+
+    assertEquals(expectedOutput.toString(),outdata);
+    //fs.delete(OUTPUT_DIR);
+    fs.delete(INPUT_DIR);
+  }
+
+  /**
+   * Launches all the tasks in order.
+   */
+  public static void main(String[] argv) throws Exception {
+    launch();
+  }
+}

Added: lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/lib/aggregate/AggregatorTests.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/lib/aggregate/AggregatorTests.java?view=auto&rev=533233
==============================================================================
--- lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/lib/aggregate/AggregatorTests.java (added)
+++ lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/lib/aggregate/AggregatorTests.java Fri Apr 27 14:36:43 2007
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred.lib.aggregate;
+
+import org.apache.hadoop.io.Text;
+import java.util.ArrayList;
+import java.util.Map.Entry;
+
+public class AggregatorTests extends ValueAggregatorBaseDescriptor {
+  
+  public ArrayList<Entry> generateKeyValPairs(Object key, Object val) {
+    ArrayList<Entry> retv = new ArrayList<Entry>();
+    String [] words = val.toString().split(" ");
+    
+    String countType;
+    String id;
+    Entry e;
+    
+    for (String word: words) {
+      long numVal = Long.parseLong(word);
+      countType = LONG_VALUE_SUM;
+      id = "count_" + word;
+      e = generateEntry(countType, id, ONE);
+      if (e != null) {
+        retv.add(e);
+      }
+      countType = LONG_VALUE_MAX;
+      id = "max";
+      e = generateEntry(countType, id, new Text(word));
+      if (e != null) {
+        retv.add(e);
+      }
+      
+      countType = LONG_VALUE_MIN;
+      id = "min";
+      e = generateEntry(countType, id, new Text(word));
+      if (e != null) {
+        retv.add(e);
+      }
+      
+      countType = STRING_VALUE_MAX;
+      id = "value_as_string_max";
+      e = generateEntry(countType, id, new Text(""+numVal));
+      if (e != null) {
+        retv.add(e);
+      }
+      
+      countType = STRING_VALUE_MIN;
+      id = "value_as_string_min";
+      e = generateEntry(countType, id, new Text(""+numVal));
+      if (e != null) {
+        retv.add(e);
+      }
+      
+      countType = UNIQ_VALUE_COUNT;
+      id = "uniq_count";
+      e = generateEntry(countType, id, new Text(word));
+      if (e != null) {
+        retv.add(e);
+      }
+      
+      countType = VALUE_HISTOGRAM;
+      id = "histogram";
+      e = generateEntry(countType, id, new Text(word));
+      if (e != null) {
+        retv.add(e);
+      }
+    }
+    return retv;
+  }
+
+}



Mime
View raw message