mahout-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From sro...@apache.org
Subject svn commit: r660055 - /lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/
Date Sun, 25 May 2008 23:15:15 GMT
Author: srowen
Date: Sun May 25 16:15:15 2008
New Revision: 660055

URL: http://svn.apache.org/viewvc?rev=660055&view=rev
Log:
A few updates for Hadoop 0.17.0; committed two mapreduces to generate item-item diffs for
SlopeOneRecommender

Added:
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ByItemIDComparator.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemItemWritable.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemPrefWritable.java
      - copied, changed from r657697, lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/RecommendedItemsWritable.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOneDiffsToAveragesJob.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOneDiffsToAveragesReducer.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOnePrefsToDiffsJob.java
      - copied, changed from r657697, lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/RecommenderJob.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOnePrefsToDiffsMapper.java
      - copied, changed from r657697, lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/RecommenderMapper.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOnePrefsToDiffsReducer.java
Modified:
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/RecommenderJob.java

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ByItemIDComparator.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ByItemIDComparator.java?rev=660055&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ByItemIDComparator.java
(added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ByItemIDComparator.java
Sun May 25 16:15:15 2008
@@ -0,0 +1,34 @@
+/**
+ * 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.mahout.cf.taste.hadoop;
+
+import java.util.Comparator;
+
+final class ByItemIDComparator implements Comparator<ItemPrefWritable> {
+
+  private static final ByItemIDComparator instance = new ByItemIDComparator();
+
+  public static ByItemIDComparator getInstance() {
+    return instance;
+  }
+
+  public int compare(ItemPrefWritable a, ItemPrefWritable b) {
+    return a.getItemID().compareTo(b.getItemID());
+  }
+
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemItemWritable.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemItemWritable.java?rev=660055&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemItemWritable.java
(added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemItemWritable.java
Sun May 25 16:15:15 2008
@@ -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.mahout.cf.taste.hadoop;
+
+import org.apache.hadoop.io.WritableComparable;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ */
+public final class ItemItemWritable implements WritableComparable {
+
+  private String itemAID;
+  private String itemBID;
+
+  public ItemItemWritable() {
+    // do nothing
+  }
+
+  public ItemItemWritable(String itemAID, String itemBID) {
+    this.itemAID = itemAID;
+    this.itemBID = itemBID;
+  }
+
+  public String getItemAID() {
+    return itemAID;
+  }
+
+  public String getItemBID() {
+    return itemBID;
+  }
+
+  public void write(DataOutput out) throws IOException {
+    out.writeUTF(getItemAID());
+    out.writeUTF(getItemBID());
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    itemAID = in.readUTF();
+    itemBID = in.readUTF();
+  }
+
+  public static ItemItemWritable read(DataInput in) throws IOException {
+    ItemItemWritable writable = new ItemItemWritable();
+    writable.readFields(in);
+    return writable;
+  }
+
+  public int compareTo(Object o) {
+    ItemItemWritable that = (ItemItemWritable) o;
+    if (this == that) {
+      return 0;
+    }
+    int compare = itemAID.compareTo(that.itemAID);
+    if (compare == 0) {
+      return itemBID.compareTo(that.itemBID);
+    } else {
+      return compare;
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return itemAID.hashCode() + 31 * itemBID.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof ItemItemWritable) {
+      ItemItemWritable that = (ItemItemWritable) o;
+      return this == that || (itemAID.equals(that.itemAID) && itemBID.equals(that.itemBID));
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return itemAID + '\t' + itemBID;
+  }
+
+}
\ No newline at end of file

Copied: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemPrefWritable.java
(from r657697, lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/RecommendedItemsWritable.java)
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemPrefWritable.java?p2=lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemPrefWritable.java&p1=lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/RecommendedItemsWritable.java&r1=657697&r2=660055&rev=660055&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/RecommendedItemsWritable.java
(original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemPrefWritable.java
Sun May 25 16:15:15 2008
@@ -18,85 +18,53 @@
 package org.apache.mahout.cf.taste.hadoop;
 
 import org.apache.hadoop.io.Writable;
-import org.apache.mahout.cf.taste.impl.model.GenericItem;
-import org.apache.mahout.cf.taste.impl.recommender.GenericRecommendedItem;
-import org.apache.mahout.cf.taste.model.Item;
-import org.apache.mahout.cf.taste.recommender.RecommendedItem;
 
 import java.io.DataInput;
 import java.io.DataOutput;
-import java.io.EOFException;
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
 
 /**
- * A {@link Writable} which encapsulates a list of {@link RecommendedItem}s. This is
- * the mapper (and reducer) output, and represents items recommended to a user.
- * The first item is the one whose estimated preference is highest.
  */
-public final class RecommendedItemsWritable implements Writable {
+public final class ItemPrefWritable implements Writable {
 
-  private List<RecommendedItem> recommended;
+  private String itemID;
+  private double prefValue;
 
-  public RecommendedItemsWritable() {
+  public ItemPrefWritable() {
     // do nothing
   }
 
-  public RecommendedItemsWritable(List<RecommendedItem> recommended) {
-    this.recommended = recommended;
+  public ItemPrefWritable(String itemID, double prefValue) {
+    this.itemID = itemID;
+    this.prefValue = prefValue;
   }
 
-  public List<RecommendedItem> getRecommendedItems() {
-    return recommended;
+  public ItemPrefWritable(ItemPrefWritable other) {
+    this(other.getItemID(), other.getPrefValue());
   }
 
-  public void write(DataOutput out) throws IOException {
-    for (RecommendedItem item : recommended) {
-      out.writeUTF(item.getItem().getID().toString());
-      out.writeDouble(item.getValue());
-    }
+  public String getItemID() {
+    return itemID;
+  }
 
+  public double getPrefValue() {
+    return prefValue;
+  }
+
+  public void write(DataOutput out) throws IOException {
+    out.writeUTF(itemID);
+    out.writeDouble(prefValue);
   }
 
   public void readFields(DataInput in) throws IOException {
-    recommended = new ArrayList<RecommendedItem>();
-    try {
-      do {
-        String itemID = in.readUTF();
-        double value = in.readDouble();
-        Item item = new GenericItem<String>(itemID);
-        RecommendedItem recommendedItem = new GenericRecommendedItem(item, value);
-        recommended.add(recommendedItem);
-      } while (true);
-    } catch (EOFException eofe) {
-      // continue; done
-    }
+    itemID = in.readUTF();
+    prefValue = in.readDouble();
   }
 
-  public static RecommendedItemsWritable read(DataInput in) throws IOException {
-    RecommendedItemsWritable writable = new RecommendedItemsWritable();
+  public static ItemPrefWritable read(DataInput in) throws IOException {
+    ItemPrefWritable writable = new ItemPrefWritable();
     writable.readFields(in);
     return writable;
   }
 
-  @Override
-  public String toString() {
-    StringBuilder result = new StringBuilder();
-    result.append('[');
-    boolean first = true;
-    for (RecommendedItem item : recommended) {
-      if (first) {
-        first = false;
-      } else {
-        result.append(',');
-      }
-      result.append(item.getItem().getID().toString());
-      result.append(':');
-      result.append(item.getValue());
-    }
-    result.append(']');
-    return result.toString();
-  }
-
-}
+}
\ No newline at end of file

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/RecommenderJob.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/RecommenderJob.java?rev=660055&r1=660054&r2=660055&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/RecommenderJob.java
(original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/RecommenderJob.java
Sun May 25 16:15:15 2008
@@ -24,6 +24,8 @@
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hadoop.mapred.TextOutputFormat;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
 import org.apache.mahout.cf.taste.recommender.Recommender;
 
@@ -60,9 +62,8 @@
     String userIDFile = args[2];
     String dataModelFile = args[3];
     String outputPath = args[4];
-    int numMappers = Integer.parseInt(args[5]);
     JobConf jobConf =
-        buildJobConf(recommendClassName, recommendationsPerUser, userIDFile, dataModelFile,
outputPath, numMappers);
+        buildJobConf(recommendClassName, recommendationsPerUser, userIDFile, dataModelFile,
outputPath);
     JobClient.runJob(jobConf);
   }
 
@@ -70,8 +71,7 @@
                                      int recommendationsPerUser,
                                      String userIDFile,
                                      String dataModelFile,
-                                     String outputPath,
-                                     int numMappers) throws IOException {
+                                     String outputPath) throws IOException {
 
     Path userIDFilePath = new Path(userIDFile);
     Path outputPathPath = new Path(outputPath);
@@ -80,30 +80,26 @@
 
     FileSystem fs = FileSystem.get(jobConf);
     if (fs.exists(outputPathPath)) {
-      fs.delete(outputPathPath);
+      fs.delete(outputPathPath, true);
     }
 
     jobConf.set(RecommenderMapper.RECOMMENDER_CLASS_NAME, recommendClassName);
     jobConf.set(RecommenderMapper.RECOMMENDATIONS_PER_USER, String.valueOf(recommendationsPerUser));
     jobConf.set(RecommenderMapper.DATA_MODEL_FILE, dataModelFile);
 
-    jobConf.setJobName(RecommenderJob.class.getSimpleName());
-
     jobConf.setInputFormat(TextInputFormat.class);
-    jobConf.setInputPath(userIDFilePath);
+    FileInputFormat.setInputPaths(jobConf, userIDFilePath);
 
-    jobConf.setNumMapTasks(numMappers);
     jobConf.setMapperClass(RecommenderMapper.class);
     jobConf.setMapOutputKeyClass(Text.class);
     jobConf.setMapOutputValueClass(RecommendedItemsWritable.class);
 
-    jobConf.setNumReduceTasks(1);
     jobConf.setReducerClass(IdentityReducer.class);
     jobConf.setOutputKeyClass(Text.class);
     jobConf.setOutputValueClass(RecommendedItemsWritable.class);
 
     jobConf.setOutputFormat(TextOutputFormat.class);
-    jobConf.setOutputPath(outputPathPath);
+    FileOutputFormat.setOutputPath(jobConf, outputPathPath);
 
     return jobConf;
   }

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOneDiffsToAveragesJob.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOneDiffsToAveragesJob.java?rev=660055&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOneDiffsToAveragesJob.java
(added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOneDiffsToAveragesJob.java
Sun May 25 16:15:15 2008
@@ -0,0 +1,74 @@
+/**
+ * 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.mahout.cf.taste.hadoop;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.SequenceFileInputFormat;
+import org.apache.hadoop.mapred.TextOutputFormat;
+import org.apache.hadoop.mapred.lib.IdentityMapper;
+
+import java.io.IOException;
+
+/**
+ */
+public final class SlopeOneDiffsToAveragesJob {
+
+  public static void main(String[] args) throws IOException {
+    String prefsFile = args[0];
+    String outputPath = args[1];
+    JobConf jobConf = buildJobConf(prefsFile, outputPath);
+    JobClient.runJob(jobConf);
+  }
+
+  public static JobConf buildJobConf(String prefsFile,
+                                     String outputPath) throws IOException {
+
+    Path prefsFilePath = new Path(prefsFile);
+    Path outputPathPath = new Path(outputPath);
+
+    JobConf jobConf = new JobConf(SlopeOnePrefsToDiffsJob.class);
+
+    FileSystem fs = FileSystem.get(jobConf);
+    if (fs.exists(outputPathPath)) {
+      fs.delete(outputPathPath, true);
+    }
+
+    jobConf.setInputFormat(SequenceFileInputFormat.class);
+    FileInputFormat.setInputPaths(jobConf, prefsFilePath);
+
+    jobConf.setMapperClass(IdentityMapper.class);
+    jobConf.setMapOutputKeyClass(ItemItemWritable.class);
+    jobConf.setMapOutputValueClass(FloatWritable.class);
+
+    jobConf.setReducerClass(SlopeOneDiffsToAveragesReducer.class);
+    jobConf.setOutputKeyClass(ItemItemWritable.class);
+    jobConf.setOutputValueClass(FloatWritable.class);
+
+    jobConf.setOutputFormat(TextOutputFormat.class);
+    FileOutputFormat.setOutputPath(jobConf, outputPathPath);
+
+    return jobConf;
+  }
+
+}
\ No newline at end of file

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOneDiffsToAveragesReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOneDiffsToAveragesReducer.java?rev=660055&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOneDiffsToAveragesReducer.java
(added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOneDiffsToAveragesReducer.java
Sun May 25 16:15:15 2008
@@ -0,0 +1,48 @@
+/**
+ * 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.mahout.cf.taste.hadoop;
+
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.mapred.MapReduceBase;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+/**
+ */
+public final class SlopeOneDiffsToAveragesReducer
+    extends MapReduceBase
+    implements Reducer<ItemItemWritable, FloatWritable, ItemItemWritable, FloatWritable>
{
+
+  public void reduce(ItemItemWritable key,
+                     Iterator<FloatWritable> values,
+                     OutputCollector<ItemItemWritable, FloatWritable> output,
+                     Reporter reporter) throws IOException {
+    int count = 0;
+    float total = 0.0f;
+    while (values.hasNext()) {
+      total += values.next().get();
+      count++;
+    }
+    output.collect(key, new FloatWritable(total / count));
+  }
+
+}
\ No newline at end of file

Copied: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOnePrefsToDiffsJob.java
(from r657697, lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/RecommenderJob.java)
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOnePrefsToDiffsJob.java?p2=lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOnePrefsToDiffsJob.java&p1=lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/RecommenderJob.java&r1=657697&r2=660055&rev=660055&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/RecommenderJob.java
(original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOnePrefsToDiffsJob.java
Sun May 25 16:15:15 2008
@@ -20,93 +20,58 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.SequenceFileOutputFormat;
 import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.mapred.FileInputFormat;
 import org.apache.hadoop.mapred.TextOutputFormat;
-import org.apache.hadoop.mapred.lib.IdentityReducer;
-import org.apache.mahout.cf.taste.recommender.Recommender;
 
 import java.io.IOException;
 
 /**
- * <p>This class configures and runs a {@link RecommenderMapper} using Hadoop.</p>
- *
- * <p>Command line arguments are:</p>
- * <ol>
- *  <li>Fully-qualified class name of {@link Recommender} to use to make recommendations.
- *   Note that it must have a no-arg constructor.</li>
- *  <li>Number of recommendations to compute per user</li>
- *  <li>Location of 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, 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>Number of mapper tasks to use</li>
- * </ol>
- *
- * <p>Example:</p>
- *
- * <p><code>org.apache.mahout.cf.taste.impl.recommender.slopeone.SlopeOneRecommender
10 path/to/users.txt
- *  path/to/data.csv path/to/reducerOutputDir 5</code></p>
- *
- * <p>TODO I am not a bit sure this works yet in a real distributed environment.</p>
  */
-public final class RecommenderJob {
+public final class SlopeOnePrefsToDiffsJob {
 
   public static void main(String[] args) throws IOException {
-    String recommendClassName = args[0];
-    int recommendationsPerUser = Integer.parseInt(args[1]);
-    String userIDFile = args[2];
-    String dataModelFile = args[3];
-    String outputPath = args[4];
-    int numMappers = Integer.parseInt(args[5]);
-    JobConf jobConf =
-        buildJobConf(recommendClassName, recommendationsPerUser, userIDFile, dataModelFile,
outputPath, numMappers);
+    String prefsFile = args[0];
+    String outputPath = args[1];
+    JobConf jobConf = buildJobConf(prefsFile, outputPath);
     JobClient.runJob(jobConf);
   }
 
-  public static JobConf buildJobConf(String recommendClassName,
-                                     int recommendationsPerUser,
-                                     String userIDFile,
-                                     String dataModelFile,
-                                     String outputPath,
-                                     int numMappers) throws IOException {
+  public static JobConf buildJobConf(String prefsFile,
+                                     String outputPath) throws IOException {
 
-    Path userIDFilePath = new Path(userIDFile);
+    Path prefsFilePath = new Path(prefsFile);
     Path outputPathPath = new Path(outputPath);
 
-    JobConf jobConf = new JobConf(Recommender.class);
+    JobConf jobConf = new JobConf(SlopeOnePrefsToDiffsJob.class);
 
     FileSystem fs = FileSystem.get(jobConf);
     if (fs.exists(outputPathPath)) {
-      fs.delete(outputPathPath);
+      fs.delete(outputPathPath, true);
     }
-    fs.mkdirs(outputPathPath);
-
-    jobConf.set(RecommenderMapper.RECOMMENDER_CLASS_NAME, recommendClassName);
-    jobConf.set(RecommenderMapper.RECOMMENDATIONS_PER_USER, String.valueOf(recommendationsPerUser));
-    jobConf.set(RecommenderMapper.DATA_MODEL_FILE, dataModelFile);
-
-    jobConf.setJobName(RecommenderJob.class.getSimpleName());
 
     jobConf.setInputFormat(TextInputFormat.class);
-    jobConf.setInputPath(userIDFilePath);
+    FileInputFormat.setInputPaths(jobConf, prefsFilePath);
 
-    jobConf.setNumMapTasks(numMappers);
-    jobConf.setMapperClass(RecommenderMapper.class);
+    jobConf.setMapperClass(SlopeOnePrefsToDiffsMapper.class);
     jobConf.setMapOutputKeyClass(Text.class);
-    jobConf.setMapOutputValueClass(RecommendedItemsWritable.class);
-
-    jobConf.setNumReduceTasks(1);
-    jobConf.setReducerClass(IdentityReducer.class);
-    jobConf.setOutputKeyClass(Text.class);
-    jobConf.setOutputValueClass(RecommendedItemsWritable.class);
+    jobConf.setMapOutputValueClass(ItemPrefWritable.class);
 
-    jobConf.setOutputFormat(TextOutputFormat.class);
-    jobConf.setOutputPath(outputPathPath);
+    jobConf.setReducerClass(SlopeOnePrefsToDiffsReducer.class);
+    jobConf.setOutputKeyClass(ItemItemWritable.class);
+    jobConf.setOutputValueClass(FloatWritable.class);
+
+    jobConf.setOutputFormat(SequenceFileOutputFormat.class);
+    SequenceFileOutputFormat.setOutputCompressionType(jobConf, SequenceFile.CompressionType.RECORD);
+    FileOutputFormat.setOutputPath(jobConf, outputPathPath);
 
     return jobConf;
   }
 
-}
+}
\ No newline at end of file

Copied: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOnePrefsToDiffsMapper.java
(from r657697, lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/RecommenderMapper.java)
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOnePrefsToDiffsMapper.java?p2=lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOnePrefsToDiffsMapper.java&p1=lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/RecommenderMapper.java&r1=657697&r2=660055&rev=660055&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/RecommenderMapper.java
(original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOnePrefsToDiffsMapper.java
Sun May 25 16:15:15 2008
@@ -17,95 +17,31 @@
 
 package org.apache.mahout.cf.taste.hadoop;
 
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MapReduceBase;
 import org.apache.hadoop.mapred.Mapper;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reporter;
-import org.apache.mahout.cf.taste.common.TasteException;
-import org.apache.mahout.cf.taste.impl.model.file.FileDataModel;
-import org.apache.mahout.cf.taste.model.DataModel;
-import org.apache.mahout.cf.taste.recommender.RecommendedItem;
-import org.apache.mahout.cf.taste.recommender.Recommender;
 
-import java.io.File;
 import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
-import java.util.List;
 
 /**
- * <p>The {@link Mapper} which takes as input a file of user IDs (treated as Strings,
note), and
- * for each unique user ID, computes recommendations with the configured {@link Recommender}.
- * The results are output as {@link RecommendedItemsWritable}.</p>
- *
- * <p>Note that there is no corresponding {@link org.apache.hadoop.mapred.Reducer};
this
- * implementation can only partially take advantage of the mapreduce paradigm and only
- * really leverages it for easy parallelization.</p>
  */
-public final class RecommenderMapper
+public final class SlopeOnePrefsToDiffsMapper
     extends MapReduceBase
-    implements Mapper<LongWritable, Text, Text, RecommendedItemsWritable> {
-
-  static final String RECOMMENDER_CLASS_NAME = "recommenderClassName";
-  static final String RECOMMENDATIONS_PER_USER = "recommendadtionsPerUser";
-  static final String DATA_MODEL_FILE = "dataModelFile";
-
-  private Recommender recommender;
-  private int recommendationsPerUser;
+    implements Mapper<LongWritable, Text, Text, ItemPrefWritable> {
 
   public void map(LongWritable key,
                   Text value,
-                  OutputCollector<Text, RecommendedItemsWritable> output,
+                  OutputCollector<Text, ItemPrefWritable> output,
                   Reporter reporter) throws IOException {
-    String userID = value.toString();
-    List<RecommendedItem> recommendedItems;
-    try {
-      recommendedItems = recommender.recommend(userID, recommendationsPerUser);
-    } catch (TasteException te) {
-      throw new RuntimeException(te);
-    }
-    RecommendedItemsWritable writable = new RecommendedItemsWritable(recommendedItems);
-    output.collect(new Text(userID), writable);
-    reporter.incrCounter(ReducerMetrics.USERS_PROCESSED, 1L);
-    reporter.incrCounter(ReducerMetrics.RECOMMENDATIONS_MADE, recommendedItems.size());
-  }
-
-  @Override
-  public void configure(JobConf jobConf) {
-    String dataModelFile = jobConf.get(DATA_MODEL_FILE);
-    String recommenderClassName = jobConf.get(RECOMMENDER_CLASS_NAME);
-    FileDataModel fileDataModel;
-    try {
-      FileSystem fs = FileSystem.get(jobConf);
-      File tempDataFile = File.createTempFile("mahout-taste-hadoop", "txt");
-      tempDataFile.deleteOnExit();
-      fs.copyToLocalFile(new Path(dataModelFile), new Path(tempDataFile.getAbsolutePath()));
-      fileDataModel = new FileDataModel(tempDataFile);
-    } catch (IOException ioe) {
-      throw new RuntimeException(ioe);
-    }
-    try {
-      Class<? extends Recommender> recommenderClass =
-          (Class<? extends Recommender>) Class.forName(recommenderClassName);
-      Constructor<? extends Recommender> constructor = recommenderClass.getConstructor(DataModel.class);
-      recommender = constructor.newInstance(fileDataModel);
-    } catch (NoSuchMethodException nsme) {
-      throw new RuntimeException(nsme);
-    } catch (ClassNotFoundException cnfe) {
-      throw new RuntimeException(cnfe);
-    } catch (InstantiationException ie) {
-      throw new RuntimeException(ie);
-    } catch (IllegalAccessException iae) {
-      throw new RuntimeException(iae);
-    } catch (InvocationTargetException ite) {
-      throw new RuntimeException(ite.getCause());
-    }
-    recommendationsPerUser = Integer.parseInt(jobConf.get(RECOMMENDATIONS_PER_USER));
+    String line = value.toString();
+    String[] tokens = line.split(",");
+    String userID = tokens[0];
+    String itemID = tokens[1];
+    double prefValue = Double.parseDouble(tokens[2]);
+    output.collect(new Text(userID), new ItemPrefWritable(itemID, prefValue));
   }
 
-}
+}
\ No newline at end of file

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOnePrefsToDiffsReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOnePrefsToDiffsReducer.java?rev=660055&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOnePrefsToDiffsReducer.java
(added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/SlopeOnePrefsToDiffsReducer.java
Sun May 25 16:15:15 2008
@@ -0,0 +1,62 @@
+/**
+ * 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.mahout.cf.taste.hadoop;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.mapred.MapReduceBase;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ */
+public final class SlopeOnePrefsToDiffsReducer
+    extends MapReduceBase
+    implements Reducer<Text, ItemPrefWritable, ItemItemWritable, FloatWritable> {
+
+  public void reduce(Text key,
+                     Iterator<ItemPrefWritable> values,
+                     OutputCollector<ItemItemWritable, FloatWritable> output,
+                     Reporter reporter) throws IOException {
+    List<ItemPrefWritable> prefs = new ArrayList<ItemPrefWritable>();
+    while (values.hasNext()) {
+      prefs.add(new ItemPrefWritable(values.next()));
+    }
+    Collections.sort(prefs, ByItemIDComparator.getInstance());
+    int size = prefs.size();
+    for (int i = 0; i < size; i++) {
+      ItemPrefWritable first = prefs.get(i);
+      String itemAID = first.getItemID();
+      double itemAValue = first.getPrefValue();
+      for (int j = i + 1; j < size; j++) {
+        ItemPrefWritable second = prefs.get(j);
+        String itemBID = second.getItemID();
+        double itemBValue = second.getPrefValue();
+        output.collect(new ItemItemWritable(itemAID, itemBID), new FloatWritable((float)
(itemBValue - itemAValue)));
+      }
+    }
+  }
+
+}
\ No newline at end of file



Mime
View raw message