hadoop-common-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From tomwh...@apache.org
Subject svn commit: r555750 - in /lucene/hadoop/trunk: ./ src/examples/org/apache/hadoop/examples/ src/java/org/apache/hadoop/mapred/lib/aggregate/ src/test/org/apache/hadoop/mapred/ src/test/org/apache/hadoop/mapred/lib/aggregate/
Date Thu, 12 Jul 2007 20:35:49 GMT
Author: tomwhite
Date: Thu Jul 12 13:35:47 2007
New Revision: 555750

URL: http://svn.apache.org/viewvc?view=rev&rev=555750
Log:
HADOOP-1547.  Provide examples for aggregate library.  Contributed by Runping Qi.

Added:
    lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/AggregateWordCount.java
    lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/lib/aggregate/TestAggregates.java
      - copied, changed from r555744, lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestAggregates.java
Removed:
    lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestAggregates.java
Modified:
    lucene/hadoop/trunk/CHANGES.txt
    lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/ExampleDriver.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/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

Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?view=diff&rev=555750&r1=555749&r2=555750
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Thu Jul 12 13:35:47 2007
@@ -340,6 +340,9 @@
 105. HADOOP-1598.  Fix license headers: adding missing; updating old.
      (Enis Soztutar via cutting)
 
+106. HADOOP-1547.  Provide examples for aggregate library.
+     (Runping Qi via tomwhite)
+
 
 Release 0.13.0 - 2007-06-08
 

Added: lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/AggregateWordCount.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/AggregateWordCount.java?view=auto&rev=555750
==============================================================================
--- lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/AggregateWordCount.java (added)
+++ lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/AggregateWordCount.java Thu
Jul 12 13:35:47 2007
@@ -0,0 +1,80 @@
+/**
+ * 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.examples;
+
+import java.util.ArrayList;
+import java.util.Map.Entry;
+import java.util.StringTokenizer;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.lib.aggregate.*;
+
+/**
+ * This is an example Aggregated Hadoop Map/Reduce application. It reads the
+ * text input files, breaks each line into words and counts them. The output is
+ * a locally sorted list of words and the count of how often they occurred.
+ * 
+ * To run: bin/hadoop jar hadoop-*-examples.jar aggregatewordcount <i>in-dir</i>
+ * <i>out-dir</i> <i>numOfReducers</i> textinputformat
+ * 
+ */
+public class AggregateWordCount {
+
+  public static class WordCountPlugInClass extends
+      ValueAggregatorBaseDescriptor {
+    public ArrayList<Entry> generateKeyValPairs(Object key, Object val) {
+      String countType = LONG_VALUE_SUM;
+      ArrayList<Entry> retv = new ArrayList<Entry>();
+      String line = val.toString();
+      StringTokenizer itr = new StringTokenizer(line);
+      while (itr.hasMoreTokens()) {
+        Entry e = generateEntry(countType, itr.nextToken(), ONE);
+        if (e != null) {
+          retv.add(e);
+        }
+      }
+      return retv;
+    }
+  }
+
+  /**
+   * The main driver for word count map/reduce program. Invoke this method to
+   * submit the map/reduce job.
+   * 
+   * @throws IOException
+   *           When there is communication problems with the job tracker.
+   */
+  public static void main(String[] args) throws IOException {
+    JobConf conf = ValueAggregatorJob.createValueAggregatorJob(args);
+    //specify the number of aggregators to be used
+    conf.setInt("aggregator.descriptor.num", 1);
+    //specify the aggregator descriptor
+    conf
+        .set(
+            "aggregator.descriptor.num.0",
+            "UserDefined,org.apache.hadoop.examples.AggregateWordCount.WordCountPlugInClass");
+    JobClient.runJob(conf);
+  }
+
+}

