You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mahout.apache.org by ra...@apache.org on 2018/06/28 14:54:48 UTC

[20/51] [partial] mahout git commit: NO-JIRA Clean up MR refactor

http://git-wip-us.apache.org/repos/asf/mahout/blob/410ed16a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/AbstractRecommender.java
----------------------------------------------------------------------
diff --git a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/AbstractRecommender.java b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/AbstractRecommender.java
new file mode 100644
index 0000000..3a62b08
--- /dev/null
+++ b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/AbstractRecommender.java
@@ -0,0 +1,140 @@
+/**
+ * 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.impl.recommender;
+
+import org.apache.mahout.cf.taste.model.PreferenceArray;
+import org.apache.mahout.cf.taste.recommender.CandidateItemsStrategy;
+
+import java.util.List;
+
+import org.apache.mahout.cf.taste.common.TasteException;
+import org.apache.mahout.cf.taste.impl.common.FastIDSet;
+import org.apache.mahout.cf.taste.model.DataModel;
+import org.apache.mahout.cf.taste.recommender.IDRescorer;
+import org.apache.mahout.cf.taste.recommender.RecommendedItem;
+import org.apache.mahout.cf.taste.recommender.Recommender;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+public abstract class AbstractRecommender implements Recommender {
+  
+  private static final Logger log = LoggerFactory.getLogger(AbstractRecommender.class);
+  
+  private final DataModel dataModel;
+  private final CandidateItemsStrategy candidateItemsStrategy;
+  
+  protected AbstractRecommender(DataModel dataModel, CandidateItemsStrategy candidateItemsStrategy) {
+    this.dataModel = Preconditions.checkNotNull(dataModel);
+    this.candidateItemsStrategy = Preconditions.checkNotNull(candidateItemsStrategy);
+  }
+
+  protected AbstractRecommender(DataModel dataModel) {
+    this(dataModel, getDefaultCandidateItemsStrategy());
+  }
+
+  protected static CandidateItemsStrategy getDefaultCandidateItemsStrategy() {
+    return new PreferredItemsNeighborhoodCandidateItemsStrategy();
+  }
+
+
+  /**
+   * <p>
+   * Default implementation which just calls
+   * {@link Recommender#recommend(long, int, org.apache.mahout.cf.taste.recommender.IDRescorer)}, with a
+   * {@link org.apache.mahout.cf.taste.recommender.Rescorer} that does nothing.
+   * </p>
+   */
+  @Override
+  public List<RecommendedItem> recommend(long userID, int howMany) throws TasteException {
+    return recommend(userID, howMany, null, false);
+  }
+
+  /**
+   * <p>
+   * Default implementation which just calls
+   * {@link Recommender#recommend(long, int, org.apache.mahout.cf.taste.recommender.IDRescorer)}, with a
+   * {@link org.apache.mahout.cf.taste.recommender.Rescorer} that does nothing.
+   * </p>
+   */
+  @Override
+  public List<RecommendedItem> recommend(long userID, int howMany, boolean includeKnownItems) throws TasteException {
+    return recommend(userID, howMany, null, includeKnownItems);
+  }
+  
+  /**
+   * <p> Delegates to {@link Recommender#recommend(long, int, IDRescorer, boolean)}
+   */
+  @Override
+  public List<RecommendedItem> recommend(long userID, int howMany, IDRescorer rescorer) throws TasteException{
+    return recommend(userID, howMany,rescorer, false);  
+  }
+  
+  /**
+   * <p>
+   * Default implementation which just calls {@link DataModel#setPreference(long, long, float)}.
+   * </p>
+   *
+   * @throws IllegalArgumentException
+   *           if userID or itemID is {@code null}, or if value is {@link Double#NaN}
+   */
+  @Override
+  public void setPreference(long userID, long itemID, float value) throws TasteException {
+    Preconditions.checkArgument(!Float.isNaN(value), "NaN value");
+    log.debug("Setting preference for user {}, item {}", userID, itemID);
+    dataModel.setPreference(userID, itemID, value);
+  }
+  
+  /**
+   * <p>
+   * Default implementation which just calls {@link DataModel#removePreference(long, long)} (Object, Object)}.
+   * </p>
+   *
+   * @throws IllegalArgumentException
+   *           if userID or itemID is {@code null}
+   */
+  @Override
+  public void removePreference(long userID, long itemID) throws TasteException {
+    log.debug("Remove preference for user '{}', item '{}'", userID, itemID);
+    dataModel.removePreference(userID, itemID);
+  }
+  
+  @Override
+  public DataModel getDataModel() {
+    return dataModel;
+  }
+
+  /**
+   * @param userID
+   *          ID of user being evaluated
+   * @param preferencesFromUser
+   *          the preferences from the user
+   * @param includeKnownItems
+   *          whether to include items already known by the user in recommendations
+   * @return all items in the {@link DataModel} for which the user has not expressed a preference and could
+   *         possibly be recommended to the user
+   * @throws TasteException
+   *           if an error occurs while listing items
+   */
+  protected FastIDSet getAllOtherItems(long userID, PreferenceArray preferencesFromUser, boolean includeKnownItems)
+      throws TasteException {
+    return candidateItemsStrategy.getCandidateItems(userID, preferencesFromUser, dataModel, includeKnownItems);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/mahout/blob/410ed16a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/AllSimilarItemsCandidateItemsStrategy.java
----------------------------------------------------------------------
diff --git a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/AllSimilarItemsCandidateItemsStrategy.java b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/AllSimilarItemsCandidateItemsStrategy.java
new file mode 100644
index 0000000..37389a7
--- /dev/null
+++ b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/AllSimilarItemsCandidateItemsStrategy.java
@@ -0,0 +1,50 @@
+/**
+ * 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.impl.recommender;
+
+import com.google.common.base.Preconditions;
+import org.apache.mahout.cf.taste.common.TasteException;
+import org.apache.mahout.cf.taste.impl.common.FastIDSet;
+import org.apache.mahout.cf.taste.model.DataModel;
+import org.apache.mahout.cf.taste.similarity.ItemSimilarity;
+
+/**
+ * returns the result of {@link ItemSimilarity#allSimilarItemIDs(long)} as candidate items
+ */
+public class AllSimilarItemsCandidateItemsStrategy extends AbstractCandidateItemsStrategy {
+
+  private final ItemSimilarity similarity;
+
+  public AllSimilarItemsCandidateItemsStrategy(ItemSimilarity similarity) {
+    Preconditions.checkArgument(similarity != null, "similarity is null");
+    this.similarity = similarity;
+  }
+
+  @Override
+  protected FastIDSet doGetCandidateItems(long[] preferredItemIDs, DataModel dataModel, boolean includeKnownItems)
+    throws TasteException {
+    FastIDSet candidateItemIDs = new FastIDSet();
+    for (long itemID : preferredItemIDs) {
+      candidateItemIDs.addAll(similarity.allSimilarItemIDs(itemID));
+    }
+    if (!includeKnownItems) {
+      candidateItemIDs.removeAll(preferredItemIDs);
+    }
+    return candidateItemIDs;
+  }
+}

http://git-wip-us.apache.org/repos/asf/mahout/blob/410ed16a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/AllUnknownItemsCandidateItemsStrategy.java
----------------------------------------------------------------------
diff --git a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/AllUnknownItemsCandidateItemsStrategy.java b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/AllUnknownItemsCandidateItemsStrategy.java
new file mode 100644
index 0000000..929eddd
--- /dev/null
+++ b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/AllUnknownItemsCandidateItemsStrategy.java
@@ -0,0 +1,41 @@
+/**
+ * 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.impl.recommender;
+
+import org.apache.mahout.cf.taste.common.TasteException;
+import org.apache.mahout.cf.taste.impl.common.FastIDSet;
+import org.apache.mahout.cf.taste.impl.common.LongPrimitiveIterator;
+import org.apache.mahout.cf.taste.model.DataModel;
+
+public final class AllUnknownItemsCandidateItemsStrategy extends AbstractCandidateItemsStrategy {
+
+  /** return all items the user has not yet seen */
+  @Override
+  protected FastIDSet doGetCandidateItems(long[] preferredItemIDs, DataModel dataModel, boolean includeKnownItems)
+    throws TasteException {
+    FastIDSet possibleItemIDs = new FastIDSet(dataModel.getNumItems());
+    LongPrimitiveIterator allItemIDs = dataModel.getItemIDs();
+    while (allItemIDs.hasNext()) {
+      possibleItemIDs.add(allItemIDs.nextLong());
+    }
+    if (!includeKnownItems) {
+      possibleItemIDs.removeAll(preferredItemIDs);
+    }
+    return possibleItemIDs;
+  }
+}

