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 2009/12/12 23:50:02 UTC
svn commit: r889995 - in
/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item:
RecommenderMapper.java ToUserVectorReducer.java
UserVectorToCooccurrenceReducer.java
Author: srowen
Date: Sat Dec 12 22:50:02 2009
New Revision: 889995
URL: http://svn.apache.org/viewvc?rev=889995&view=rev
Log:
More improvements to item-based CF Hadoop job
Modified:
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
lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/UserVectorToCooccurrenceReducer.java
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=889995&r1=889994&r2=889995&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 Sat Dec 12 22:50:02 2009
@@ -27,8 +27,11 @@
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.hadoop.MapFilesMap;
import org.apache.mahout.cf.taste.hadoop.RecommendedItemsWritable;
+import org.apache.mahout.cf.taste.impl.common.Cache;
+import org.apache.mahout.cf.taste.impl.common.Retriever;
import org.apache.mahout.cf.taste.impl.recommender.GenericRecommendedItem;
import org.apache.mahout.cf.taste.recommender.RecommendedItem;
import org.apache.mahout.matrix.SparseVector;
@@ -53,6 +56,7 @@
private int recommendationsPerUser;
private MapFilesMap<IntWritable,LongWritable> indexItemIDMap;
private MapFilesMap<IntWritable,Vector> cooccurrenceColumnMap;
+ private Cache<IntWritable,Vector> cooccurrenceColumnCache;
@Override
public void configure(JobConf jobConf) {
@@ -66,6 +70,7 @@
} catch (IOException ioe) {
throw new IllegalStateException(ioe);
}
+ cooccurrenceColumnCache = new Cache<IntWritable,Vector>(new CooccurrenceCache(cooccurrenceColumnMap), 100);
}
@Override
@@ -76,12 +81,20 @@
Iterator<Vector.Element> userVectorIterator = userVector.iterateNonZero();
Vector recommendationVector = new SparseVector(Integer.MAX_VALUE, 1000);
- Vector columnVector = new SparseVector(Integer.MAX_VALUE, 1000);
while (userVectorIterator.hasNext()) {
Vector.Element element = userVectorIterator.next();
int index = element.index();
double value = element.get();
- cooccurrenceColumnMap.get(new IntWritable(index), columnVector);
+ Vector columnVector;
+ try {
+ columnVector = cooccurrenceColumnCache.get(new IntWritable(index));
+ } catch (TasteException te) {
+ if (te.getCause() instanceof IOException) {
+ throw (IOException) te.getCause();
+ } else {
+ throw new IOException(te.getCause());
+ }
+ }
columnVector.times(value).addTo(recommendationVector);
}
@@ -92,13 +105,16 @@
LongWritable itemID = new LongWritable();
while (recommendationVectorIterator.hasNext()) {
Vector.Element element = recommendationVectorIterator.next();
- if (topItems.size() < recommendationsPerUser) {
- indexItemIDMap.get(new IntWritable(element.index()), itemID);
- topItems.add(new GenericRecommendedItem(itemID.get(), (float) element.get()));
- } else if (element.get() > topItems.peek().getValue()) {
- indexItemIDMap.get(new IntWritable(element.index()), itemID);
- topItems.add(new GenericRecommendedItem(itemID.get(), (float) element.get()));
- topItems.poll();
+ int index = element.index();
+ if (userVector.get(index) != 0.0) {
+ if (topItems.size() < recommendationsPerUser) {
+ indexItemIDMap.get(new IntWritable(index), itemID);
+ topItems.add(new GenericRecommendedItem(itemID.get(), (float) element.get()));
+ } else if (element.get() > topItems.peek().getValue()) {
+ indexItemIDMap.get(new IntWritable(index), itemID);
+ topItems.add(new GenericRecommendedItem(itemID.get(), (float) element.get()));
+ topItems.poll();
+ }
}
}
@@ -114,4 +130,30 @@
cooccurrenceColumnMap.close();
}
+ private static class CooccurrenceCache implements Retriever<IntWritable,Vector> {
+
+ private final MapFilesMap<IntWritable,Vector> map;
+ private Vector columnVector;
+
+ private CooccurrenceCache(MapFilesMap<IntWritable,Vector> map) {
+ this.map = map;
+ columnVector = new SparseVector(Integer.MAX_VALUE, 1000);
+ }
+
+ @Override
+ public Vector get(IntWritable key) throws TasteException {
+ Vector value;
+ try {
+ value = map.get(key, columnVector);
+ } catch (IOException ioe) {
+ throw new TasteException(ioe);
+ }
+ if (value == null) {
+ return null;
+ }
+ columnVector = new SparseVector(Integer.MAX_VALUE, 1000);
+ return value;
+ }
+
+ }
}
\ No newline at end of file
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=889995&r1=889994&r2=889995&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 Sat Dec 12 22:50:02 2009
@@ -53,7 +53,7 @@
extends MapReduceBase
implements Reducer<LongWritable, ItemPrefWritable, LongWritable, SparseVector> {
- public static final int MAX_PREFS_CONSIDERED = 50;
+ public static final int MAX_PREFS_CONSIDERED = 20;
@Override
public void reduce(LongWritable userID,
Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/UserVectorToCooccurrenceReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/UserVectorToCooccurrenceReducer.java?rev=889995&r1=889994&r2=889995&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/UserVectorToCooccurrenceReducer.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/UserVectorToCooccurrenceReducer.java Sat Dec 12 22:50:02 2009
@@ -23,6 +23,7 @@
import org.apache.hadoop.mapred.Reducer;
import org.apache.hadoop.mapred.Reporter;
import org.apache.mahout.matrix.SparseVector;
+import org.apache.mahout.matrix.Vector;
import java.io.IOException;
import java.util.Iterator;
@@ -42,6 +43,13 @@
int index2 = index2s.next().get();
cooccurrenceRow.set(index2, cooccurrenceRow.get(index2) + 1.0);
}
+ Iterator<Vector.Element> cooccurrences = cooccurrenceRow.iterateNonZero();
+ while (cooccurrences.hasNext()) {
+ Vector.Element element = cooccurrences.next();
+ if (element.get() <= 1.0) { // purge small values
+ element.set(0.0);
+ }
+ }
output.collect(index1, cooccurrenceRow);
}
}