Modified: lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/ExampleDriver.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/ExampleDriver.java?view=diff&rev=555750&r1=555749&r2=555750
==============================================================================
--- lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/ExampleDriver.java (original)
+++ lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/ExampleDriver.java Thu Jul
12 13:35:47 2007
@@ -31,6 +31,8 @@
     try {
       pgd.addClass("wordcount", WordCount.class, 
                    "A map/reduce program that counts the words in the input files.");
+      pgd.addClass("aggregatewordcount", AggregateWordCount.class, 
+                   "An Aggregate based map/reduce program that counts the words in the input
files.");
       pgd.addClass("grep", Grep.class, 
                    "A map/reduce program that counts the matches of a regex in the input.");
       pgd.addClass("randomwriter", RandomWriter.class, 

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=555750&r1=555749&r2=555750
==============================================================================
--- 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
Thu Jul 12 13:35:47 2007
@@ -85,7 +85,7 @@
    *          input value
    * @return a list of aggregation id/value pairs. An aggregation id encodes an
    *         aggregation type which is used to guide the way to aggregate the
-   *         value in the reduce/combiner phrase of an Abacus based job.
+   *         value in the reduce/combiner phrase of an Aggregate based job.
    */
   public ArrayList<Entry> generateKeyValPairs(Object key, Object val) {
     ArrayList<Entry> retv = new ArrayList<Entry>();

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=555750&r1=555749&r2=555750
==============================================================================
--- 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
Thu Jul 12 13:35:47 2007
@@ -127,7 +127,7 @@
    *          input value
    * @return a list of aggregation id/value pairs. An aggregation id encodes an
    *         aggregation type which is used to guide the way to aggregate the
-   *         value in the reduce/combiner phrase of an Abacus based job.
+   *         value in the reduce/combiner phrase of an Aggregate based job.
    */
   public ArrayList<Entry> generateKeyValPairs(Object key, Object val) {
     ArrayList<Entry> retv = new ArrayList<Entry>();

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=555750&r1=555749&r2=555750
==============================================================================
--- 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
Thu Jul 12 13:35:47 2007
@@ -29,7 +29,7 @@
 import org.apache.hadoop.mapred.Reporter;
 
 /**
- * This class implements the generic combiner of Abacus.
+ * This class implements the generic combiner of Aggregate.
  */
 public class ValueAggregatorCombiner extends ValueAggregatorJobBase {
 

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=555750&r1=555749&r2=555750
==============================================================================
--- 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
Thu Jul 12 13:35:47 2007
@@ -29,8 +29,8 @@
  * 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
  * id encodes an aggregation type which is used to guide the way to aggregate
- * the value in the reduce/combiner phrase of an Abacus based job.The mapper in
- * an Abacus based map/reduce job may create one or more of
+ * the value in the reduce/combiner phrase of an Aggregate based job.The mapper in
+ * an Aggregate based map/reduce job may create one or more of
  * ValueAggregatorDescriptor objects at configuration time. For each input
  * key/value pair, the mapper will use those objects to create aggregation
  * id/value pairs.
@@ -44,7 +44,7 @@
 
   /**
    * Generate a list of aggregation-id/value pairs for the given key/value pair.
-   * This function is usually called by the mapper of an Abacus based job.
+   * This function is usually called by the mapper of an Aggregate based job.
    * 
    * @param key
    *          input key
@@ -52,7 +52,7 @@
    *          input value
    * @return a list of aggregation id/value pairs. An aggregation id encodes an
    *         aggregation type which is used to guide the way to aggregate the
-   *         value in the reduce/combiner phrase of an Abacus based job.
+   *         value in the reduce/combiner phrase of an Aggregate based job.
    */
   public ArrayList<Entry> generateKeyValPairs(Object key, Object val);
 

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=555750&r1=555749&r2=555750
==============================================================================
--- 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
Thu Jul 12 13:35:47 2007
@@ -21,7 +21,6 @@
 import java.io.IOException;
 import java.util.ArrayList;
 
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.InputFormat;
@@ -35,36 +34,34 @@
 import org.apache.hadoop.mapred.RunningJob;
 
 /**
- * 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.
+ * This is the main class for creating a map/reduce job using Aggregate
+ * framework. The Aggregate is a specialization of map/reduce framework,
+ * specilizing for performing various simple aggregations.
  * 
  * Generally speaking, in order to implement an application using Map/Reduce
  * model, the developer is to implement Map and Reduce functions (and possibly
  * combine function). However, a lot of applications related to counting and
- * statistics computing have very similar characteristics. Abacus abstracts out
- * the general patterns of these functions and implementing those patterns. In
- * particular, the package provides generic mapper/redducer/combiner classes,
+ * statistics computing have very similar characteristics. Aggregate abstracts
+ * out the general patterns of these functions and implementing those patterns.
+ * In particular, the package provides generic mapper/redducer/combiner classes,
  * and a set of built-in value aggregators, and a generic utility class that
  * helps user create map/reduce jobs using the generic class. The built-in
  * aggregators include:
  * 
- *      sum over numeric values 
- *      count the number of distinct values 
- *      compute the histogram of values 
- *      compute the minimum, maximum, media,average, standard deviation of numeric values
+ * sum over numeric values count the number of distinct values compute the
+ * histogram of values compute the minimum, maximum, media,average, standard
+ * deviation of numeric values
  * 
- * The developer using Abacus will need only to provide a plugin class
+ * The developer using Aggregate will need only to provide a plugin class
  * conforming to the following interface:
  * 
- *      public interface ValueAggregatorDescriptor { 
- *          public ArrayList<Entry> generateKeyValPairs(Object key, Object value);

- *          public void configure(JobConfjob); 
- *     } 
- * 
- * The package also provides a base class,
- * ValueAggregatorBaseDescriptor, implementing the above interface. The user can
- * extend the base class and implement generateKeyValPairs accordingly.
+ * public interface ValueAggregatorDescriptor { public ArrayList<Entry>
+ * generateKeyValPairs(Object key, Object value); public void
+ * configure(JobConfjob); }
+ * 
+ * The package also provides a base class, ValueAggregatorBaseDescriptor,
+ * implementing the above interface. The user can extend the base class and
+ * implement generateKeyValPairs accordingly.
  * 
  * The primary work of generateKeyValPairs is to emit one or more key/value
  * pairs based on the input key/value pair. The key in an output key/value pair
@@ -72,7 +69,7 @@
  * value will be aggregated onto the aggregation id according the aggregation
  * type.
  * 
- * This class offers a function to generate a map/reduce job using Abacus
+ * This class offers a function to generate a map/reduce job using Aggregate
  * framework. The function takes the following parameters: input directory spec
  * input format (text or sequence file) output directory a file specifying the
  * user plugin class
@@ -91,7 +88,7 @@
   }
 
   /**
-   * Create an Abacus based map/reduce job.
+   * Create an Aggregate based map/reduce job.
    * 
    * @param args the arguments used for job creation
    * @return a JobConf object ready for submission.
@@ -102,7 +99,8 @@
     throws IOException {
 
     if (args.length < 2) {
-      System.out.println("usage: inputDirs outDir [numOfReducer [textinputformat|seq [specfile
[jobName]]]]");
+      System.out.println("usage: inputDirs outDir "
+          + "[numOfReducer [textinputformat|seq [specfile [jobName]]]]");
       System.exit(1);
     }
     String inputDir = args[0];
@@ -113,9 +111,12 @@
     }
 
     Class<? extends InputFormat> theInputFormat =
-      SequenceFileInputFormat.class;
-    if (args.length > 3 && args[3].compareToIgnoreCase("textinputformat") == 0)
{
+      TextInputFormat.class;
+    if (args.length > 3 && 
+        args[3].compareToIgnoreCase("textinputformat") == 0) {
       theInputFormat = TextInputFormat.class;
+    } else {
+      theInputFormat = SequenceFileInputFormat.class;
     }
 
     Path specFile = null;
@@ -130,11 +131,16 @@
       jobName = args[5];
     }
     
-    JobConf theJob = new JobConf(ValueAggregatorJob.class);
+    JobConf theJob = new JobConf();
     if (specFile != null) {
       theJob.addDefaultResource(specFile);
     }
-    FileSystem fs = FileSystem.get(theJob);
+    String userJarFile = theJob.get("user.jar.file");
+    if (userJarFile == null) {
+      theJob.setJarByClass(ValueAggregator.class);
+    } else {
+      theJob.setJar(userJarFile);
+    }
     theJob.setJobName("ValueAggregatorJob: " + jobName);
 
     String[] inputDirsSpecs = inputDir.split(",");
@@ -159,46 +165,13 @@
   }
 
   /**
-   * Submit/run a map/reduce job.
-   * 
-   * @param job
-   * @return true for success
-   * @throws IOException
-   */
-  public static boolean runJob(JobConf job) throws IOException {
-    JobClient jc = new JobClient(job);
-    boolean sucess = true;
-    RunningJob running = null;
-    try {
-      running = jc.submitJob(job);
-      String jobId = running.getJobID();
-      System.out.println("Job " + jobId + " is submitted");
-      while (!running.isComplete()) {
-        System.out.println("Job " + jobId + " is still running.");
-        try {
-          Thread.sleep(60000);
-        } catch (InterruptedException e) {
-        }
-        running = jc.getJob(jobId);
-      }
-      sucess = running.isSuccessful();
-    } finally {
-      if (!sucess && (running != null)) {
-        running.killJob();
-      }
-      jc.close();
-    }
-    return sucess;
-  }
-
-  /**
-   * create and run an Abacus based map/reduce job.
+   * create and run an Aggregate based map/reduce job.
    * 
    * @param args the arguments used for job creation
    * @throws IOException
    */
   public static void main(String args[]) throws IOException {
     JobConf job = ValueAggregatorJob.createValueAggregatorJob(args);
-    runJob(job);
+    JobClient.runJob(job);
   }
 }

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=555750&r1=555749&r2=555750
==============================================================================
--- 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
Thu Jul 12 13:35:47 2007
@@ -27,7 +27,7 @@
 
 /**
  * This abstract class implements some common functionalities of the
- * the generic mapper, reducer and combiner classes of Abacus.
+ * the generic mapper, reducer and combiner classes of Aggregate.
  */
 public abstract class ValueAggregatorJobBase implements Mapper, Reducer {
 
@@ -75,16 +75,7 @@
   }
 
   protected void logSpec() {
-    StringBuffer sb = new StringBuffer();
-    sb.append("\n");
-    if (aggregatorDescriptorList == null) {
-      sb.append(" aggregatorDescriptorList: null");
-    } else {
-      sb.append(" aggregatorDescriptorList: ");
-      for (int i = 0; i < aggregatorDescriptorList.size(); i++) {
-        sb.append(" ").append(aggregatorDescriptorList.get(i).toString());
-      }
-    }
+
   }
 
   public void close() throws IOException {

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=555750&r1=555749&r2=555750
==============================================================================
--- 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
Thu Jul 12 13:35:47 2007
@@ -28,7 +28,7 @@
 import org.apache.hadoop.mapred.Reporter;
 
 /**
- * This class implements the generic mapper of Abacus.
+ * This class implements the generic mapper of Aggregate.
  */
 public class ValueAggregatorMapper extends ValueAggregatorJobBase {
 

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=555750&r1=555749&r2=555750
==============================================================================
--- 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
Thu Jul 12 13:35:47 2007
@@ -28,7 +28,7 @@
 import org.apache.hadoop.mapred.Reporter;
 
 /**
- * This class implements the generic reducer of Abacus.
+ * This class implements the generic reducer of Aggregate.
  * 
  * 
  */

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=555750&r1=555749&r2=555750
==============================================================================
--- 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
Thu Jul 12 13:35:47 2007
@@ -24,7 +24,6 @@
 import java.util.Map.Entry;
 import java.util.Arrays;
 
-import org.apache.hadoop.io.Text;
 
 /**
  * This class implements a value aggregator that computes the 

Copied: lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/lib/aggregate/TestAggregates.java
(from r555744, 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/lib/aggregate/TestAggregates.java?view=diff&rev=555750&p1=lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestAggregates.java&r1=555744&p2=lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/lib/aggregate/TestAggregates.java&r2=555750
==============================================================================
--- lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestAggregates.java (original)
+++ lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/lib/aggregate/TestAggregates.java
Thu Jul 12 13:35:47 2007
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.mapred;
+package org.apache.hadoop.mapred.lib.aggregate;
 
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.io.*;
+import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.mapred.lib.*;
-import org.apache.hadoop.mapred.lib.aggregate.*;
 import junit.framework.TestCase;
 import java.io.*;
 import java.util.*;



Mime
View raw message