http://git-wip-us.apache.org/repos/asf/mahout/blob/410ed16a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/ByRescoreComparator.java
----------------------------------------------------------------------
diff --git a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/ByRescoreComparator.java b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/ByRescoreComparator.java
new file mode 100644
index 0000000..1677ea8
--- /dev/null
+++ b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/ByRescoreComparator.java
@@ -0,0 +1,65 @@
+/**
+ * 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.impl.recommender;
+
+import java.io.Serializable;
+import java.util.Comparator;
+
+import org.apache.mahout.cf.taste.recommender.IDRescorer;
+import org.apache.mahout.cf.taste.recommender.RecommendedItem;
+
+/**
+ * <p>
+ * Defines ordering on {@link RecommendedItem} by the rescored value of the recommendations' estimated
+ * preference value, from high to low.
+ * </p>
+ */
+final class ByRescoreComparator implements Comparator<RecommendedItem>, Serializable {
+  
+  private final IDRescorer rescorer;
+  
+  ByRescoreComparator(IDRescorer rescorer) {
+    this.rescorer = rescorer;
+  }
+  
+  @Override
+  public int compare(RecommendedItem o1, RecommendedItem o2) {
+    double rescored1;
+    double rescored2;
+    if (rescorer == null) {
+      rescored1 = o1.getValue();
+      rescored2 = o2.getValue();
+    } else {
+      rescored1 = rescorer.rescore(o1.getItemID(), o1.getValue());
+      rescored2 = rescorer.rescore(o2.getItemID(), o2.getValue());
+    }
+    if (rescored1 < rescored2) {
+      return 1;
+    } else if (rescored1 > rescored2) {
+      return -1;
+    } else {
+      return 0;
+    }
+  }
+  
+  @Override
+  public String toString() {
+    return "ByRescoreComparator[rescorer:" + rescorer + ']';
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/mahout/blob/410ed16a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/ByValueRecommendedItemComparator.java
----------------------------------------------------------------------
diff --git a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/ByValueRecommendedItemComparator.java b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/ByValueRecommendedItemComparator.java
new file mode 100644
index 0000000..57c5f3d
--- /dev/null
+++ b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/ByValueRecommendedItemComparator.java
@@ -0,0 +1,43 @@
+/**
+ * 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.impl.recommender;
+
+import java.io.Serializable;
+import java.util.Comparator;
+
+import org.apache.mahout.cf.taste.recommender.RecommendedItem;
+
+/**
+ * Defines a natural ordering from most-preferred item (highest value) to least-preferred.
+ */
+public final class ByValueRecommendedItemComparator implements Comparator<RecommendedItem>, Serializable {
+
+  private static final Comparator<RecommendedItem> INSTANCE = new ByValueRecommendedItemComparator();
+
+  public static Comparator<RecommendedItem> getInstance() {
+    return INSTANCE;
+  }
+
+  @Override
+  public int compare(RecommendedItem o1, RecommendedItem o2) {
+    float value1 = o1.getValue();
+    float value2 = o2.getValue();
+    return value1 > value2 ? -1 : value1 < value2 ? 1 : 0;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/mahout/blob/410ed16a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/CachingRecommender.java
----------------------------------------------------------------------
diff --git a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/CachingRecommender.java b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/CachingRecommender.java
new file mode 100644
index 0000000..7ed8cc3
--- /dev/null
+++ b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/CachingRecommender.java
@@ -0,0 +1,251 @@
+/**
+ * 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.impl.recommender;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+import org.apache.mahout.cf.taste.common.Refreshable;
+import org.apache.mahout.cf.taste.common.TasteException;
+import org.apache.mahout.cf.taste.impl.common.Cache;
+import org.apache.mahout.cf.taste.impl.common.RefreshHelper;
+import org.apache.mahout.cf.taste.impl.common.Retriever;
+import org.apache.mahout.cf.taste.impl.model.PlusAnonymousUserDataModel;
+import org.apache.mahout.cf.taste.model.DataModel;
+import org.apache.mahout.cf.taste.recommender.IDRescorer;
+import org.apache.mahout.cf.taste.recommender.RecommendedItem;
+import org.apache.mahout.cf.taste.recommender.Recommender;
+import org.apache.mahout.common.LongPair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * <p>
+ * A {@link Recommender} which caches the results from another {@link Recommender} in memory.
+ *
+ * TODO: Should be checked for thread safety
+ * </p>
+ */
+public final class CachingRecommender implements Recommender {
+  
+  private static final Logger log = LoggerFactory.getLogger(CachingRecommender.class);
+  
+  private final Recommender recommender;
+  private final int[] maxHowMany;
+  private final Retriever<Long,Recommendations> recommendationsRetriever;
+  private final Cache<Long,Recommendations> recommendationCache;
+  private final Cache<LongPair,Float> estimatedPrefCache;
+  private final RefreshHelper refreshHelper;
+  private IDRescorer currentRescorer;
+  private boolean currentlyIncludeKnownItems;
+  
+  public CachingRecommender(Recommender recommender) throws TasteException {
+    Preconditions.checkArgument(recommender != null, "recommender is null");
+    this.recommender = recommender;
+    maxHowMany = new int[]{1};
+    // Use "num users" as an upper limit on cache size. Rough guess.
+    int numUsers = recommender.getDataModel().getNumUsers();
+    recommendationsRetriever = new RecommendationRetriever();
+    recommendationCache = new Cache<>(recommendationsRetriever, numUsers);
+    estimatedPrefCache = new Cache<>(new EstimatedPrefRetriever(), numUsers);
+    refreshHelper = new RefreshHelper(new Callable<Object>() {
+      @Override
+      public Object call() {
+        clear();
+        return null;
+      }
+    });
+    refreshHelper.addDependency(recommender);
+  }
+  
+  private void setCurrentRescorer(IDRescorer rescorer) {
+    if (rescorer == null) {
+      if (currentRescorer != null) {
+        currentRescorer = null;
+        clear();
+      }
+    } else {
+      if (!rescorer.equals(currentRescorer)) {
+        currentRescorer = rescorer;
+        clear();
+      }
+    }
+  }
+
+  public void setCurrentlyIncludeKnownItems(boolean currentlyIncludeKnownItems) {
+    this.currentlyIncludeKnownItems = currentlyIncludeKnownItems;
+  }
+
+  @Override
+  public List<RecommendedItem> recommend(long userID, int howMany) throws TasteException {
+    return recommend(userID, howMany, null, false);
+  }
+
+  @Override
+  public List<RecommendedItem> recommend(long userID, int howMany, boolean includeKnownItems) throws TasteException {
+    return recommend(userID, howMany, null, includeKnownItems);
+  }
+
+  @Override
+  public List<RecommendedItem> recommend(long userID, int howMany,IDRescorer rescorer) throws TasteException {
+      return recommend(userID, howMany, rescorer, false);
+  }
+ 
+  @Override
+  public List<RecommendedItem> recommend(long userID, int howMany,IDRescorer rescorer, boolean includeKnownItems)
+    throws TasteException {
+    Preconditions.checkArgument(howMany >= 1, "howMany must be at least 1");
+    synchronized (maxHowMany) {
+      if (howMany > maxHowMany[0]) {
+        maxHowMany[0] = howMany;
+      }
+    }
+
+    // Special case, avoid caching an anonymous user
+    if (userID == PlusAnonymousUserDataModel.TEMP_USER_ID) {
+      return recommendationsRetriever.get(PlusAnonymousUserDataModel.TEMP_USER_ID).getItems();
+    }
+
+    setCurrentRescorer(rescorer);
+    setCurrentlyIncludeKnownItems(includeKnownItems);
+
+    Recommendations recommendations = recommendationCache.get(userID);
+    if (recommendations.getItems().size() < howMany && !recommendations.isNoMoreRecommendableItems()) {
+      clear(userID);
+      recommendations = recommendationCache.get(userID);
+      if (recommendations.getItems().size() < howMany) {
+        recommendations.setNoMoreRecommendableItems(true);
+      }
+    }
+
+    List<RecommendedItem> recommendedItems = recommendations.getItems();
+    return recommendedItems.size() > howMany ? recommendedItems.subList(0, howMany) : recommendedItems;
+  }
+  
+  @Override
+  public float estimatePreference(long userID, long itemID) throws TasteException {
+    return estimatedPrefCache.get(new LongPair(userID, itemID));
+  }
+  
+  @Override
+  public void setPreference(long userID, long itemID, float value) throws TasteException {
+    recommender.setPreference(userID, itemID, value);
+    clear(userID);
+  }
+  
+  @Override
+  public void removePreference(long userID, long itemID) throws TasteException {
+    recommender.removePreference(userID, itemID);
+    clear(userID);
+  }
+  
+  @Override
+  public DataModel getDataModel() {
+    return recommender.getDataModel();
+  }
+  
+  @Override
+  public void refresh(Collection<Refreshable> alreadyRefreshed) {
+    refreshHelper.refresh(alreadyRefreshed);
+  }
+  
+  /**
+   * <p>
+   * Clears cached recommendations for the given user.
+   * </p>
+   * 
+   * @param userID
+   *          clear cached data associated with this user ID
+   */
+  public void clear(final long userID) {
+    log.debug("Clearing recommendations for user ID '{}'", userID);
+    recommendationCache.remove(userID);
+    estimatedPrefCache.removeKeysMatching(new Cache.MatchPredicate<LongPair>() {
+      @Override
+      public boolean matches(LongPair userItemPair) {
+        return userItemPair.getFirst() == userID;
+      }
+    });
+  }
+  
+  /**
+   * <p>
+   * Clears all cached recommendations.
+   * </p>
+   */
+  public void clear() {
+    log.debug("Clearing all recommendations...");
+    recommendationCache.clear();
+    estimatedPrefCache.clear();
+  }
+  
+  @Override
+  public String toString() {
+    return "CachingRecommender[recommender:" + recommender + ']';
+  }
+  
+  private final class RecommendationRetriever implements Retriever<Long,Recommendations> {
+    @Override
+    public Recommendations get(Long key) throws TasteException {
+      log.debug("Retrieving new recommendations for user ID '{}'", key);
+      int howMany = maxHowMany[0];
+      IDRescorer rescorer = currentRescorer;
+      List<RecommendedItem> recommendations =
+          rescorer == null ? recommender.recommend(key, howMany, null, currentlyIncludeKnownItems) :
+              recommender.recommend(key, howMany, rescorer, currentlyIncludeKnownItems);
+      return new Recommendations(Collections.unmodifiableList(recommendations));
+    }
+  }
+  
+  private final class EstimatedPrefRetriever implements Retriever<LongPair,Float> {
+    @Override
+    public Float get(LongPair key) throws TasteException {
+      long userID = key.getFirst();
+      long itemID = key.getSecond();
+      log.debug("Retrieving estimated preference for user ID '{}' and item ID '{}'", userID, itemID);
+      return recommender.estimatePreference(userID, itemID);
+    }
+  }
+  
+  private static final class Recommendations {
+    
+    private final List<RecommendedItem> items;
+    private boolean noMoreRecommendableItems;
+    
+    private Recommendations(List<RecommendedItem> items) {
+      this.items = items;
+    }
+    
+    List<RecommendedItem> getItems() {
+      return items;
+    }
+    
+    boolean isNoMoreRecommendableItems() {
+      return noMoreRecommendableItems;
+    }
+    
+    void setNoMoreRecommendableItems(boolean noMoreRecommendableItems) {
+      this.noMoreRecommendableItems = noMoreRecommendableItems;
+    }
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/mahout/blob/410ed16a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/EstimatedPreferenceCapper.java
----------------------------------------------------------------------
diff --git a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/EstimatedPreferenceCapper.java b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/EstimatedPreferenceCapper.java
new file mode 100644
index 0000000..f0f389f
--- /dev/null
+++ b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/EstimatedPreferenceCapper.java
@@ -0,0 +1,46 @@
+/**
+ * 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.impl.recommender;
+
+import org.apache.mahout.cf.taste.model.DataModel;
+
+/**
+ * Simple class which encapsulates restricting a preference value
+ * to a predefined range. The simple logic is wrapped up here for
+ * performance reasons.
+ */
+public final class EstimatedPreferenceCapper {
+
+  private final float min;
+  private final float max;
+
+  public EstimatedPreferenceCapper(DataModel model) {
+    min = model.getMinPreference();
+    max = model.getMaxPreference();
+  }
+
+  public float capEstimate(float estimate) {
+    if (estimate > max) {
+      estimate = max;
+    } else if (estimate < min) {
+      estimate = min;
+    }
+    return estimate;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/mahout/blob/410ed16a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/GenericBooleanPrefItemBasedRecommender.java
----------------------------------------------------------------------
diff --git a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/GenericBooleanPrefItemBasedRecommender.java b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/GenericBooleanPrefItemBasedRecommender.java
new file mode 100644
index 0000000..40e21a3
--- /dev/null
+++ b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/GenericBooleanPrefItemBasedRecommender.java
@@ -0,0 +1,71 @@
+/**
+ * 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.impl.recommender;
+
+import org.apache.mahout.cf.taste.common.TasteException;
+import org.apache.mahout.cf.taste.model.DataModel;
+import org.apache.mahout.cf.taste.model.PreferenceArray;
+import org.apache.mahout.cf.taste.recommender.CandidateItemsStrategy;
+import org.apache.mahout.cf.taste.recommender.MostSimilarItemsCandidateItemsStrategy;
+import org.apache.mahout.cf.taste.similarity.ItemSimilarity;
+
+/**
+ * A variant on {@link GenericItemBasedRecommender} which is appropriate for use when no notion of preference
+ * value exists in the data.
+ *
+ * @see org.apache.mahout.cf.taste.impl.recommender.GenericBooleanPrefUserBasedRecommender
+ */
+public final class GenericBooleanPrefItemBasedRecommender extends GenericItemBasedRecommender {
+
+  public GenericBooleanPrefItemBasedRecommender(DataModel dataModel, ItemSimilarity similarity) {
+    super(dataModel, similarity);
+  }
+
+  public GenericBooleanPrefItemBasedRecommender(DataModel dataModel, ItemSimilarity similarity,
+      CandidateItemsStrategy candidateItemsStrategy, MostSimilarItemsCandidateItemsStrategy
+      mostSimilarItemsCandidateItemsStrategy) {
+    super(dataModel, similarity, candidateItemsStrategy, mostSimilarItemsCandidateItemsStrategy);
+  }
+  
+  /**
+   * This computation is in a technical sense, wrong, since in the domain of "boolean preference users" where
+   * all preference values are 1, this method should only ever return 1.0 or NaN. This isn't terribly useful
+   * however since it means results can't be ranked by preference value (all are 1). So instead this returns a
+   * sum of similarities.
+   */
+  @Override
+  protected float doEstimatePreference(long userID, PreferenceArray preferencesFromUser, long itemID)
+    throws TasteException {
+    double[] similarities = getSimilarity().itemSimilarities(itemID, preferencesFromUser.getIDs());
+    boolean foundAPref = false;
+    double totalSimilarity = 0.0;
+    for (double theSimilarity : similarities) {
+      if (!Double.isNaN(theSimilarity)) {
+        foundAPref = true;
+        totalSimilarity += theSimilarity;
+      }
+    }
+    return foundAPref ? (float) totalSimilarity : Float.NaN;
+  }
+  
+  @Override
+  public String toString() {
+    return "GenericBooleanPrefItemBasedRecommender";
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/mahout/blob/410ed16a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/GenericBooleanPrefUserBasedRecommender.java
----------------------------------------------------------------------
diff --git a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/GenericBooleanPrefUserBasedRecommender.java b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/GenericBooleanPrefUserBasedRecommender.java
new file mode 100644
index 0000000..15fcc9f
--- /dev/null
+++ b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/GenericBooleanPrefUserBasedRecommender.java
@@ -0,0 +1,82 @@
+/**
+ * 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.impl.recommender;
+
+import org.apache.mahout.cf.taste.common.TasteException;
+import org.apache.mahout.cf.taste.impl.common.FastIDSet;
+import org.apache.mahout.cf.taste.model.DataModel;
+import org.apache.mahout.cf.taste.neighborhood.UserNeighborhood;
+import org.apache.mahout.cf.taste.similarity.UserSimilarity;
+
+/**
+ * A variant on {@link GenericUserBasedRecommender} which is appropriate for use when no notion of preference
+ * value exists in the data.
+ */
+public final class GenericBooleanPrefUserBasedRecommender extends GenericUserBasedRecommender {
+  
+  public GenericBooleanPrefUserBasedRecommender(DataModel dataModel,
+                                                UserNeighborhood neighborhood,
+                                                UserSimilarity similarity) {
+    super(dataModel, neighborhood, similarity);
+  }
+  
+  /**
+   * This computation is in a technical sense, wrong, since in the domain of "boolean preference users" where
+   * all preference values are 1, this method should only ever return 1.0 or NaN. This isn't terribly useful
+   * however since it means results can't be ranked by preference value (all are 1). So instead this returns a
+   * sum of similarities to any other user in the neighborhood who has also rated the item.
+   */
+  @Override
+  protected float doEstimatePreference(long theUserID, long[] theNeighborhood, long itemID) throws TasteException {
+    if (theNeighborhood.length == 0) {
+      return Float.NaN;
+    }
+    DataModel dataModel = getDataModel();
+    UserSimilarity similarity = getSimilarity();
+    float totalSimilarity = 0.0f;
+    boolean foundAPref = false;
+    for (long userID : theNeighborhood) {
+      // See GenericItemBasedRecommender.doEstimatePreference() too
+      if (userID != theUserID && dataModel.getPreferenceValue(userID, itemID) != null) {
+        foundAPref = true;
+        totalSimilarity += (float) similarity.userSimilarity(theUserID, userID);
+      }
+    }
+    return foundAPref ? totalSimilarity : Float.NaN;
+  }
+  
+  @Override
+  protected FastIDSet getAllOtherItems(long[] theNeighborhood, long theUserID, boolean includeKnownItems)
+      throws TasteException {
+    DataModel dataModel = getDataModel();
+    FastIDSet possibleItemIDs = new FastIDSet();
+    for (long userID : theNeighborhood) {
+      possibleItemIDs.addAll(dataModel.getItemIDsFromUser(userID));
+    }
+    if (!includeKnownItems) {
+      possibleItemIDs.removeAll(dataModel.getItemIDsFromUser(theUserID));
+    }
+    return possibleItemIDs;
+  }
+  
+  @Override
+  public String toString() {
+    return "GenericBooleanPrefUserBasedRecommender";
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/mahout/blob/410ed16a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/GenericItemBasedRecommender.java
----------------------------------------------------------------------
diff --git a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/GenericItemBasedRecommender.java b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/GenericItemBasedRecommender.java
new file mode 100644
index 0000000..6dc8aa5
--- /dev/null
+++ b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/GenericItemBasedRecommender.java
@@ -0,0 +1,378 @@
+/**
+ * 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.impl.recommender;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+import org.apache.mahout.cf.taste.common.Refreshable;
+import org.apache.mahout.cf.taste.common.TasteException;
+import org.apache.mahout.cf.taste.impl.common.FastIDSet;
+import org.apache.mahout.cf.taste.impl.common.FullRunningAverage;
+import org.apache.mahout.cf.taste.impl.common.RefreshHelper;
+import org.apache.mahout.cf.taste.impl.common.RunningAverage;
+import org.apache.mahout.cf.taste.model.DataModel;
+import org.apache.mahout.cf.taste.model.PreferenceArray;
+import org.apache.mahout.cf.taste.recommender.CandidateItemsStrategy;
+import org.apache.mahout.cf.taste.recommender.IDRescorer;
+import org.apache.mahout.cf.taste.recommender.ItemBasedRecommender;
+import org.apache.mahout.cf.taste.recommender.MostSimilarItemsCandidateItemsStrategy;
+import org.apache.mahout.cf.taste.recommender.RecommendedItem;
+import org.apache.mahout.cf.taste.recommender.Rescorer;
+import org.apache.mahout.cf.taste.similarity.ItemSimilarity;
+import org.apache.mahout.common.LongPair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * <p>
+ * A simple {@link org.apache.mahout.cf.taste.recommender.Recommender} which uses a given
+ * {@link org.apache.mahout.cf.taste.model.DataModel} and
+ * {@link org.apache.mahout.cf.taste.similarity.ItemSimilarity} to produce recommendations. This class
+ * represents Taste's support for item-based recommenders.
+ * </p>
+ * 
+ * <p>
+ * The {@link org.apache.mahout.cf.taste.similarity.ItemSimilarity} is the most important point to discuss
+ * here. Item-based recommenders are useful because they can take advantage of something to be very fast: they
+ * base their computations on item similarity, not user similarity, and item similarity is relatively static.
+ * It can be precomputed, instead of re-computed in real time.
+ * </p>
+ * 
+ * <p>
+ * Thus it's strongly recommended that you use
+ * {@link org.apache.mahout.cf.taste.impl.similarity.GenericItemSimilarity} with pre-computed similarities if
+ * you're going to use this class. You can use
+ * {@link org.apache.mahout.cf.taste.impl.similarity.PearsonCorrelationSimilarity} too, which computes
+ * similarities in real-time, but will probably find this painfully slow for large amounts of data.
+ * </p>
+ */
+public class GenericItemBasedRecommender extends AbstractRecommender implements ItemBasedRecommender {
+  
+  private static final Logger log = LoggerFactory.getLogger(GenericItemBasedRecommender.class);
+  
+  private final ItemSimilarity similarity;
+  private final MostSimilarItemsCandidateItemsStrategy mostSimilarItemsCandidateItemsStrategy;
+  private final RefreshHelper refreshHelper;
+  private EstimatedPreferenceCapper capper;
+
+  private static final boolean EXCLUDE_ITEM_IF_NOT_SIMILAR_TO_ALL_BY_DEFAULT = true;
+
+  public GenericItemBasedRecommender(DataModel dataModel,
+                                     ItemSimilarity similarity,
+                                     CandidateItemsStrategy candidateItemsStrategy,
+                                     MostSimilarItemsCandidateItemsStrategy mostSimilarItemsCandidateItemsStrategy) {
+    super(dataModel, candidateItemsStrategy);
+    Preconditions.checkArgument(similarity != null, "similarity is null");
+    this.similarity = similarity;
+    Preconditions.checkArgument(mostSimilarItemsCandidateItemsStrategy != null,
+        "mostSimilarItemsCandidateItemsStrategy is null");
+    this.mostSimilarItemsCandidateItemsStrategy = mostSimilarItemsCandidateItemsStrategy;
+    this.refreshHelper = new RefreshHelper(new Callable<Void>() {
+      @Override
+      public Void call() {
+        capper = buildCapper();
+        return null;
+      }
+    });
+    refreshHelper.addDependency(dataModel);
+    refreshHelper.addDependency(similarity);
+    refreshHelper.addDependency(candidateItemsStrategy);
+    refreshHelper.addDependency(mostSimilarItemsCandidateItemsStrategy);
+    capper = buildCapper();
+  }
+
+  public GenericItemBasedRecommender(DataModel dataModel, ItemSimilarity similarity) {
+    this(dataModel,
+         similarity,
+         AbstractRecommender.getDefaultCandidateItemsStrategy(),
+         getDefaultMostSimilarItemsCandidateItemsStrategy());
+  }
+
+  protected static MostSimilarItemsCandidateItemsStrategy getDefaultMostSimilarItemsCandidateItemsStrategy() {
+    return new PreferredItemsNeighborhoodCandidateItemsStrategy();
+  }
+
+  public ItemSimilarity getSimilarity() {
+    return similarity;
+  }
+  
+  @Override
+  public List<RecommendedItem> recommend(long userID, int howMany, IDRescorer rescorer, boolean includeKnownItems)
+    throws TasteException {
+    Preconditions.checkArgument(howMany >= 1, "howMany must be at least 1");
+    log.debug("Recommending items for user ID '{}'", userID);
+
+    PreferenceArray preferencesFromUser = getDataModel().getPreferencesFromUser(userID);
+    if (preferencesFromUser.length() == 0) {
+      return Collections.emptyList();
+    }
+
+    FastIDSet possibleItemIDs = getAllOtherItems(userID, preferencesFromUser, includeKnownItems);
+
+    TopItems.Estimator<Long> estimator = new Estimator(userID, preferencesFromUser);
+
+    List<RecommendedItem> topItems = TopItems.getTopItems(howMany, possibleItemIDs.iterator(), rescorer,
+      estimator);
+
+    log.debug("Recommendations are: {}", topItems);
+    return topItems;
+  }
+  
+  @Override
+  public float estimatePreference(long userID, long itemID) throws TasteException {
+    PreferenceArray preferencesFromUser = getDataModel().getPreferencesFromUser(userID);
+    Float actualPref = getPreferenceForItem(preferencesFromUser, itemID);
+    if (actualPref != null) {
+      return actualPref;
+    }
+    return doEstimatePreference(userID, preferencesFromUser, itemID);
+  }
+
+  private static Float getPreferenceForItem(PreferenceArray preferencesFromUser, long itemID) {
+    int size = preferencesFromUser.length();
+    for (int i = 0; i < size; i++) {
+      if (preferencesFromUser.getItemID(i) == itemID) {
+        return preferencesFromUser.getValue(i);
+      }
+    }
+    return null;
+  }
+
+  @Override
+  public List<RecommendedItem> mostSimilarItems(long itemID, int howMany) throws TasteException {
+    return mostSimilarItems(itemID, howMany, null);
+  }
+  
+  @Override
+  public List<RecommendedItem> mostSimilarItems(long itemID, int howMany,
+                                                Rescorer<LongPair> rescorer) throws TasteException {
+    TopItems.Estimator<Long> estimator = new MostSimilarEstimator(itemID, similarity, rescorer);
+    return doMostSimilarItems(new long[] {itemID}, howMany, estimator);
+  }
+  
+  @Override
+  public List<RecommendedItem> mostSimilarItems(long[] itemIDs, int howMany) throws TasteException {
+    TopItems.Estimator<Long> estimator = new MultiMostSimilarEstimator(itemIDs, similarity, null,
+        EXCLUDE_ITEM_IF_NOT_SIMILAR_TO_ALL_BY_DEFAULT);
+    return doMostSimilarItems(itemIDs, howMany, estimator);
+  }
+  
+  @Override
+  public List<RecommendedItem> mostSimilarItems(long[] itemIDs, int howMany,
+                                                Rescorer<LongPair> rescorer) throws TasteException {
+    TopItems.Estimator<Long> estimator = new MultiMostSimilarEstimator(itemIDs, similarity, rescorer,
+        EXCLUDE_ITEM_IF_NOT_SIMILAR_TO_ALL_BY_DEFAULT);
+    return doMostSimilarItems(itemIDs, howMany, estimator);
+  }
+
+  @Override
+  public List<RecommendedItem> mostSimilarItems(long[] itemIDs,
+                                                int howMany,
+                                                boolean excludeItemIfNotSimilarToAll) throws TasteException {
+    TopItems.Estimator<Long> estimator = new MultiMostSimilarEstimator(itemIDs, similarity, null,
+        excludeItemIfNotSimilarToAll);
+    return doMostSimilarItems(itemIDs, howMany, estimator);
+  }
+
+  @Override
+  public List<RecommendedItem> mostSimilarItems(long[] itemIDs, int howMany,
+                                                Rescorer<LongPair> rescorer,
+                                                boolean excludeItemIfNotSimilarToAll) throws TasteException {
+    TopItems.Estimator<Long> estimator = new MultiMostSimilarEstimator(itemIDs, similarity, rescorer,
+        excludeItemIfNotSimilarToAll);
+    return doMostSimilarItems(itemIDs, howMany, estimator);
+  }
+
+  @Override
+  public List<RecommendedItem> recommendedBecause(long userID, long itemID, int howMany) throws TasteException {
+    Preconditions.checkArgument(howMany >= 1, "howMany must be at least 1");
+
+    DataModel model = getDataModel();
+    TopItems.Estimator<Long> estimator = new RecommendedBecauseEstimator(userID, itemID);
+
+    PreferenceArray prefs = model.getPreferencesFromUser(userID);
+    int size = prefs.length();
+    FastIDSet allUserItems = new FastIDSet(size);
+    for (int i = 0; i < size; i++) {
+      allUserItems.add(prefs.getItemID(i));
+    }
+    allUserItems.remove(itemID);
+
+    return TopItems.getTopItems(howMany, allUserItems.iterator(), null, estimator);
+  }
+  
+  private List<RecommendedItem> doMostSimilarItems(long[] itemIDs,
+                                                   int howMany,
+                                                   TopItems.Estimator<Long> estimator) throws TasteException {
+    FastIDSet possibleItemIDs = mostSimilarItemsCandidateItemsStrategy.getCandidateItems(itemIDs, getDataModel());
+    return TopItems.getTopItems(howMany, possibleItemIDs.iterator(), null, estimator);
+  }
+  
+  protected float doEstimatePreference(long userID, PreferenceArray preferencesFromUser, long itemID)
+    throws TasteException {
+    double preference = 0.0;
+    double totalSimilarity = 0.0;
+    int count = 0;
+    double[] similarities = similarity.itemSimilarities(itemID, preferencesFromUser.getIDs());
+    for (int i = 0; i < similarities.length; i++) {
+      double theSimilarity = similarities[i];
+      if (!Double.isNaN(theSimilarity)) {
+        // Weights can be negative!
+        preference += theSimilarity * preferencesFromUser.getValue(i);
+        totalSimilarity += theSimilarity;
+        count++;
+      }
+    }
+    // Throw out the estimate if it was based on no data points, of course, but also if based on
+    // just one. This is a bit of a band-aid on the 'stock' item-based algorithm for the moment.
+    // The reason is that in this case the estimate is, simply, the user's rating for one item
+    // that happened to have a defined similarity. The similarity score doesn't matter, and that
+    // seems like a bad situation.
+    if (count <= 1) {
+      return Float.NaN;
+    }
+    float estimate = (float) (preference / totalSimilarity);
+    if (capper != null) {
+      estimate = capper.capEstimate(estimate);
+    }
+    return estimate;
+  }
+
+  @Override
+  public void refresh(Collection<Refreshable> alreadyRefreshed) {
+    refreshHelper.refresh(alreadyRefreshed);
+  }
+  
+  @Override
+  public String toString() {
+    return "GenericItemBasedRecommender[similarity:" + similarity + ']';
+  }
+
+  private EstimatedPreferenceCapper buildCapper() {
+    DataModel dataModel = getDataModel();
+    if (Float.isNaN(dataModel.getMinPreference()) && Float.isNaN(dataModel.getMaxPreference())) {
+      return null;
+    } else {
+      return new EstimatedPreferenceCapper(dataModel);
+    }
+  }
+  
+  public static class MostSimilarEstimator implements TopItems.Estimator<Long> {
+    
+    private final long toItemID;
+    private final ItemSimilarity similarity;
+    private final Rescorer<LongPair> rescorer;
+    
+    public MostSimilarEstimator(long toItemID, ItemSimilarity similarity, Rescorer<LongPair> rescorer) {
+      this.toItemID = toItemID;
+      this.similarity = similarity;
+      this.rescorer = rescorer;
+    }
+    
+    @Override
+    public double estimate(Long itemID) throws TasteException {
+      LongPair pair = new LongPair(toItemID, itemID);
+      if (rescorer != null && rescorer.isFiltered(pair)) {
+        return Double.NaN;
+      }
+      double originalEstimate = similarity.itemSimilarity(toItemID, itemID);
+      return rescorer == null ? originalEstimate : rescorer.rescore(pair, originalEstimate);
+    }
+  }
+  
+  private final class Estimator implements TopItems.Estimator<Long> {
+    
+    private final long userID;
+    private final PreferenceArray preferencesFromUser;
+    
+    private Estimator(long userID, PreferenceArray preferencesFromUser) {
+      this.userID = userID;
+      this.preferencesFromUser = preferencesFromUser;
+    }
+    
+    @Override
+    public double estimate(Long itemID) throws TasteException {
+      return doEstimatePreference(userID, preferencesFromUser, itemID);
+    }
+  }
+  
+  private static final class MultiMostSimilarEstimator implements TopItems.Estimator<Long> {
+    
+    private final long[] toItemIDs;
+    private final ItemSimilarity similarity;
+    private final Rescorer<LongPair> rescorer;
+    private final boolean excludeItemIfNotSimilarToAll;
+    
+    private MultiMostSimilarEstimator(long[] toItemIDs, ItemSimilarity similarity, Rescorer<LongPair> rescorer,
+        boolean excludeItemIfNotSimilarToAll) {
+      this.toItemIDs = toItemIDs;
+      this.similarity = similarity;
+      this.rescorer = rescorer;
+      this.excludeItemIfNotSimilarToAll = excludeItemIfNotSimilarToAll;
+    }
+    
+    @Override
+    public double estimate(Long itemID) throws TasteException {
+      RunningAverage average = new FullRunningAverage();
+      double[] similarities = similarity.itemSimilarities(itemID, toItemIDs);
+      for (int i = 0; i < toItemIDs.length; i++) {
+        long toItemID = toItemIDs[i];
+        LongPair pair = new LongPair(toItemID, itemID);
+        if (rescorer != null && rescorer.isFiltered(pair)) {
+          continue;
+        }
+        double estimate = similarities[i];
+        if (rescorer != null) {
+          estimate = rescorer.rescore(pair, estimate);
+        }
+        if (excludeItemIfNotSimilarToAll || !Double.isNaN(estimate)) {
+          average.addDatum(estimate);
+        }
+      }
+      double averageEstimate = average.getAverage();
+      return averageEstimate == 0 ? Double.NaN : averageEstimate;
+    }
+  }
+  
+  private final class RecommendedBecauseEstimator implements TopItems.Estimator<Long> {
+    
+    private final long userID;
+    private final long recommendedItemID;
+
+    private RecommendedBecauseEstimator(long userID, long recommendedItemID) {
+      this.userID = userID;
+      this.recommendedItemID = recommendedItemID;
+    }
+    
+    @Override
+    public double estimate(Long itemID) throws TasteException {
+      Float pref = getDataModel().getPreferenceValue(userID, itemID);
+      if (pref == null) {
+        return Float.NaN;
+      }
+      double similarityValue = similarity.itemSimilarity(recommendedItemID, itemID);
+      return (1.0 + similarityValue) * pref;
+    }
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/mahout/blob/410ed16a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/GenericRecommendedItem.java
----------------------------------------------------------------------
diff --git a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/GenericRecommendedItem.java b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/GenericRecommendedItem.java
new file mode 100644
index 0000000..8c8f6ce
--- /dev/null
+++ b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/GenericRecommendedItem.java
@@ -0,0 +1,76 @@
+/**
+ * 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.impl.recommender;
+
+import java.io.Serializable;
+
+import org.apache.mahout.cf.taste.recommender.RecommendedItem;
+import org.apache.mahout.common.RandomUtils;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * <p>
+ * A simple implementation of {@link RecommendedItem}.
+ * </p>
+ */
+public final class GenericRecommendedItem implements RecommendedItem, Serializable {
+  
+  private final long itemID;
+  private final float value;
+  
+  /**
+   * @throws IllegalArgumentException
+   *           if item is null or value is NaN
+   */
+  public GenericRecommendedItem(long itemID, float value) {
+    Preconditions.checkArgument(!Float.isNaN(value), "value is NaN");
+    this.itemID = itemID;
+    this.value = value;
+  }
+  
+  @Override
+  public long getItemID() {
+    return itemID;
+  }
+  
+  @Override
+  public float getValue() {
+    return value;
+  }
+
+  @Override
+  public String toString() {
+    return "RecommendedItem[item:" + itemID + ", value:" + value + ']';
+  }
+  
+  @Override
+  public int hashCode() {
+    return (int) itemID ^ RandomUtils.hashFloat(value);
+  }
+  
+  @Override
+  public boolean equals(Object o) {
+    if (!(o instanceof GenericRecommendedItem)) {
+      return false;
+    }
+    RecommendedItem other = (RecommendedItem) o;
+    return itemID == other.getItemID() && value == other.getValue();
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/mahout/blob/410ed16a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/GenericUserBasedRecommender.java
----------------------------------------------------------------------
diff --git a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/GenericUserBasedRecommender.java b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/GenericUserBasedRecommender.java
new file mode 100644
index 0000000..1e2ef73
--- /dev/null
+++ b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/GenericUserBasedRecommender.java
@@ -0,0 +1,247 @@
+/**
+ * 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.impl.recommender;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+import org.apache.mahout.cf.taste.common.Refreshable;
+import org.apache.mahout.cf.taste.common.TasteException;
+import org.apache.mahout.cf.taste.impl.common.FastIDSet;
+import org.apache.mahout.cf.taste.impl.common.RefreshHelper;
+import org.apache.mahout.cf.taste.model.DataModel;
+import org.apache.mahout.cf.taste.neighborhood.UserNeighborhood;
+import org.apache.mahout.cf.taste.recommender.IDRescorer;
+import org.apache.mahout.cf.taste.recommender.RecommendedItem;
+import org.apache.mahout.cf.taste.recommender.Rescorer;
+import org.apache.mahout.cf.taste.recommender.UserBasedRecommender;
+import org.apache.mahout.cf.taste.similarity.UserSimilarity;
+import org.apache.mahout.common.LongPair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * <p>
+ * A simple {@link org.apache.mahout.cf.taste.recommender.Recommender}
+ * which uses a given {@link DataModel} and {@link UserNeighborhood} to produce recommendations.
+ * </p>
+ */
+public class GenericUserBasedRecommender extends AbstractRecommender implements UserBasedRecommender {
+  
+  private static final Logger log = LoggerFactory.getLogger(GenericUserBasedRecommender.class);
+  
+  private final UserNeighborhood neighborhood;
+  private final UserSimilarity similarity;
+  private final RefreshHelper refreshHelper;
+  private EstimatedPreferenceCapper capper;
+  
+  public GenericUserBasedRecommender(DataModel dataModel,
+                                     UserNeighborhood neighborhood,
+                                     UserSimilarity similarity) {
+    super(dataModel);
+    Preconditions.checkArgument(neighborhood != null, "neighborhood is null");
+    this.neighborhood = neighborhood;
+    this.similarity = similarity;
+    this.refreshHelper = new RefreshHelper(new Callable<Void>() {
+      @Override
+      public Void call() {
+        capper = buildCapper();
+        return null;
+      }
+    });
+    refreshHelper.addDependency(dataModel);
+    refreshHelper.addDependency(similarity);
+    refreshHelper.addDependency(neighborhood);
+    capper = buildCapper();
+  }
+  
+  public UserSimilarity getSimilarity() {
+    return similarity;
+  }
+  
+  @Override
+  public List<RecommendedItem> recommend(long userID, int howMany, IDRescorer rescorer, boolean includeKnownItems)
+    throws TasteException {
+    Preconditions.checkArgument(howMany >= 1, "howMany must be at least 1");
+
+    log.debug("Recommending items for user ID '{}'", userID);
+
+    long[] theNeighborhood = neighborhood.getUserNeighborhood(userID);
+
+    if (theNeighborhood.length == 0) {
+      return Collections.emptyList();
+    }
+
+    FastIDSet allItemIDs = getAllOtherItems(theNeighborhood, userID, includeKnownItems);
+
+    TopItems.Estimator<Long> estimator = new Estimator(userID, theNeighborhood);
+
+    List<RecommendedItem> topItems = TopItems
+        .getTopItems(howMany, allItemIDs.iterator(), rescorer, estimator);
+
+    log.debug("Recommendations are: {}", topItems);
+    return topItems;
+  }
+  
+  @Override
+  public float estimatePreference(long userID, long itemID) throws TasteException {
+    DataModel model = getDataModel();
+    Float actualPref = model.getPreferenceValue(userID, itemID);
+    if (actualPref != null) {
+      return actualPref;
+    }
+    long[] theNeighborhood = neighborhood.getUserNeighborhood(userID);
+    return doEstimatePreference(userID, theNeighborhood, itemID);
+  }
+  
+  @Override
+  public long[] mostSimilarUserIDs(long userID, int howMany) throws TasteException {
+    return mostSimilarUserIDs(userID, howMany, null);
+  }
+  
+  @Override
+  public long[] mostSimilarUserIDs(long userID, int howMany, Rescorer<LongPair> rescorer) throws TasteException {
+    TopItems.Estimator<Long> estimator = new MostSimilarEstimator(userID, similarity, rescorer);
+    return doMostSimilarUsers(howMany, estimator);
+  }
+  
+  private long[] doMostSimilarUsers(int howMany, TopItems.Estimator<Long> estimator) throws TasteException {
+    DataModel model = getDataModel();
+    return TopItems.getTopUsers(howMany, model.getUserIDs(), null, estimator);
+  }
+  
+  protected float doEstimatePreference(long theUserID, long[] theNeighborhood, long itemID) throws TasteException {
+    if (theNeighborhood.length == 0) {
+      return Float.NaN;
+    }
+    DataModel dataModel = getDataModel();
+    double preference = 0.0;
+    double totalSimilarity = 0.0;
+    int count = 0;
+    for (long userID : theNeighborhood) {
+      if (userID != theUserID) {
+        // See GenericItemBasedRecommender.doEstimatePreference() too
+        Float pref = dataModel.getPreferenceValue(userID, itemID);
+        if (pref != null) {
+          double theSimilarity = similarity.userSimilarity(theUserID, userID);
+          if (!Double.isNaN(theSimilarity)) {
+            preference += theSimilarity * pref;
+            totalSimilarity += theSimilarity;
+            count++;
+          }
+        }
+      }
+    }
+    // Throw out the estimate if it was based on no data points, of course, but also if based on
+    // just one. This is a bit of a band-aid on the 'stock' item-based algorithm for the moment.
+    // The reason is that in this case the estimate is, simply, the user's rating for one item
+    // that happened to have a defined similarity. The similarity score doesn't matter, and that
+    // seems like a bad situation.
+    if (count <= 1) {
+      return Float.NaN;
+    }
+    float estimate = (float) (preference / totalSimilarity);
+    if (capper != null) {
+      estimate = capper.capEstimate(estimate);
+    }
+    return estimate;
+  }
+  
+  protected FastIDSet getAllOtherItems(long[] theNeighborhood, long theUserID, boolean includeKnownItems)
+    throws TasteException {
+    DataModel dataModel = getDataModel();
+    FastIDSet possibleItemIDs = new FastIDSet();
+    for (long userID : theNeighborhood) {
+      possibleItemIDs.addAll(dataModel.getItemIDsFromUser(userID));
+    }
+    if (!includeKnownItems) {
+      possibleItemIDs.removeAll(dataModel.getItemIDsFromUser(theUserID));
+    }
+    return possibleItemIDs;
+  }
+  
+  @Override
+  public void refresh(Collection<Refreshable> alreadyRefreshed) {
+    refreshHelper.refresh(alreadyRefreshed);
+  }
+  
+  @Override
+  public String toString() {
+    return "GenericUserBasedRecommender[neighborhood:" + neighborhood + ']';
+  }
+
+  private EstimatedPreferenceCapper buildCapper() {
+    DataModel dataModel = getDataModel();
+    if (Float.isNaN(dataModel.getMinPreference()) && Float.isNaN(dataModel.getMaxPreference())) {
+      return null;
+    } else {
+      return new EstimatedPreferenceCapper(dataModel);
+    }
+  }
+  
+  private static final class MostSimilarEstimator implements TopItems.Estimator<Long> {
+    
+    private final long toUserID;
+    private final UserSimilarity similarity;
+    private final Rescorer<LongPair> rescorer;
+    
+    private MostSimilarEstimator(long toUserID, UserSimilarity similarity, Rescorer<LongPair> rescorer) {
+      this.toUserID = toUserID;
+      this.similarity = similarity;
+      this.rescorer = rescorer;
+    }
+    
+    @Override
+    public double estimate(Long userID) throws TasteException {
+      // Don't consider the user itself as a possible most similar user
+      if (userID == toUserID) {
+        return Double.NaN;
+      }
+      if (rescorer == null) {
+        return similarity.userSimilarity(toUserID, userID);
+      } else {
+        LongPair pair = new LongPair(toUserID, userID);
+        if (rescorer.isFiltered(pair)) {
+          return Double.NaN;
+        }
+        double originalEstimate = similarity.userSimilarity(toUserID, userID);
+        return rescorer.rescore(pair, originalEstimate);
+      }
+    }
+  }
+  
+  private final class Estimator implements TopItems.Estimator<Long> {
+    
+    private final long theUserID;
+    private final long[] theNeighborhood;
+    
+    Estimator(long theUserID, long[] theNeighborhood) {
+      this.theUserID = theUserID;
+      this.theNeighborhood = theNeighborhood;
+    }
+    
+    @Override
+    public double estimate(Long itemID) throws TasteException {
+      return doEstimatePreference(theUserID, theNeighborhood, itemID);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/mahout/blob/410ed16a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/ItemAverageRecommender.java
----------------------------------------------------------------------
diff --git a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/ItemAverageRecommender.java b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/ItemAverageRecommender.java
new file mode 100644
index 0000000..618c65f
--- /dev/null
+++ b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/ItemAverageRecommender.java
@@ -0,0 +1,199 @@
+/**
+ * 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.impl.recommender;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.mahout.cf.taste.common.NoSuchUserException;
+import org.apache.mahout.cf.taste.common.Refreshable;
+import org.apache.mahout.cf.taste.common.TasteException;
+import org.apache.mahout.cf.taste.impl.common.FastByIDMap;
+import org.apache.mahout.cf.taste.impl.common.FastIDSet;
+import org.apache.mahout.cf.taste.impl.common.FullRunningAverage;
+import org.apache.mahout.cf.taste.impl.common.LongPrimitiveIterator;
+import org.apache.mahout.cf.taste.impl.common.RefreshHelper;
+import org.apache.mahout.cf.taste.impl.common.RunningAverage;
+import org.apache.mahout.cf.taste.model.DataModel;
+import org.apache.mahout.cf.taste.model.PreferenceArray;
+import org.apache.mahout.cf.taste.recommender.IDRescorer;
+import org.apache.mahout.cf.taste.recommender.RecommendedItem;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * <p>
+ * A simple recommender that always estimates preference for an item to be the average of all known preference
+ * values for that item. No information about users is taken into account. This implementation is provided for
+ * experimentation; while simple and fast, it may not produce very good recommendations.
+ * </p>
+ */
+public final class ItemAverageRecommender extends AbstractRecommender {
+  
+  private static final Logger log = LoggerFactory.getLogger(ItemAverageRecommender.class);
+  
+  private final FastByIDMap<RunningAverage> itemAverages;
+  private final ReadWriteLock buildAveragesLock;
+  private final RefreshHelper refreshHelper;
+  
+  public ItemAverageRecommender(DataModel dataModel) throws TasteException {
+    super(dataModel);
+    this.itemAverages = new FastByIDMap<>();
+    this.buildAveragesLock = new ReentrantReadWriteLock();
+    this.refreshHelper = new RefreshHelper(new Callable<Object>() {
+      @Override
+      public Object call() throws TasteException {
+        buildAverageDiffs();
+        return null;
+      }
+    });
+    refreshHelper.addDependency(dataModel);
+    buildAverageDiffs();
+  }
+  
+  @Override
+  public List<RecommendedItem> recommend(long userID, int howMany, IDRescorer rescorer, boolean includeKnownItems)
+    throws TasteException {
+    Preconditions.checkArgument(howMany >= 1, "howMany must be at least 1");
+    log.debug("Recommending items for user ID '{}'", userID);
+
+    PreferenceArray preferencesFromUser = getDataModel().getPreferencesFromUser(userID);
+    FastIDSet possibleItemIDs = getAllOtherItems(userID, preferencesFromUser, includeKnownItems);
+
+    TopItems.Estimator<Long> estimator = new Estimator();
+
+    List<RecommendedItem> topItems = TopItems.getTopItems(howMany, possibleItemIDs.iterator(), rescorer,
+      estimator);
+
+    log.debug("Recommendations are: {}", topItems);
+    return topItems;
+  }
+  
+  @Override
+  public float estimatePreference(long userID, long itemID) throws TasteException {
+    DataModel dataModel = getDataModel();
+    Float actualPref = dataModel.getPreferenceValue(userID, itemID);
+    if (actualPref != null) {
+      return actualPref;
+    }
+    return doEstimatePreference(itemID);
+  }
+  
+  private float doEstimatePreference(long itemID) {
+    buildAveragesLock.readLock().lock();
+    try {
+      RunningAverage average = itemAverages.get(itemID);
+      return average == null ? Float.NaN : (float) average.getAverage();
+    } finally {
+      buildAveragesLock.readLock().unlock();
+    }
+  }
+  
+  private void buildAverageDiffs() throws TasteException {
+    try {
+      buildAveragesLock.writeLock().lock();
+      DataModel dataModel = getDataModel();
+      LongPrimitiveIterator it = dataModel.getUserIDs();
+      while (it.hasNext()) {
+        PreferenceArray prefs = dataModel.getPreferencesFromUser(it.nextLong());
+        int size = prefs.length();
+        for (int i = 0; i < size; i++) {
+          long itemID = prefs.getItemID(i);
+          RunningAverage average = itemAverages.get(itemID);
+          if (average == null) {
+            average = new FullRunningAverage();
+            itemAverages.put(itemID, average);
+          }
+          average.addDatum(prefs.getValue(i));
+        }
+      }
+    } finally {
+      buildAveragesLock.writeLock().unlock();
+    }
+  }
+  
+  @Override
+  public void setPreference(long userID, long itemID, float value) throws TasteException {
+    DataModel dataModel = getDataModel();
+    double prefDelta;
+    try {
+      Float oldPref = dataModel.getPreferenceValue(userID, itemID);
+      prefDelta = oldPref == null ? value : value - oldPref;
+    } catch (NoSuchUserException nsee) {
+      prefDelta = value;
+    }
+    super.setPreference(userID, itemID, value);
+    try {
+      buildAveragesLock.writeLock().lock();
+      RunningAverage average = itemAverages.get(itemID);
+      if (average == null) {
+        RunningAverage newAverage = new FullRunningAverage();
+        newAverage.addDatum(prefDelta);
+        itemAverages.put(itemID, newAverage);
+      } else {
+        average.changeDatum(prefDelta);
+      }
+    } finally {
+      buildAveragesLock.writeLock().unlock();
+    }
+  }
+  
+  @Override
+  public void removePreference(long userID, long itemID) throws TasteException {
+    DataModel dataModel = getDataModel();
+    Float oldPref = dataModel.getPreferenceValue(userID, itemID);
+    super.removePreference(userID, itemID);
+    if (oldPref != null) {
+      try {
+        buildAveragesLock.writeLock().lock();
+        RunningAverage average = itemAverages.get(itemID);
+        if (average == null) {
+          throw new IllegalStateException("No preferences exist for item ID: " + itemID);
+        } else {
+          average.removeDatum(oldPref);
+        }
+      } finally {
+        buildAveragesLock.writeLock().unlock();
+      }
+    }
+  }
+  
+  @Override
+  public void refresh(Collection<Refreshable> alreadyRefreshed) {
+    refreshHelper.refresh(alreadyRefreshed);
+  }
+  
+  @Override
+  public String toString() {
+    return "ItemAverageRecommender";
+  }
+  
+  private final class Estimator implements TopItems.Estimator<Long> {
+    
+    @Override
+    public double estimate(Long itemID) {
+      return doEstimatePreference(itemID);
+    }
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/mahout/blob/410ed16a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/ItemUserAverageRecommender.java
----------------------------------------------------------------------
diff --git a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/ItemUserAverageRecommender.java b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/ItemUserAverageRecommender.java
new file mode 100644
index 0000000..b2bcd24
--- /dev/null
+++ b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/ItemUserAverageRecommender.java
@@ -0,0 +1,240 @@
+/**
+ * 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.impl.recommender;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.mahout.cf.taste.common.NoSuchUserException;
+import org.apache.mahout.cf.taste.common.Refreshable;
+import org.apache.mahout.cf.taste.common.TasteException;
+import org.apache.mahout.cf.taste.impl.common.FastByIDMap;
+import org.apache.mahout.cf.taste.impl.common.FastIDSet;
+import org.apache.mahout.cf.taste.impl.common.FullRunningAverage;
+import org.apache.mahout.cf.taste.impl.common.LongPrimitiveIterator;
+import org.apache.mahout.cf.taste.impl.common.RefreshHelper;
+import org.apache.mahout.cf.taste.impl.common.RunningAverage;
+import org.apache.mahout.cf.taste.model.DataModel;
+import org.apache.mahout.cf.taste.model.PreferenceArray;
+import org.apache.mahout.cf.taste.recommender.IDRescorer;
+import org.apache.mahout.cf.taste.recommender.RecommendedItem;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * <p>
+ * Like {@link ItemAverageRecommender}, except that estimated preferences are adjusted for the users' average
+ * preference value. For example, say user X has not rated item Y. Item Y's average preference value is 3.5.
+ * User X's average preference value is 4.2, and the average over all preference values is 4.0. User X prefers
+ * items 0.2 higher on average, so, the estimated preference for user X, item Y is 3.5 + 0.2 = 3.7.
+ * </p>
+ */
+public final class ItemUserAverageRecommender extends AbstractRecommender {
+  
+  private static final Logger log = LoggerFactory.getLogger(ItemUserAverageRecommender.class);
+  
+  private final FastByIDMap<RunningAverage> itemAverages;
+  private final FastByIDMap<RunningAverage> userAverages;
+  private final RunningAverage overallAveragePrefValue;
+  private final ReadWriteLock buildAveragesLock;
+  private final RefreshHelper refreshHelper;
+  
+  public ItemUserAverageRecommender(DataModel dataModel) throws TasteException {
+    super(dataModel);
+    this.itemAverages = new FastByIDMap<>();
+    this.userAverages = new FastByIDMap<>();
+    this.overallAveragePrefValue = new FullRunningAverage();
+    this.buildAveragesLock = new ReentrantReadWriteLock();
+    this.refreshHelper = new RefreshHelper(new Callable<Object>() {
+      @Override
+      public Object call() throws TasteException {
+        buildAverageDiffs();
+        return null;
+      }
+    });
+    refreshHelper.addDependency(dataModel);
+    buildAverageDiffs();
+  }
+  
+  @Override
+  public List<RecommendedItem> recommend(long userID, int howMany, IDRescorer rescorer, boolean includeKnownItems)
+    throws TasteException {
+    Preconditions.checkArgument(howMany >= 1, "howMany must be at least 1");
+    log.debug("Recommending items for user ID '{}'", userID);
+
+    PreferenceArray preferencesFromUser = getDataModel().getPreferencesFromUser(userID);
+    FastIDSet possibleItemIDs = getAllOtherItems(userID, preferencesFromUser, includeKnownItems);
+
+    TopItems.Estimator<Long> estimator = new Estimator(userID);
+
+    List<RecommendedItem> topItems = TopItems.getTopItems(howMany, possibleItemIDs.iterator(), rescorer,
+      estimator);
+
+    log.debug("Recommendations are: {}", topItems);
+    return topItems;
+  }
+  
+  @Override
+  public float estimatePreference(long userID, long itemID) throws TasteException {
+    DataModel dataModel = getDataModel();
+    Float actualPref = dataModel.getPreferenceValue(userID, itemID);
+    if (actualPref != null) {
+      return actualPref;
+    }
+    return doEstimatePreference(userID, itemID);
+  }
+  
+  private float doEstimatePreference(long userID, long itemID) {
+    buildAveragesLock.readLock().lock();
+    try {
+      RunningAverage itemAverage = itemAverages.get(itemID);
+      if (itemAverage == null) {
+        return Float.NaN;
+      }
+      RunningAverage userAverage = userAverages.get(userID);
+      if (userAverage == null) {
+        return Float.NaN;
+      }
+      double userDiff = userAverage.getAverage() - overallAveragePrefValue.getAverage();
+      return (float) (itemAverage.getAverage() + userDiff);
+    } finally {
+      buildAveragesLock.readLock().unlock();
+    }
+  }
+
+  private void buildAverageDiffs() throws TasteException {
+    try {
+      buildAveragesLock.writeLock().lock();
+      DataModel dataModel = getDataModel();
+      LongPrimitiveIterator it = dataModel.getUserIDs();
+      while (it.hasNext()) {
+        long userID = it.nextLong();
+        PreferenceArray prefs = dataModel.getPreferencesFromUser(userID);
+        int size = prefs.length();
+        for (int i = 0; i < size; i++) {
+          long itemID = prefs.getItemID(i);
+          float value = prefs.getValue(i);
+          addDatumAndCreateIfNeeded(itemID, value, itemAverages);
+          addDatumAndCreateIfNeeded(userID, value, userAverages);
+          overallAveragePrefValue.addDatum(value);
+        }
+      }
+    } finally {
+      buildAveragesLock.writeLock().unlock();
+    }
+  }
+  
+  private static void addDatumAndCreateIfNeeded(long itemID, float value, FastByIDMap<RunningAverage> averages) {
+    RunningAverage itemAverage = averages.get(itemID);
+    if (itemAverage == null) {
+      itemAverage = new FullRunningAverage();
+      averages.put(itemID, itemAverage);
+    }
+    itemAverage.addDatum(value);
+  }
+  
+  @Override
+  public void setPreference(long userID, long itemID, float value) throws TasteException {
+    DataModel dataModel = getDataModel();
+    double prefDelta;
+    try {
+      Float oldPref = dataModel.getPreferenceValue(userID, itemID);
+      prefDelta = oldPref == null ? value : value - oldPref;
+    } catch (NoSuchUserException nsee) {
+      prefDelta = value;
+    }
+    super.setPreference(userID, itemID, value);
+    try {
+      buildAveragesLock.writeLock().lock();
+      RunningAverage itemAverage = itemAverages.get(itemID);
+      if (itemAverage == null) {
+        RunningAverage newItemAverage = new FullRunningAverage();
+        newItemAverage.addDatum(prefDelta);
+        itemAverages.put(itemID, newItemAverage);
+      } else {
+        itemAverage.changeDatum(prefDelta);
+      }
+      RunningAverage userAverage = userAverages.get(userID);
+      if (userAverage == null) {
+        RunningAverage newUserAveragae = new FullRunningAverage();
+        newUserAveragae.addDatum(prefDelta);
+        userAverages.put(userID, newUserAveragae);
+      } else {
+        userAverage.changeDatum(prefDelta);
+      }
+      overallAveragePrefValue.changeDatum(prefDelta);
+    } finally {
+      buildAveragesLock.writeLock().unlock();
+    }
+  }
+  
+  @Override
+  public void removePreference(long userID, long itemID) throws TasteException {
+    DataModel dataModel = getDataModel();
+    Float oldPref = dataModel.getPreferenceValue(userID, itemID);
+    super.removePreference(userID, itemID);
+    if (oldPref != null) {
+      try {
+        buildAveragesLock.writeLock().lock();
+        RunningAverage itemAverage = itemAverages.get(itemID);
+        if (itemAverage == null) {
+          throw new IllegalStateException("No preferences exist for item ID: " + itemID);
+        }
+        itemAverage.removeDatum(oldPref);
+        RunningAverage userAverage = userAverages.get(userID);
+        if (userAverage == null) {
+          throw new IllegalStateException("No preferences exist for user ID: " + userID);
+        }
+        userAverage.removeDatum(oldPref);
+        overallAveragePrefValue.removeDatum(oldPref);
+      } finally {
+        buildAveragesLock.writeLock().unlock();
+      }
+    }
+  }
+  
+  @Override
+  public void refresh(Collection<Refreshable> alreadyRefreshed) {
+    refreshHelper.refresh(alreadyRefreshed);
+  }
+  
+  @Override
+  public String toString() {
+    return "ItemUserAverageRecommender";
+  }
+  
+  private final class Estimator implements TopItems.Estimator<Long> {
+    
+    private final long userID;
+    
+    private Estimator(long userID) {
+      this.userID = userID;
+    }
+    
+    @Override
+    public double estimate(Long itemID) {
+      return doEstimatePreference(userID, itemID);
+    }
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/mahout/blob/410ed16a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/NullRescorer.java
----------------------------------------------------------------------
diff --git a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/NullRescorer.java b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/NullRescorer.java
new file mode 100644
index 0000000..e0eda7a
--- /dev/null
+++ b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/NullRescorer.java
@@ -0,0 +1,86 @@
+/**
+ * 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.impl.recommender;
+
+import org.apache.mahout.cf.taste.recommender.IDRescorer;
+import org.apache.mahout.cf.taste.recommender.Rescorer;
+import org.apache.mahout.common.LongPair;
+
+/**
+ * <p>
+ * A simple {@link Rescorer} which always returns the original score.
+ * </p>
+ */
+public final class NullRescorer<T> implements Rescorer<T>, IDRescorer {
+  
+  private static final IDRescorer USER_OR_ITEM_INSTANCE = new NullRescorer<>();
+  private static final Rescorer<LongPair> ITEM_ITEM_PAIR_INSTANCE = new NullRescorer<>();
+  private static final Rescorer<LongPair> USER_USER_PAIR_INSTANCE = new NullRescorer<>();
+
+  private NullRescorer() {
+  }
+
+  public static IDRescorer getItemInstance() {
+    return USER_OR_ITEM_INSTANCE;
+  }
+  
+  public static IDRescorer getUserInstance() {
+    return USER_OR_ITEM_INSTANCE;
+  }
+  
+  public static Rescorer<LongPair> getItemItemPairInstance() {
+    return ITEM_ITEM_PAIR_INSTANCE;
+  }
+  
+  public static Rescorer<LongPair> getUserUserPairInstance() {
+    return USER_USER_PAIR_INSTANCE;
+  }
+
+  /**
+   * @param thing
+   *          to rescore
+   * @param originalScore
+   *          current score for item
+   * @return same originalScore as new score, always
+   */
+  @Override
+  public double rescore(T thing, double originalScore) {
+    return originalScore;
+  }
+  
+  @Override
+  public boolean isFiltered(T thing) {
+    return false;
+  }
+  
+  @Override
+  public double rescore(long id, double originalScore) {
+    return originalScore;
+  }
+  
+  @Override
+  public boolean isFiltered(long id) {
+    return false;
+  }
+  
+  @Override
+  public String toString() {
+    return "NullRescorer";
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/mahout/blob/410ed16a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/PreferredItemsNeighborhoodCandidateItemsStrategy.java
----------------------------------------------------------------------
diff --git a/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/PreferredItemsNeighborhoodCandidateItemsStrategy.java b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/PreferredItemsNeighborhoodCandidateItemsStrategy.java
new file mode 100644
index 0000000..6297d0b
--- /dev/null
+++ b/community/mahout-mr/mr/src/main/java/org/apache/mahout/cf/taste/impl/recommender/PreferredItemsNeighborhoodCandidateItemsStrategy.java
@@ -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.impl.recommender;
+
+import org.apache.mahout.cf.taste.common.TasteException;
+import org.apache.mahout.cf.taste.impl.common.FastIDSet;
+import org.apache.mahout.cf.taste.model.DataModel;
+import org.apache.mahout.cf.taste.model.PreferenceArray;
+
+public final class PreferredItemsNeighborhoodCandidateItemsStrategy extends AbstractCandidateItemsStrategy {
+
+  /**
+   * returns all items that have not been rated by the user and that were preferred by another user
+   * that has preferred at least one item that the current user has preferred too
+   */
+  @Override
+  protected FastIDSet doGetCandidateItems(long[] preferredItemIDs, DataModel dataModel, boolean includeKnownItems)
+    throws TasteException {
+    FastIDSet possibleItemsIDs = new FastIDSet();
+    for (long itemID : preferredItemIDs) {
+      PreferenceArray itemPreferences = dataModel.getPreferencesForItem(itemID);
+      int numUsersPreferringItem = itemPreferences.length();
+      for (int index = 0; index < numUsersPreferringItem; index++) {
+        possibleItemsIDs.addAll(dataModel.getItemIDsFromUser(itemPreferences.getUserID(index)));
+      }
+    }
+    if (!includeKnownItems) {
+      possibleItemsIDs.removeAll(preferredItemIDs);
+    }
+    return possibleItemsIDs;
+  }
+
+}