mahout-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From sro...@apache.org
Subject svn commit: r887757 - in /lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop: item/RecommenderJob.java item/RecommenderMapper.java pseudo/RecommenderJob.java
Date Sun, 06 Dec 2009 20:00:21 GMT
Author: srowen
Date: Sun Dec  6 20:00:21 2009
New Revision: 887757

URL: http://svn.apache.org/viewvc?rev=887757&view=rev
Log:
Finally working item-based Hadoop recommender

Modified:
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/RecommenderJob.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/RecommenderMapper.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/pseudo/RecommenderJob.java

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/RecommenderJob.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/RecommenderJob.java?rev=887757&r1=887756&r2=887757&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/RecommenderJob.java
(original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/RecommenderJob.java
Sun Dec  6 20:00:21 2009
@@ -37,6 +37,22 @@
 import java.io.IOException;
 import java.util.Map;
 
+/**
+ * Runs a completely distributed recommender job as a series of mapreduces.</p>
+ *
+ * <p>Command line arguments are:</p>
+ *
+ * <ol>
+ *  <li>numRecommendations: Number of recommendations to compute per user</li>
+ *  <li>input: Directory containing a text file containing user IDs
+ *   for which recommendations should be computed, one per line</li>
+ *  <li>output: output path where recommender output should go</li>
+ *  <li>jarFile: JAR file containing implementation code</li>
+ *  <li>tempDir: directory in which to place intermediate data files</li>
+ * </ol>
+ *
+ * @see org.apache.mahout.cf.taste.hadoop.pseudo.RecommenderJob
+ */
 public final class RecommenderJob extends AbstractJob {
 
   @Override
@@ -94,7 +110,7 @@
                                                 SequenceFileOutputFormat.class);
     JobClient.runJob(toCooccurrenceConf);
 
-    JobConf recommenderConf = prepareJobConf(cooccurrencePath,
+    JobConf recommenderConf = prepareJobConf(userVectorPath,
                                              outputPath,
                                              jarFile,
                                              SequenceFileInputFormat.class,

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/RecommenderMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/RecommenderMapper.java?rev=887757&r1=887756&r2=887757&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/RecommenderMapper.java
(original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/RecommenderMapper.java
Sun Dec  6 20:00:21 2009
@@ -18,6 +18,7 @@
 package org.apache.mahout.cf.taste.hadoop.item;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
@@ -53,7 +54,6 @@
 
   private FileSystem fs;
   private Path cooccurrencePath;
-  private Path itemIDIndexPath;
   private int recommendationsPerUser;
   private FastByIDMap<Long> indexItemIDMap;
 
@@ -65,15 +65,19 @@
       throw new IllegalStateException(ioe);
     }
     cooccurrencePath = new Path(jobConf.get(COOCCURRENCE_PATH)).makeQualified(fs);
-    itemIDIndexPath = new Path(jobConf.get(ITEMID_INDEX_PATH)).makeQualified(fs);
+    Path itemIDIndexPath = new Path(jobConf.get(ITEMID_INDEX_PATH)).makeQualified(fs);
     recommendationsPerUser = jobConf.getInt(RECOMMENDATIONS_PER_USER, 10);
     indexItemIDMap = new FastByIDMap<Long>();
     try {
-      SequenceFile.Reader reader = new SequenceFile.Reader(fs, itemIDIndexPath, new Configuration());
       IntWritable index = new IntWritable();
       LongWritable itemID = new LongWritable();
-      while (reader.next(index, itemID)) {
-        indexItemIDMap.put(index.get(), itemID.get());
+      Configuration conf = new Configuration();
+      for (FileStatus status : fs.listStatus(itemIDIndexPath)) {
+        SequenceFile.Reader reader = new SequenceFile.Reader(fs, status.getPath(), conf);
+        while (reader.next(index, itemID)) {
+          indexItemIDMap.put(index.get(), itemID.get());
+        }
+        reader.close();
       }
     } catch (IOException ioe) {
       throw new IllegalStateException(ioe);
@@ -85,19 +89,22 @@
                   SparseVector userVector,
                   OutputCollector<LongWritable, RecommendedItemsWritable> output,
                   Reporter reporter) throws IOException {
-
-    SequenceFile.Reader reader = new SequenceFile.Reader(fs, cooccurrencePath, new Configuration());
     IntWritable indexWritable = new IntWritable();
     Vector cooccurrenceVector = new SparseVector(Integer.MAX_VALUE, 1000);
+    Configuration conf = new Configuration();
     Queue<RecommendedItem> topItems =
         new PriorityQueue<RecommendedItem>(recommendationsPerUser + 1, Collections.reverseOrder());
-    while (reader.next(indexWritable, cooccurrenceVector)) {
-      Long itemID = indexItemIDMap.get(indexWritable.get());
-      if (itemID != null) {
-        processOneRecommendation(userVector, itemID, cooccurrenceVector, topItems);
-      } else {
-        throw new IllegalStateException("Found index without item ID: " + indexWritable.get());
+    for (FileStatus status : fs.listStatus(cooccurrencePath)) {
+      SequenceFile.Reader reader = new SequenceFile.Reader(fs, status.getPath(), conf);
+      while (reader.next(indexWritable, cooccurrenceVector)) {
+        Long itemID = indexItemIDMap.get(indexWritable.get());
+        if (itemID != null) {
+          processOneRecommendation(userVector, itemID, cooccurrenceVector, topItems);
+        } else {
+          throw new IllegalStateException("Found index without item ID: " + indexWritable.get());
+        }
       }
+      reader.close();
     }
     List<RecommendedItem> recommendations = new ArrayList<RecommendedItem>(topItems.size());
     recommendations.addAll(topItems);

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/pseudo/RecommenderJob.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/pseudo/RecommenderJob.java?rev=887757&r1=887756&r2=887757&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/pseudo/RecommenderJob.java
(original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/pseudo/RecommenderJob.java
Sun Dec  6 20:00:21 2009
@@ -42,34 +42,21 @@
  * <p>Command line arguments are:</p>
  *
  * <ol>
- *  <li>Fully-qualified class name of {@link Recommender} to use to make
+ *  <li>recommenderClassName: Fully-qualified class name of {@link Recommender} to
use to make
  *   recommendations. Note that it must have a constructor which takes a
  *   {@link org.apache.mahout.cf.taste.model.DataModel} argument.</li>
- *  <li>Number of recommendations to compute per user</li>
- *  <li>Location of a text file containing user IDs
+ *  <li>numRecommendations: Number of recommendations to compute per user</li>
+ *  <li>input: Directory containing a text file containing user IDs
  *   for which recommendations should be computed, one per line</li>
- *  <li>Location of a data model file containing preference data,
+ *  <li>dataModelFile: Location of a data model file containing preference data,
  *   suitable for use with {@link org.apache.mahout.cf.taste.impl.model.file.FileDataModel}</li>
- *  <li>Output path where reducer output should go</li>
- *  <li>JAR file containing implementation code</li>
+ *  <li>output: output path where recommender output should go</li>
+ *  <li>jarFile: JAR file containing implementation code</li>
  * </ol>
  *
- * <p>Example arguments:</p>
- *
- * {@code
- * --recommenderClassName org.apache.mahout.cf.taste.impl.recommender.slopeone.SlopeOneRecommender
- * --numRecommendations 10 --input path/to/users.txt --dataModelFile path/to/data.csv
- * --output path/to/reducerOutputDir --jarFile recommender.jar
- * }
- *
  * <p>For example, to get started trying this out, set up Hadoop in a
  * pseudo-distributed manner: http://hadoop.apache.org/common/docs/current/quickstart.html
- * You can stop at the point where it instructs you to copy files into HDFS. Instead, proceed
as follow.</p>
- *
- * {@code
- * hadoop fs -mkdir input
- * hadoop fs -mkdir output
- * }
+ * You can stop at the point where it instructs you to copy files into HDFS.</p>
  *
  * <p>Assume your preference data file is <code>input.csv</code>. You will
also need to create a file
  * containing all user IDs to write recommendations for, as something like <code>users.txt</code>.



Mime
View raw message