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/05 08:47:10 UTC

svn commit: r930806 - in /lucene/mahout/trunk/core/src: main/java/org/apache/mahout/cf/taste/hadoop/ main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/ main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/writables/ main/java/org/apach...

Author: srowen
Date: Mon Apr  5 06:47:09 2010
New Revision: 930806

URL: http://svn.apache.org/viewvc?rev=930806&view=rev
Log:
MAHOUT-362 More refinement of writables

Added:
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/EntityEntityWritable.java
      - copied, changed from r930802, 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/EntityPrefWritableArrayWritable.java
      - copied, changed from r930805, lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/writables/UserPrefArrayWritable.java
Removed:
    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/similarity/item/writables/UserPrefArrayWritable.java
Modified:
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/CosineSimilarityReducer.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/ItemSimilarityJob.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/PreferredItemsPerUserMapper.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/ToItemVectorReducer.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/writables/ItemPairWritable.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/writables/ItemPrefWithLengthArrayWritable.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/writables/ItemPrefWithLengthWritable.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/slopeone/SlopeOneAverageDiffsJob.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/slopeone/SlopeOneDiffsToAveragesReducer.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/slopeone/SlopeOnePrefsToDiffsReducer.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/hadoop/similarity/item/ItemSimilarityTest.java

Copied: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/EntityEntityWritable.java (from r930802, 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/EntityEntityWritable.java?p2=lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/EntityEntityWritable.java&p1=lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemItemWritable.java&r1=930802&r2=930806&rev=930806&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/ItemItemWritable.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/EntityEntityWritable.java Mon Apr  5 06:47:09 2010
@@ -25,50 +25,50 @@ import org.apache.hadoop.io.WritableComp
 import org.apache.mahout.common.RandomUtils;
 
 /** A {@link WritableComparable} encapsulating two items. */
-public final class ItemItemWritable implements WritableComparable<ItemItemWritable> {
+public final class EntityEntityWritable implements WritableComparable<EntityEntityWritable> {
   
-  private long itemAID;
-  private long itemBID;
+  private long aID;
+  private long bID;
   
-  public ItemItemWritable() {
+  public EntityEntityWritable() {
   // do nothing
   }
   
-  public ItemItemWritable(long itemAID, long itemBID) {
-    this.itemAID = itemAID;
-    this.itemBID = itemBID;
+  public EntityEntityWritable(long aID, long bID) {
+    this.aID = aID;
+    this.bID = bID;
   }
   
-  public long getItemAID() {
-    return itemAID;
+  public long getAID() {
+    return aID;
   }
   
-  public long getItemBID() {
-    return itemBID;
+  public long getBID() {
+    return bID;
   }
   
   @Override
   public void write(DataOutput out) throws IOException {
-    out.writeLong(itemAID);
-    out.writeLong(itemBID);
+    out.writeLong(aID);
+    out.writeLong(bID);
   }
   
   @Override
   public void readFields(DataInput in) throws IOException {
-    itemAID = in.readLong();
-    itemBID = in.readLong();
+    aID = in.readLong();
+    bID = in.readLong();
   }
   
-  public static ItemItemWritable read(DataInput in) throws IOException {
-    ItemItemWritable writable = new ItemItemWritable();
+  public static EntityEntityWritable read(DataInput in) throws IOException {
+    EntityEntityWritable writable = new EntityEntityWritable();
     writable.readFields(in);
     return writable;
   }
   
   @Override
-  public int compareTo(ItemItemWritable that) {
-    int aCompare = compare(itemAID, that.getItemAID());
-    return aCompare == 0 ? compare(itemBID, that.getItemBID()) : aCompare;
+  public int compareTo(EntityEntityWritable that) {
+    int aCompare = compare(aID, that.getAID());
+    return aCompare == 0 ? compare(bID, that.getBID()) : aCompare;
   }
   
   private static int compare(long a, long b) {
@@ -77,21 +77,21 @@ public final class ItemItemWritable impl
   
   @Override
   public int hashCode() {
-    return RandomUtils.hashLong(itemAID) + 31 * RandomUtils.hashLong(itemBID);
+    return RandomUtils.hashLong(aID) + 31 * RandomUtils.hashLong(bID);
   }
   
   @Override
   public boolean equals(Object o) {
-    if (o instanceof ItemItemWritable) {
-      ItemItemWritable that = (ItemItemWritable) o;
-      return (itemAID == that.getItemAID()) && (itemBID == that.getItemBID());
+    if (o instanceof EntityEntityWritable) {
+      EntityEntityWritable that = (EntityEntityWritable) o;
+      return (aID == that.getAID()) && (bID == that.getBID());
     }
     return false;
   }
   
   @Override
   public String toString() {
-    return itemAID + "\t" + itemBID;
+    return aID + "\t" + bID;
   }
   
 }
\ No newline at end of file

Copied: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/EntityPrefWritableArrayWritable.java (from r930805, lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/writables/UserPrefArrayWritable.java)
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/EntityPrefWritableArrayWritable.java?p2=lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/EntityPrefWritableArrayWritable.java&p1=lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/writables/UserPrefArrayWritable.java&r1=930805&r2=930806&rev=930806&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/writables/UserPrefArrayWritable.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/EntityPrefWritableArrayWritable.java Mon Apr  5 06:47:09 2010
@@ -15,27 +15,24 @@
  * limitations under the License.
  */
 
-package org.apache.mahout.cf.taste.hadoop.similarity.item.writables;
+package org.apache.mahout.cf.taste.hadoop;
 
 import org.apache.hadoop.io.ArrayWritable;
-import org.apache.mahout.cf.taste.hadoop.EntityPrefWritable;
 
 /**
  * An {@link ArrayWritable} holding {@link EntityPrefWritable}s
- *
- * Used to represent an item-vector
  */
-public final class UserPrefArrayWritable extends ArrayWritable {
+public class EntityPrefWritableArrayWritable extends ArrayWritable {
 
-  public UserPrefArrayWritable() {
+  public EntityPrefWritableArrayWritable() {
     super(EntityPrefWritable.class);
   }
 
-  public UserPrefArrayWritable(EntityPrefWritable[] userPrefs) {
-    super(EntityPrefWritable.class, userPrefs);
+  public EntityPrefWritableArrayWritable(EntityPrefWritable[] prefs) {
+    super(EntityPrefWritable.class, prefs);
   }
 
-  public EntityPrefWritable[] getUserPrefs() {
+  public EntityPrefWritable[] getPrefs() {
     return (EntityPrefWritable[]) toArray();
   }
 }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/CosineSimilarityReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/CosineSimilarityReducer.java?rev=930806&r1=930805&r2=930806&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/CosineSimilarityReducer.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/CosineSimilarityReducer.java Mon Apr  5 06:47:09 2010
@@ -22,14 +22,14 @@ import java.io.IOException;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.mahout.cf.taste.hadoop.ItemItemWritable;
+import org.apache.mahout.cf.taste.hadoop.EntityEntityWritable;
 import org.apache.mahout.cf.taste.hadoop.similarity.item.writables.ItemPairWritable;
 
 /**
  * Finally compute the cosine for each item-pair
  */
 public final class CosineSimilarityReducer
-    extends Reducer<ItemPairWritable,FloatWritable,ItemItemWritable,DoubleWritable> {
+    extends Reducer<ItemPairWritable,FloatWritable, EntityEntityWritable,DoubleWritable> {
 
   @Override
   protected void reduce(ItemPairWritable pair, Iterable<FloatWritable> numeratorSummands, Context context)

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/ItemSimilarityJob.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/ItemSimilarityJob.java?rev=930806&r1=930805&r2=930806&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/ItemSimilarityJob.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/ItemSimilarityJob.java Mon Apr  5 06:47:09 2010
@@ -37,12 +37,12 @@ import org.apache.hadoop.mapreduce.lib.o
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.mahout.cf.taste.hadoop.EntityPrefWritable;
+import org.apache.mahout.cf.taste.hadoop.EntityPrefWritableArrayWritable;
 import org.apache.mahout.cf.taste.hadoop.EntityWritable;
-import org.apache.mahout.cf.taste.hadoop.ItemItemWritable;
+import org.apache.mahout.cf.taste.hadoop.EntityEntityWritable;
 import org.apache.mahout.cf.taste.hadoop.similarity.item.writables.ItemPairWritable;
 import org.apache.mahout.cf.taste.hadoop.similarity.item.writables.ItemPrefWithLengthArrayWritable;
 import org.apache.mahout.cf.taste.hadoop.similarity.item.writables.ItemPrefWithLengthWritable;
-import org.apache.mahout.cf.taste.hadoop.similarity.item.writables.UserPrefArrayWritable;
 import org.apache.mahout.common.AbstractJob;
 
 /**
@@ -123,7 +123,7 @@ public final class ItemSimilarityJob ext
 
     Job itemVectors = createJob(originalConf, "itemVectors", inputPath, itemVectorsPath, UserPrefsPerItemMapper.class,
         EntityWritable.class, EntityPrefWritable.class, ToItemVectorReducer.class, EntityWritable.class,
-        UserPrefArrayWritable.class, TextInputFormat.class, SequenceFileOutputFormat.class, true);
+        EntityPrefWritableArrayWritable.class, TextInputFormat.class, SequenceFileOutputFormat.class, true);
 
     itemVectors.waitForCompletion(true);
 
@@ -135,7 +135,7 @@ public final class ItemSimilarityJob ext
 
     Job similarity = createJob(originalConf, "similarity", userVectorsPath, outputPath,
         CopreferredItemsMapper.class, ItemPairWritable.class, FloatWritable.class, CosineSimilarityReducer.class,
-        ItemItemWritable.class, DoubleWritable.class, SequenceFileInputFormat.class, TextOutputFormat.class, false);
+        EntityEntityWritable.class, DoubleWritable.class, SequenceFileInputFormat.class, TextOutputFormat.class, false);
 
     similarity.waitForCompletion(true);
 

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/PreferredItemsPerUserMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/PreferredItemsPerUserMapper.java?rev=930806&r1=930805&r2=930806&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/PreferredItemsPerUserMapper.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/PreferredItemsPerUserMapper.java Mon Apr  5 06:47:09 2010
@@ -21,22 +21,22 @@ import java.io.IOException;
 
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.mahout.cf.taste.hadoop.EntityPrefWritable;
+import org.apache.mahout.cf.taste.hadoop.EntityPrefWritableArrayWritable;
 import org.apache.mahout.cf.taste.hadoop.EntityWritable;
 import org.apache.mahout.cf.taste.hadoop.similarity.item.writables.ItemPrefWithLengthWritable;
-import org.apache.mahout.cf.taste.hadoop.similarity.item.writables.UserPrefArrayWritable;
 
 /**
  * for each item-vector, we compute its length here and map out all entries with the user as key,
  * so we can create the user-vectors in the reducer
  */
 public final class PreferredItemsPerUserMapper
-    extends Mapper<EntityWritable,UserPrefArrayWritable,EntityWritable,ItemPrefWithLengthWritable> {
+    extends Mapper<EntityWritable, EntityPrefWritableArrayWritable,EntityWritable,ItemPrefWithLengthWritable> {
 
   @Override
-  protected void map(EntityWritable item, UserPrefArrayWritable userPrefsArray, Context context)
+  protected void map(EntityWritable item, EntityPrefWritableArrayWritable userPrefsArray, Context context)
       throws IOException, InterruptedException {
 
-    EntityPrefWritable[] userPrefs = userPrefsArray.getUserPrefs();
+    EntityPrefWritable[] userPrefs = userPrefsArray.getPrefs();
 
     double length = 0.0;
     for (EntityPrefWritable userPref : userPrefs) {

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/ToItemVectorReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/ToItemVectorReducer.java?rev=930806&r1=930805&r2=930806&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/ToItemVectorReducer.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/ToItemVectorReducer.java Mon Apr  5 06:47:09 2010
@@ -23,15 +23,15 @@ import java.util.Set;
 
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.mahout.cf.taste.hadoop.EntityPrefWritable;
+import org.apache.mahout.cf.taste.hadoop.EntityPrefWritableArrayWritable;
 import org.apache.mahout.cf.taste.hadoop.EntityWritable;
-import org.apache.mahout.cf.taste.hadoop.similarity.item.writables.UserPrefArrayWritable;
 
 /**
  * For each single item, collect all users with their preferences
  * (thereby building the item vectors of the user-item-matrix)
  */
 public final class ToItemVectorReducer
-    extends Reducer<EntityWritable, EntityPrefWritable, EntityWritable,UserPrefArrayWritable> {
+    extends Reducer<EntityWritable, EntityPrefWritable, EntityWritable, EntityPrefWritableArrayWritable> {
 
   @Override
   protected void reduce(EntityWritable item, Iterable<EntityPrefWritable> userPrefs, Context context)
@@ -43,7 +43,7 @@ public final class ToItemVectorReducer
       collectedUserPrefs.add(userPref.clone());
     }
 
-    context.write(item, new UserPrefArrayWritable(
+    context.write(item, new EntityPrefWritableArrayWritable(
         collectedUserPrefs.toArray(new EntityPrefWritable[collectedUserPrefs.size()])));
   }
 

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/writables/ItemPairWritable.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/writables/ItemPairWritable.java?rev=930806&r1=930805&r2=930806&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/writables/ItemPairWritable.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/writables/ItemPairWritable.java Mon Apr  5 06:47:09 2010
@@ -22,7 +22,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 
 import org.apache.hadoop.io.WritableComparable;
-import org.apache.mahout.cf.taste.hadoop.ItemItemWritable;
+import org.apache.mahout.cf.taste.hadoop.EntityEntityWritable;
 
 /**
  * A {@link WritableComparable} encapsulating two items together with their
@@ -30,26 +30,26 @@ import org.apache.mahout.cf.taste.hadoop
  */
 public final class ItemPairWritable implements WritableComparable<ItemPairWritable> {
 
-  private ItemItemWritable itemItemWritable;
+  private EntityEntityWritable itemItemWritable;
   private double multipliedLength;
 
   public ItemPairWritable() {
   }
 
   public ItemPairWritable(long itemAID, long itemBID, double multipliedLength) {
-    this.itemItemWritable = new ItemItemWritable(itemAID, itemBID);
+    this.itemItemWritable = new EntityEntityWritable(itemAID, itemBID);
     this.multipliedLength = multipliedLength;
   }
 
   public long getItemAID() {
-    return itemItemWritable.getItemAID();
+    return itemItemWritable.getAID();
   }
 
   public long getItemBID() {
-    return itemItemWritable.getItemBID();
+    return itemItemWritable.getBID();
   }
 
-  public ItemItemWritable getItemItemWritable() {
+  public EntityEntityWritable getItemItemWritable() {
     return itemItemWritable;
   }
 
@@ -59,7 +59,7 @@ public final class ItemPairWritable impl
 
   @Override
   public void readFields(DataInput in) throws IOException {
-    itemItemWritable = ItemItemWritable.read(in);
+    itemItemWritable = EntityEntityWritable.read(in);
     multipliedLength = in.readDouble();
   }
 

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/writables/ItemPrefWithLengthArrayWritable.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/writables/ItemPrefWithLengthArrayWritable.java?rev=930806&r1=930805&r2=930806&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/writables/ItemPrefWithLengthArrayWritable.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/writables/ItemPrefWithLengthArrayWritable.java Mon Apr  5 06:47:09 2010
@@ -24,7 +24,7 @@ import org.apache.hadoop.io.ArrayWritabl
  *
  * Used as user-vector
  */
-public final class ItemPrefWithLengthArrayWritable extends ArrayWritable {
+public class ItemPrefWithLengthArrayWritable extends ArrayWritable {
 
   public ItemPrefWithLengthArrayWritable() {
     super(ItemPrefWithLengthWritable.class);
@@ -37,4 +37,5 @@ public final class ItemPrefWithLengthArr
   public ItemPrefWithLengthWritable[] getItemPrefs() {
     return (ItemPrefWithLengthWritable[]) toArray();
   }
+
 }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/writables/ItemPrefWithLengthWritable.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/writables/ItemPrefWithLengthWritable.java?rev=930806&r1=930805&r2=930806&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/writables/ItemPrefWithLengthWritable.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/similarity/item/writables/ItemPrefWithLengthWritable.java Mon Apr  5 06:47:09 2010
@@ -22,6 +22,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 
 import org.apache.hadoop.io.Writable;
+import org.apache.mahout.cf.taste.hadoop.EntityPrefWritable;
 import org.apache.mahout.common.RandomUtils;
 
 /**
@@ -29,24 +30,22 @@ import org.apache.mahout.common.RandomUt
  * stored along with the length of the item-vector
  *
  */
-public final class ItemPrefWithLengthWritable implements Writable {
+public final class ItemPrefWithLengthWritable implements Writable, Cloneable {
 
-  private long itemID;
+  private EntityPrefWritable itemPref;
   private double length;
-  private float prefValue;
 
   public ItemPrefWithLengthWritable() {
   // do nothing
   }
 
   public ItemPrefWithLengthWritable(long itemID, double length, float prefValue) {
-    this.itemID = itemID;
+    this.itemPref = new EntityPrefWritable(itemID, prefValue);
     this.length = length;
-    this.prefValue = prefValue;
   }
 
   public long getItemID() {
-    return itemID;
+    return itemPref.getID();
   }
 
   public double getLength() {
@@ -54,39 +53,38 @@ public final class ItemPrefWithLengthWri
   }
 
   public float getPrefValue() {
-    return prefValue;
-  }
-
-  public ItemPrefWithLengthWritable deepCopy() {
-    return new ItemPrefWithLengthWritable(itemID, length, prefValue);
+    return itemPref.getPrefValue();
   }
 
   @Override
   public void write(DataOutput out) throws IOException {
-    out.writeLong(itemID);
+    itemPref.write(out);
     out.writeDouble(length);
-    out.writeFloat(prefValue);
   }
 
   @Override
   public void readFields(DataInput in) throws IOException {
-    itemID = in.readLong();
+    itemPref = EntityPrefWritable.read(in);
     length = in.readDouble();
-    prefValue = in.readFloat();
   }
 
   @Override
   public int hashCode() {
-    return RandomUtils.hashLong(itemID) + 31 * RandomUtils.hashDouble(length) + 31 * RandomUtils.hashFloat(prefValue);
+    return itemPref.hashCode() + 31 * RandomUtils.hashDouble(length);
   }
 
   @Override
   public boolean equals(Object o) {
     if (o instanceof ItemPrefWithLengthWritable) {
       ItemPrefWithLengthWritable other = (ItemPrefWithLengthWritable) o;
-      return (itemID == other.getItemID() && length == other.getLength() && prefValue == other.getPrefValue());
+      return itemPref.equals(other.itemPref) && length == other.getLength();
     }
     return false;
   }
 
+  @Override
+  public ItemPrefWithLengthWritable clone() {
+    return new ItemPrefWithLengthWritable(itemPref.getID(), length, itemPref.getPrefValue());
+  }
+
 }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/slopeone/SlopeOneAverageDiffsJob.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/slopeone/SlopeOneAverageDiffsJob.java?rev=930806&r1=930805&r2=930806&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/slopeone/SlopeOneAverageDiffsJob.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/slopeone/SlopeOneAverageDiffsJob.java Mon Apr  5 06:47:09 2010
@@ -33,8 +33,8 @@ import org.apache.hadoop.mapred.TextInpu
 import org.apache.hadoop.mapred.TextOutputFormat;
 import org.apache.hadoop.mapred.lib.IdentityMapper;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.mahout.cf.taste.hadoop.EntityEntityWritable;
 import org.apache.mahout.common.AbstractJob;
-import org.apache.mahout.cf.taste.hadoop.ItemItemWritable;
 import org.apache.mahout.cf.taste.hadoop.EntityPrefWritable;
 import org.apache.mahout.cf.taste.hadoop.ToItemPrefsMapper;
 
@@ -55,13 +55,13 @@ public final class SlopeOneAverageDiffsJ
     
     JobConf prefsToDiffsJobConf = prepareJobConf(prefsFile, averagesOutputPath,
       TextInputFormat.class, ToItemPrefsMapper.class, LongWritable.class, EntityPrefWritable.class,
-      SlopeOnePrefsToDiffsReducer.class, ItemItemWritable.class, FloatWritable.class,
+      SlopeOnePrefsToDiffsReducer.class, EntityEntityWritable.class, FloatWritable.class,
       SequenceFileOutputFormat.class);
     JobClient.runJob(prefsToDiffsJobConf);
     
     JobConf diffsToAveragesJobConf = prepareJobConf(averagesOutputPath, outputPath,
-      SequenceFileInputFormat.class, IdentityMapper.class, ItemItemWritable.class, FloatWritable.class,
-      SlopeOneDiffsToAveragesReducer.class, ItemItemWritable.class, FloatWritable.class,
+      SequenceFileInputFormat.class, IdentityMapper.class, EntityEntityWritable.class, FloatWritable.class,
+      SlopeOneDiffsToAveragesReducer.class, EntityEntityWritable.class, FloatWritable.class,
       TextOutputFormat.class);
     diffsToAveragesJobConf.setClass("mapred.output.compression.codec", GzipCodec.class,
       CompressionCodec.class);

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/slopeone/SlopeOneDiffsToAveragesReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/slopeone/SlopeOneDiffsToAveragesReducer.java?rev=930806&r1=930805&r2=930806&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/slopeone/SlopeOneDiffsToAveragesReducer.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/slopeone/SlopeOneDiffsToAveragesReducer.java Mon Apr  5 06:47:09 2010
@@ -25,15 +25,15 @@ import org.apache.hadoop.mapred.MapReduc
 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.ItemItemWritable;
+import org.apache.mahout.cf.taste.hadoop.EntityEntityWritable;
 
 public final class SlopeOneDiffsToAveragesReducer extends MapReduceBase implements
-    Reducer<ItemItemWritable,FloatWritable,ItemItemWritable,FloatWritable> {
+    Reducer<EntityEntityWritable,FloatWritable, EntityEntityWritable,FloatWritable> {
   
   @Override
-  public void reduce(ItemItemWritable key,
+  public void reduce(EntityEntityWritable key,
                      Iterator<FloatWritable> values,
-                     OutputCollector<ItemItemWritable,FloatWritable> output,
+                     OutputCollector<EntityEntityWritable,FloatWritable> output,
                      Reporter reporter) throws IOException {
     int count = 0;
     double total = 0.0;

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/slopeone/SlopeOnePrefsToDiffsReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/slopeone/SlopeOnePrefsToDiffsReducer.java?rev=930806&r1=930805&r2=930806&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/slopeone/SlopeOnePrefsToDiffsReducer.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/slopeone/SlopeOnePrefsToDiffsReducer.java Mon Apr  5 06:47:09 2010
@@ -29,16 +29,16 @@ import org.apache.hadoop.mapred.MapReduc
 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.EntityEntityWritable;
 import org.apache.mahout.cf.taste.hadoop.EntityPrefWritable;
-import org.apache.mahout.cf.taste.hadoop.ItemItemWritable;
 
 public final class SlopeOnePrefsToDiffsReducer extends MapReduceBase implements
-    Reducer<LongWritable, EntityPrefWritable,ItemItemWritable,FloatWritable> {
+    Reducer<LongWritable, EntityPrefWritable, EntityEntityWritable,FloatWritable> {
   
   @Override
   public void reduce(LongWritable key,
                      Iterator<EntityPrefWritable> values,
-                     OutputCollector<ItemItemWritable,FloatWritable> output,
+                     OutputCollector<EntityEntityWritable,FloatWritable> output,
                      Reporter reporter) throws IOException {
     List<EntityPrefWritable> prefs = new ArrayList<EntityPrefWritable>();
     while (values.hasNext()) {
@@ -54,7 +54,7 @@ public final class SlopeOnePrefsToDiffsR
         EntityPrefWritable second = prefs.get(j);
         long itemBID = second.getID();
         float itemBValue = second.getPrefValue();
-        output.collect(new ItemItemWritable(itemAID, itemBID), new FloatWritable(itemBValue - itemAValue));
+        output.collect(new EntityEntityWritable(itemAID, itemBID), new FloatWritable(itemBValue - itemAValue));
       }
     }
   }

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/hadoop/similarity/item/ItemSimilarityTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/hadoop/similarity/item/ItemSimilarityTest.java?rev=930806&r1=930805&r2=930806&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/hadoop/similarity/item/ItemSimilarityTest.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/hadoop/similarity/item/ItemSimilarityTest.java Mon Apr  5 06:47:09 2010
@@ -42,12 +42,12 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.mahout.cf.taste.hadoop.EntityPrefWritable;
+import org.apache.mahout.cf.taste.hadoop.EntityPrefWritableArrayWritable;
 import org.apache.mahout.cf.taste.hadoop.EntityWritable;
-import org.apache.mahout.cf.taste.hadoop.ItemItemWritable;
+import org.apache.mahout.cf.taste.hadoop.EntityEntityWritable;
 import org.apache.mahout.cf.taste.hadoop.similarity.item.writables.ItemPairWritable;
 import org.apache.mahout.cf.taste.hadoop.similarity.item.writables.ItemPrefWithLengthArrayWritable;
 import org.apache.mahout.cf.taste.hadoop.similarity.item.writables.ItemPrefWithLengthWritable;
-import org.apache.mahout.cf.taste.hadoop.similarity.item.writables.UserPrefArrayWritable;
 import org.apache.mahout.common.MahoutTestCase;
 import org.easymock.IArgumentMatcher;
 import org.easymock.classextension.EasyMock;
@@ -85,14 +85,14 @@ public class ItemSimilarityTest extends 
     verify(ctx);
   }
 
-  static UserPrefArrayWritable equalToUserPrefs(final Collection<EntityPrefWritable> prefsToCheck) {
+  static EntityPrefWritableArrayWritable equalToUserPrefs(final Collection<EntityPrefWritable> prefsToCheck) {
     EasyMock.reportMatcher(new IArgumentMatcher() {
       @Override
       public boolean matches(Object argument) {
-        if (argument instanceof UserPrefArrayWritable) {
-          UserPrefArrayWritable userPrefArray = (UserPrefArrayWritable) argument;
+        if (argument instanceof EntityPrefWritableArrayWritable) {
+          EntityPrefWritableArrayWritable userPrefArray = (EntityPrefWritableArrayWritable) argument;
           Set<EntityPrefWritable> set = new HashSet<EntityPrefWritable>();
-          set.addAll(Arrays.asList(userPrefArray.getUserPrefs()));
+          set.addAll(Arrays.asList(userPrefArray.getPrefs()));
 
           if (set.size() != prefsToCheck.size()) {
             return false;
@@ -117,9 +117,9 @@ public class ItemSimilarityTest extends 
 
   public void testPreferredItemsPerUserMapper() throws Exception {
     Mapper.Context ctx = createMock(Mapper.Context.class);
-    UserPrefArrayWritable userPrefs = createMock(UserPrefArrayWritable.class);
+    EntityPrefWritableArrayWritable userPrefs = createMock(EntityPrefWritableArrayWritable.class);
 
-    expect(userPrefs.getUserPrefs())
+    expect(userPrefs.getPrefs())
         .andReturn(new EntityPrefWritable[] { new EntityPrefWritable(12L, 2.0f), new EntityPrefWritable(56L, 3.0f) });
 
     double length = Math.sqrt(Math.pow(2.0f, 2) + Math.pow(3.0f, 2));
@@ -204,7 +204,7 @@ public class ItemSimilarityTest extends 
   public void testCosineSimilarityReducer() throws Exception {
     Reducer.Context ctx = createMock(Reducer.Context.class);
 
-    ctx.write(new ItemItemWritable(12L, 34L), new DoubleWritable(0.5d));
+    ctx.write(new EntityEntityWritable(12L, 34L), new DoubleWritable(0.5d));
 
     replay(ctx);