You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mahout.apache.org by sr...@apache.org on 2010/04/02 20:54:16 UTC
svn commit: r930352 - in
/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop:
ItemPrefWritable.java ItemWritable.java ToItemPrefsMapper.java
item/RecommenderJob.java item/RecommenderMapper.java
item/ToUserVectorReducer.java
Author: srowen
Date: Fri Apr 2 18:54:15 2010
New Revision: 930352
URL: http://svn.apache.org/viewvc?rev=930352&view=rev
Log:
MAHOUT-359 optimize better for boolean prefs, without pref value
Added:
lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemWritable.java
- copied, changed from r930052, lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemPrefWritable.java
Modified:
lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemPrefWritable.java
lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ToItemPrefsMapper.java
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/item/ToUserVectorReducer.java
Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemPrefWritable.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemPrefWritable.java?rev=930352&r1=930351&r2=930352&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemPrefWritable.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemPrefWritable.java Fri Apr 2 18:54:15 2010
@@ -24,41 +24,36 @@ import java.io.IOException;
import org.apache.hadoop.io.Writable;
/** A {@link Writable} encapsulating an item ID and a preference value. */
-public final class ItemPrefWritable implements Writable {
+public final class ItemPrefWritable extends ItemWritable {
- private long itemID;
private float prefValue;
public ItemPrefWritable() {
- // do nothing
+ // do nothing
}
public ItemPrefWritable(long itemID, float prefValue) {
- this.itemID = itemID;
+ super(itemID);
this.prefValue = prefValue;
}
public ItemPrefWritable(ItemPrefWritable other) {
this(other.getItemID(), other.getPrefValue());
}
-
- public long getItemID() {
- return itemID;
- }
-
+
public float getPrefValue() {
return prefValue;
}
@Override
public void write(DataOutput out) throws IOException {
- out.writeLong(itemID);
+ super.write(out);
out.writeFloat(prefValue);
}
@Override
public void readFields(DataInput in) throws IOException {
- itemID = in.readLong();
+ super.readFields(in);
prefValue = in.readFloat();
}
Copied: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemWritable.java (from r930052, lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemPrefWritable.java)
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemWritable.java?p2=lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemWritable.java&p1=lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemPrefWritable.java&r1=930052&r2=930352&rev=930352&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemPrefWritable.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemWritable.java Fri Apr 2 18:54:15 2010
@@ -17,55 +17,47 @@
package org.apache.mahout.cf.taste.hadoop;
+import org.apache.hadoop.io.Writable;
+
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
-import org.apache.hadoop.io.Writable;
+/** A {@link Writable} encapsulating an item ID. */
+public class ItemWritable implements Writable {
-/** A {@link Writable} encapsulating an item ID and a preference value. */
-public final class ItemPrefWritable implements Writable {
-
private long itemID;
- private float prefValue;
-
- public ItemPrefWritable() {
- // do nothing
+
+ public ItemWritable() {
+ // do nothing
}
-
- public ItemPrefWritable(long itemID, float prefValue) {
+
+ public ItemWritable(long itemID) {
this.itemID = itemID;
- this.prefValue = prefValue;
}
-
- public ItemPrefWritable(ItemPrefWritable other) {
- this(other.getItemID(), other.getPrefValue());
+
+ public ItemWritable(ItemWritable other) {
+ this(other.getItemID());
}
-
+
public long getItemID() {
return itemID;
}
-
- public float getPrefValue() {
- return prefValue;
- }
-
+
@Override
public void write(DataOutput out) throws IOException {
out.writeLong(itemID);
- out.writeFloat(prefValue);
}
-
+
@Override
public void readFields(DataInput in) throws IOException {
itemID = in.readLong();
- prefValue = in.readFloat();
}
-
- public static ItemPrefWritable read(DataInput in) throws IOException {
- ItemPrefWritable writable = new ItemPrefWritable();
+
+ public static ItemWritable read(DataInput in) throws IOException {
+ ItemWritable writable = new ItemWritable();
writable.readFields(in);
return writable;
}
-
+
}
\ No newline at end of file
Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ToItemPrefsMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ToItemPrefsMapper.java?rev=930352&r1=930351&r2=930352&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ToItemPrefsMapper.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ToItemPrefsMapper.java Fri Apr 2 18:54:15 2010
@@ -22,10 +22,12 @@ import java.util.regex.Pattern;
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.hadoop.item.RecommenderJob;
/**
* <h1>Input</h1>
@@ -48,20 +50,31 @@ import org.apache.hadoop.mapred.Reporter
* </p>
*/
public final class ToItemPrefsMapper extends MapReduceBase implements
- Mapper<LongWritable,Text,LongWritable,ItemPrefWritable> {
+ Mapper<LongWritable,Text,LongWritable,ItemWritable> {
private static final Pattern COMMA = Pattern.compile(",");
+
+ private boolean booleanData;
+
+ @Override
+ public void configure(JobConf jobConf) {
+ booleanData = jobConf.getBoolean(RecommenderJob.BOOLEAN_DATA, false);
+ }
@Override
public void map(LongWritable key,
Text value,
- OutputCollector<LongWritable,ItemPrefWritable> output,
+ OutputCollector<LongWritable,ItemWritable> output,
Reporter reporter) throws IOException {
String[] tokens = ToItemPrefsMapper.COMMA.split(value.toString());
long userID = Long.parseLong(tokens[0]);
long itemID = Long.parseLong(tokens[1]);
- float prefValue = tokens.length > 2 ? Float.parseFloat(tokens[2]) : 1.0f;
- output.collect(new LongWritable(userID), new ItemPrefWritable(itemID, prefValue));
+ if (booleanData) {
+ output.collect(new LongWritable(userID), new ItemWritable(itemID));
+ } else {
+ float prefValue = tokens.length > 2 ? Float.parseFloat(tokens[2]) : 1.0f;
+ output.collect(new LongWritable(userID), new ItemPrefWritable(itemID, prefValue));
+ }
}
}
\ No newline at end of file
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=930352&r1=930351&r2=930352&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 Fri Apr 2 18:54:15 2010
@@ -35,6 +35,7 @@ import org.apache.hadoop.mapred.TextInpu
import org.apache.hadoop.mapred.TextOutputFormat;
import org.apache.hadoop.mapred.lib.IdentityReducer;
import org.apache.hadoop.util.ToolRunner;
+import org.apache.mahout.cf.taste.hadoop.ItemWritable;
import org.apache.mahout.common.AbstractJob;
import org.apache.mahout.cf.taste.hadoop.ItemPrefWritable;
import org.apache.mahout.cf.taste.hadoop.RecommendedItemsWritable;
@@ -52,6 +53,7 @@ import org.apache.mahout.math.VectorWrit
* <li>-Dmapred.output.dir=(path): output path where recommender output should go</li>
* <li>--usersFile (path): file containing user IDs to recommend for (optional)</li>
* <li>--numRecommendations (integer): Number of recommendations to compute per user (optional; default 10)</li>
+ * <li>--booleanData (boolean): Treat input data as having to pref values (false)</li>
* </ol>
*
* <p>General command line options are documented in {@link AbstractJob}.</p>
@@ -60,15 +62,21 @@ import org.apache.mahout.math.VectorWrit
* arguments.</p>
*/
public final class RecommenderJob extends AbstractJob {
+
+ public static final String BOOLEAN_DATA = "booleanData";
@Override
public int run(String[] args) throws IOException {
Option numReccomendationsOpt = AbstractJob.buildOption("numRecommendations", "n",
"Number of recommendations per user", "10");
- Option usersFileOpt = AbstractJob.buildOption("usersFile", "u", "File of users to recommend for", null);
-
- Map<String,String> parsedArgs = AbstractJob.parseArguments(args, numReccomendationsOpt, usersFileOpt);
+ Option usersFileOpt = AbstractJob.buildOption("usersFile", "u",
+ "File of users to recommend for", null);
+ Option booleanDataOpt = AbstractJob.buildOption("booleanData", "b",
+ "Treat input as without pref values", Boolean.FALSE.toString());
+
+ Map<String,String> parsedArgs = AbstractJob.parseArguments(
+ args, numReccomendationsOpt, usersFileOpt, booleanDataOpt);
if (parsedArgs == null) {
return -1;
}
@@ -79,6 +87,7 @@ public final class RecommenderJob extend
String tempDirPath = parsedArgs.get("--tempDir");
int recommendationsPerUser = Integer.parseInt(parsedArgs.get("--numRecommendations"));
String usersFile = parsedArgs.get("--usersFile");
+ boolean booleanData = Boolean.valueOf(parsedArgs.get("--booleanData"));
String userVectorPath = tempDirPath + "/userVectors";
String itemIDIndexPath = tempDirPath + "/itemIDIndex";
@@ -90,8 +99,10 @@ public final class RecommenderJob extend
JobClient.runJob(itemIDIndexConf);
JobConf toUserVectorConf = prepareJobConf(inputPath, userVectorPath,
- TextInputFormat.class, ToItemPrefsMapper.class, LongWritable.class, ItemPrefWritable.class,
+ TextInputFormat.class, ToItemPrefsMapper.class, LongWritable.class,
+ booleanData ? ItemWritable.class : ItemPrefWritable.class,
ToUserVectorReducer.class, LongWritable.class, VectorWritable.class, SequenceFileOutputFormat.class);
+ toUserVectorConf.setBoolean(BOOLEAN_DATA, booleanData);
JobClient.runJob(toUserVectorConf);
JobConf toCooccurrenceConf = prepareJobConf(userVectorPath, cooccurrencePath,
@@ -104,6 +115,7 @@ public final class RecommenderJob extend
SequenceFileInputFormat.class, RecommenderMapper.class, LongWritable.class,
RecommendedItemsWritable.class, IdentityReducer.class, LongWritable.class,
RecommendedItemsWritable.class, TextOutputFormat.class);
+ recommenderConf.setBoolean(BOOLEAN_DATA, booleanData);
recommenderConf.set(RecommenderMapper.COOCCURRENCE_PATH, cooccurrencePath);
recommenderConf.set(RecommenderMapper.ITEMID_INDEX_PATH, itemIDIndexPath);
recommenderConf.setInt(RecommenderMapper.RECOMMENDATIONS_PER_USER, recommendationsPerUser);
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=930352&r1=930351&r2=930352&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 Fri Apr 2 18:54:15 2010
@@ -62,6 +62,7 @@ public final class RecommenderMapper ext
private MapFilesMap<IntWritable,VectorWritable> cooccurrenceColumnMap;
private Cache<IntWritable,Vector> cooccurrenceColumnCache;
private FastIDSet usersToRecommendFor;
+ private boolean booleanData;
@Override
public void configure(JobConf jobConf) {
@@ -88,6 +89,7 @@ public final class RecommenderMapper ext
throw new IllegalStateException(ioe);
}
cooccurrenceColumnCache = new Cache<IntWritable,Vector>(new CooccurrenceCache(cooccurrenceColumnMap), 100);
+ booleanData = jobConf.getBoolean(RecommenderJob.BOOLEAN_DATA, false);
}
@Override
@@ -105,7 +107,6 @@ public final class RecommenderMapper ext
while (userVectorIterator.hasNext()) {
Vector.Element element = userVectorIterator.next();
int index = element.index();
- double value = element.get();
Vector columnVector;
try {
columnVector = cooccurrenceColumnCache.get(new IntWritable(index));
@@ -117,7 +118,12 @@ public final class RecommenderMapper ext
}
}
if (columnVector != null) {
- columnVector.times(value).addTo(recommendationVector);
+ if (booleanData) { // because 'value' is 1.0
+ columnVector.addTo(recommendationVector);
+ } else {
+ double value = element.get();
+ columnVector.times(value).addTo(recommendationVector);
+ }
}
}
Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/ToUserVectorReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/ToUserVectorReducer.java?rev=930352&r1=930351&r2=930352&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/ToUserVectorReducer.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/ToUserVectorReducer.java Fri Apr 2 18:54:15 2010
@@ -23,11 +23,13 @@ import java.util.PriorityQueue;
import java.util.Queue;
import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapred.JobConf;
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 org.apache.mahout.cf.taste.hadoop.ItemPrefWritable;
+import org.apache.mahout.cf.taste.hadoop.ItemWritable;
import org.apache.mahout.math.RandomAccessSparseVector;
import org.apache.mahout.math.Vector;
import org.apache.mahout.math.VectorWritable;
@@ -56,26 +58,38 @@ import org.apache.mahout.math.VectorWrit
*
*/
public final class ToUserVectorReducer extends MapReduceBase implements
- Reducer<LongWritable,ItemPrefWritable,LongWritable,VectorWritable> {
+ Reducer<LongWritable,ItemWritable,LongWritable,VectorWritable> {
public static final int MAX_PREFS_CONSIDERED = 20;
private final VectorWritable vectorWritable = new VectorWritable();
+ private boolean booleanData;
+
+ @Override
+ public void configure(JobConf jobConf) {
+ booleanData = jobConf.getBoolean(RecommenderJob.BOOLEAN_DATA, false);
+ }
@Override
public void reduce(LongWritable userID,
- Iterator<ItemPrefWritable> itemPrefs,
+ Iterator<ItemWritable> itemPrefs,
OutputCollector<LongWritable,VectorWritable> output,
Reporter reporter) throws IOException {
if (itemPrefs.hasNext()) {
RandomAccessSparseVector userVector = new RandomAccessSparseVector(Integer.MAX_VALUE, 100);
while (itemPrefs.hasNext()) {
- ItemPrefWritable itemPref = itemPrefs.next();
+ ItemWritable itemPref = itemPrefs.next();
int index = ItemIDIndexMapper.idToIndex(itemPref.getItemID());
- userVector.set(index, itemPref.getPrefValue());
+ float value;
+ if (itemPref instanceof ItemPrefWritable) {
+ value = ((ItemPrefWritable) itemPref).getPrefValue();
+ } else {
+ value = 1.0f;
+ }
+ userVector.set(index, value);
}
- if (userVector.getNumNondefaultElements() > MAX_PREFS_CONSIDERED) {
+ if (!booleanData && userVector.getNumNondefaultElements() > MAX_PREFS_CONSIDERED) {
double cutoff = findTopNPrefsCutoff(MAX_PREFS_CONSIDERED,
userVector);
RandomAccessSparseVector filteredVector = new RandomAccessSparseVector(Integer.MAX_VALUE,