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/02/19 14:45:19 UTC

svn commit: r911810 - in /lucene/mahout/trunk: core/src/main/java/org/apache/mahout/common/ core/src/test/java/org/apache/mahout/cf/taste/common/ core/src/test/java/org/apache/mahout/cf/taste/impl/common/ core/src/test/java/org/apache/mahout/cf/taste/i...

Author: srowen
Date: Fri Feb 19 13:45:18 2010
New Revision: 911810

URL: http://svn.apache.org/viewvc?rev=911810&view=rev
Log:
Add several more unit tests for cf.taste. Make random numbers all but guaranteed to be deterministic during unit tests, to allow for repeatable tests of components with randomness.

Added:
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/RandomWrapper.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/BitSetTest.java
      - copied, changed from r910395, lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/CacheTest.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/InvertedRunningAverageTest.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/LongPrimitiveArrayIteratorTest.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/SamplingLongPrimitiveIteratorTest.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/BooleanItemPreferenceArrayTest.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/BooleanUserPreferenceArrayTest.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/GenericItemPreferenceArrayTest.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/GenericUserPreferenceArrayTest.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/MemoryIDMigratorTest.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/recommender/ItemAverageRecommenderTest.java
      - copied, changed from r911233, lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/recommender/GenericUserBasedRecommenderTest.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/recommender/ItemUserAverageRecommenderTest.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/recommender/RandomRecommenderTest.java
Modified:
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/RandomUtils.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/common/CommonTest.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/hadoop/DatasetSplit.java
    lucene/mahout/trunk/examples/src/test/java/org/apache/mahout/ga/watchmaker/cd/hadoop/DatasetSplitTest.java

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/RandomUtils.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/RandomUtils.java?rev=911810&r1=911809&r2=911810&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/RandomUtils.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/RandomUtils.java Fri Feb 19 13:45:18 2010
@@ -17,10 +17,10 @@
 
 package org.apache.mahout.common;
 
-import java.nio.charset.Charset;
+import java.util.Collections;
+import java.util.Map;
 import java.util.Random;
-
-import org.uncommons.maths.random.MersenneTwisterRNG;
+import java.util.WeakHashMap;
 
 /**
  * <p>
@@ -33,27 +33,34 @@
  * </p>
  */
 public final class RandomUtils {
-  
-  private static final byte[] STANDARD_SEED = "Mahout=Hadoop+ML".getBytes(Charset.forName("US-ASCII"));
-  
-  private static boolean testSeed;
-  
+
   /** The largest prime less than 2<sup>31</sup>-1 that is the smaller of a twin prime pair. */
   public static final int MAX_INT_SMALLER_TWIN_PRIME = 2147482949;
+
+  private static final Map<RandomWrapper,Boolean> instances =
+      Collections.synchronizedMap(new WeakHashMap<RandomWrapper,Boolean>());
   
   private RandomUtils() { }
   
   public static void useTestSeed() {
-    testSeed = true;
+    RandomWrapper.useTestSeed();
+    synchronized (instances) {
+      for (RandomWrapper rng : instances.keySet()) {
+        rng.reset();
+      }
+    }
   }
   
   public static Random getRandom() {
-    return testSeed ? new MersenneTwisterRNG(STANDARD_SEED)
-        : new MersenneTwisterRNG();
+    RandomWrapper random = new RandomWrapper();
+    instances.put(random, Boolean.TRUE);
+    return random;
   }
   
   public static Random getRandom(long seed) {
-    return new MersenneTwisterRNG(longSeedtoBytes(seed));
+    RandomWrapper random = new RandomWrapper(seed);
+    instances.put(random, Boolean.TRUE);
+    return random;
   }
   
   public static byte[] longSeedtoBytes(long seed) {
@@ -78,11 +85,11 @@
   
   /** @return what {@link Double#hashCode()} would return for the same value */
   public static int hashDouble(double value) {
-    // Just copied from Double.hashCode
     long bits = Double.doubleToLongBits(value);
     return (int) (bits ^ bits >>> 32);
   }
-  
+
+  /** @return what {@link Float#hashCode()} would return for the same value */
   public static int hashFloat(float value) {
     return Float.floatToIntBits(value);
   }

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/RandomWrapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/RandomWrapper.java?rev=911810&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/RandomWrapper.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/RandomWrapper.java Fri Feb 19 13:45:18 2010
@@ -0,0 +1,126 @@
+/**
+ * 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.common;
+
+import org.uncommons.maths.random.MersenneTwisterRNG;
+
+import java.nio.charset.Charset;
+import java.util.Random;
+
+public final class RandomWrapper extends Random {
+
+  private static final byte[] STANDARD_SEED = "Mahout=Hadoop+ML".getBytes(Charset.forName("US-ASCII"));
+
+  private static boolean testSeed;
+
+  private Random random;
+  private final Long fixedSeed;
+
+  RandomWrapper() {
+    this.fixedSeed = null;
+    random = buildRandom();
+  }
+
+  RandomWrapper(long fixedSeed) {
+    this.fixedSeed = fixedSeed;
+    random = buildRandom();
+  }
+
+  static void useTestSeed() {
+    testSeed = true;
+  }
+
+  private Random buildRandom() {
+    if (testSeed) {
+      return new MersenneTwisterRNG(STANDARD_SEED);
+    } else if (fixedSeed == null) {
+      return new MersenneTwisterRNG();
+    } else {
+      return new MersenneTwisterRNG(RandomUtils.longSeedtoBytes(fixedSeed));
+    }
+  }
+
+  public Random getRandom() {
+    return random;
+  }
+
+  void reset() {
+    random = buildRandom();
+  }
+
+  public long getSeed() {
+    return RandomUtils.seedBytesToLong(((MersenneTwisterRNG) random).getSeed());
+  }
+
+  @Override
+  public void setSeed(long seed) {
+    // Since this will be called by the java.util.Random() constructor before we construct
+    // the delegate... and because we don't actually care about the result of this for our
+    // purpose:
+    if (random != null) {
+      random.setSeed(seed);
+    }
+  }
+
+  @Override
+  protected int next(int bits) {
+    // Ugh, can't delegate this method -- it's protected
+    // Callers can't use it and other methods are delegated, so shouldn't matter
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void nextBytes(byte[] bytes) {
+    random.nextBytes(bytes);
+  }
+
+  @Override
+  public int nextInt() {
+    return random.nextInt();
+  }
+
+  @Override
+  public int nextInt(int n) {
+    return random.nextInt(n);
+  }
+
+  @Override
+  public long nextLong() {
+    return random.nextLong();
+  }
+
+  @Override
+  public boolean nextBoolean() {
+    return random.nextBoolean();
+  }
+
+  @Override
+  public float nextFloat() {
+    return random.nextFloat();
+  }
+
+  @Override
+  public double nextDouble() {
+    return random.nextDouble();
+  }
+
+  @Override
+  public double nextGaussian() {
+    return random.nextGaussian();
+  }
+}

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/common/CommonTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/common/CommonTest.java?rev=911810&r1=911809&r2=911810&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/common/CommonTest.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/common/CommonTest.java Fri Feb 19 13:45:18 2010
@@ -27,7 +27,7 @@
 /** <p>Tests common classes.</p> */
 public final class CommonTest extends TasteTestCase {
 
-  public void testException() {
+  public void testTasteException() {
     // Just make sure this all doesn't, ah, throw an exception
     TasteException te1 = new TasteException();
     TasteException te2 = new TasteException(te1);
@@ -37,4 +37,20 @@
     te4.printStackTrace(new PrintWriter(new OutputStreamWriter(new ByteArrayOutputStream())));
   }
 
+  public void testNSUException() {
+    // Just make sure this all doesn't, ah, throw an exception
+    TasteException te1 = new NoSuchUserException();
+    TasteException te4 = new NoSuchUserException(te1.toString());
+    te4.printStackTrace(new PrintStream(new ByteArrayOutputStream()));
+    te4.printStackTrace(new PrintWriter(new OutputStreamWriter(new ByteArrayOutputStream())));
+  }
+
+  public void testNSIException() {
+    // Just make sure this all doesn't, ah, throw an exception
+    TasteException te1 = new NoSuchItemException();
+    TasteException te4 = new NoSuchItemException(te1.toString());
+    te4.printStackTrace(new PrintStream(new ByteArrayOutputStream()));
+    te4.printStackTrace(new PrintWriter(new OutputStreamWriter(new ByteArrayOutputStream())));
+  }
+
 }

Copied: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/BitSetTest.java (from r910395, lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/CacheTest.java)
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/BitSetTest.java?p2=lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/BitSetTest.java&p1=lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/CacheTest.java&r1=910395&r2=911810&rev=911810&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/CacheTest.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/BitSetTest.java Fri Feb 19 13:45:18 2010
@@ -17,42 +17,58 @@
 
 package org.apache.mahout.cf.taste.impl.common;
 
-import org.apache.mahout.cf.taste.common.TasteException;
 import org.apache.mahout.cf.taste.impl.TasteTestCase;
-import org.apache.mahout.common.RandomUtils;
 
-import java.util.Random;
+public final class BitSetTest extends TasteTestCase {
 
-public final class CacheTest extends TasteTestCase {
-  
-  public void testLotsOfGets() throws TasteException {
-    Retriever<Object,Object> retriever = new IdentityRetriever();
-    Cache<Object,Object> cache = new Cache<Object,Object>(retriever, 1000);
-    for (int i = 0; i < 1000000; i++) {
-      assertEquals(i, cache.get(i));
+  private static final int NUM_BITS = 100;
+
+  public void testGetSet() {
+    BitSet bitSet = new BitSet(NUM_BITS);
+    for (int i = 0; i < NUM_BITS; i++) {
+      assertFalse(bitSet.get(i));
     }
+    bitSet.set(0);
+    bitSet.set(NUM_BITS-1);
+    assertTrue(bitSet.get(0));
+    assertTrue(bitSet.get(NUM_BITS-1));
   }
-  
-  public void testMixedUsage() throws TasteException {
-    Random random = RandomUtils.getRandom();
-    Retriever<Object,Object> retriever = new IdentityRetriever();
-    Cache<Object,Object> cache = new Cache<Object,Object>(retriever, 1000);
-    for (int i = 0; i < 1000000; i++) {
-      double r = random.nextDouble();
-      if (r < 0.01) {
-        cache.clear();
-      } else if (r < 0.1) {
-        cache.remove(r - 100);
-      } else {
-        assertEquals(i, cache.get(i));
-      }
+
+  public void testBounds() {
+    BitSet bitSet = new BitSet(NUM_BITS);
+    try {
+      bitSet.set(1000);
+      fail("Should have thrown exception");
+    } catch (ArrayIndexOutOfBoundsException aioobe) {
+      // continue
+    }
+    try {
+      bitSet.set(-1);
+      fail("Should have thrown exception");
+    } catch (ArrayIndexOutOfBoundsException aioobe) {
+      // continue
     }
   }
-  
-  private static class IdentityRetriever implements Retriever<Object,Object> {
-    @Override
-    public Object get(Object key) {
-      return key;
+
+  public void testClear() {
+    BitSet bitSet = new BitSet(NUM_BITS);
+    for (int i = 0; i < NUM_BITS; i++) {
+      bitSet.set(i);
+    }
+    for (int i = 0; i < NUM_BITS; i++) {
+      assertTrue(bitSet.get(i));
     }
+    bitSet.clear();
+    for (int i = 0; i < NUM_BITS; i++) {
+      assertFalse(bitSet.get(i));
+    }
+  }
+
+  public void testClone() {
+    BitSet bitSet = new BitSet(NUM_BITS);
+    bitSet.set(NUM_BITS-1);
+    bitSet = bitSet.clone();
+    assertTrue(bitSet.get(NUM_BITS-1));
   }
+
 }
\ No newline at end of file

Added: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/InvertedRunningAverageTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/InvertedRunningAverageTest.java?rev=911810&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/InvertedRunningAverageTest.java (added)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/InvertedRunningAverageTest.java Fri Feb 19 13:45:18 2010
@@ -0,0 +1,95 @@
+/**
+ * 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.common;
+
+import org.apache.mahout.cf.taste.impl.TasteTestCase;
+
+public final class InvertedRunningAverageTest extends TasteTestCase {
+
+  public void testAverage() {
+    RunningAverage avg = new FullRunningAverage();
+    InvertedRunningAverage inverted = new InvertedRunningAverage(avg);
+    assertEquals(0, inverted.getCount());
+    avg.addDatum(1.0);
+    assertEquals(1, inverted.getCount());
+    assertEquals(-1.0, inverted.getAverage());
+    avg.addDatum(2.0);
+    assertEquals(2, inverted.getCount());
+    assertEquals(-1.5, inverted.getAverage());
+  }
+
+  public void testUnsupported() {
+    RunningAverage avg = new FullRunningAverage();
+    InvertedRunningAverage inverted = new InvertedRunningAverage(avg);
+    try {
+      inverted.addDatum(1.0);
+      fail("Should have thrown exception");
+    } catch (UnsupportedOperationException uoe) {
+      // good
+    }
+    try {
+      inverted.changeDatum(1.0);
+      fail("Should have thrown exception");
+    } catch (UnsupportedOperationException uoe) {
+      // good
+    }
+    try {
+      inverted.removeDatum(1.0);
+      fail("Should have thrown exception");
+    } catch (UnsupportedOperationException uoe) {
+      // good
+    }
+  }
+
+  public void testAverageAndStdDev() {
+    RunningAverageAndStdDev avg = new FullRunningAverageAndStdDev();
+    InvertedRunningAverageAndStdDev inverted = new InvertedRunningAverageAndStdDev(avg);
+    assertEquals(0, inverted.getCount());
+    avg.addDatum(1.0);
+    assertEquals(1, inverted.getCount());
+    assertEquals(-1.0, inverted.getAverage());
+    avg.addDatum(2.0);
+    assertEquals(2, inverted.getCount());
+    assertEquals(-1.5, inverted.getAverage());
+    assertEquals(Math.sqrt(2.0)/2.0, inverted.getStandardDeviation());
+  }
+
+  public void testAndStdDevUnsupported() {
+    RunningAverageAndStdDev avg = new FullRunningAverageAndStdDev();
+    InvertedRunningAverageAndStdDev inverted = new InvertedRunningAverageAndStdDev(avg);
+    try {
+      inverted.addDatum(1.0);
+      fail("Should have thrown exception");
+    } catch (UnsupportedOperationException uoe) {
+      // good
+    }
+    try {
+      inverted.changeDatum(1.0);
+      fail("Should have thrown exception");
+    } catch (UnsupportedOperationException uoe) {
+      // good
+    }
+    try {
+      inverted.removeDatum(1.0);
+      fail("Should have thrown exception");
+    } catch (UnsupportedOperationException uoe) {
+      // good
+    }
+  }
+
+}
\ No newline at end of file

Added: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/LongPrimitiveArrayIteratorTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/LongPrimitiveArrayIteratorTest.java?rev=911810&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/LongPrimitiveArrayIteratorTest.java (added)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/LongPrimitiveArrayIteratorTest.java Fri Feb 19 13:45:18 2010
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.mahout.cf.taste.impl.common;
+
+import org.apache.mahout.cf.taste.impl.TasteTestCase;
+
+import java.util.NoSuchElementException;
+
+public final class LongPrimitiveArrayIteratorTest extends TasteTestCase {
+
+  public void testEmpty() {
+    LongPrimitiveArrayIterator it = new LongPrimitiveArrayIterator(new long[0]);
+    assertFalse(it.hasNext());
+    try {
+      it.next();
+      fail("Should have thrown exception");
+    } catch (NoSuchElementException nsee) {
+      // good
+    }
+  }
+
+  public void testNext() {
+    LongPrimitiveArrayIterator it = new LongPrimitiveArrayIterator(new long[] {3,2,1});
+    assertTrue(it.hasNext());
+    assertEquals(3, (long) it.next());
+    assertTrue(it.hasNext());
+    assertEquals(2, it.nextLong());
+    assertTrue(it.hasNext());
+    assertEquals(1, (long) it.next());    
+    assertFalse(it.hasNext());
+    try {
+      it.nextLong();
+      fail("Should have thrown exception");
+    } catch (NoSuchElementException nsee) {
+      // good
+    }
+  }
+
+  public void testPeekSkip() {
+    LongPrimitiveArrayIterator it = new LongPrimitiveArrayIterator(new long[] {3,2,1});
+    assertEquals(3, it.peek());
+    it.skip(2);
+    assertEquals(1, it.nextLong());
+    assertFalse(it.hasNext());
+  }
+
+}
\ No newline at end of file

Added: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/SamplingLongPrimitiveIteratorTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/SamplingLongPrimitiveIteratorTest.java?rev=911810&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/SamplingLongPrimitiveIteratorTest.java (added)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/common/SamplingLongPrimitiveIteratorTest.java Fri Feb 19 13:45:18 2010
@@ -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.common;
+
+import org.apache.mahout.cf.taste.impl.TasteTestCase;
+
+import java.util.NoSuchElementException;
+
+public final class SamplingLongPrimitiveIteratorTest extends TasteTestCase {
+
+  public void testEmpty() {
+    LongPrimitiveArrayIterator it = new LongPrimitiveArrayIterator(new long[0]);
+    SamplingLongPrimitiveIterator sample = new SamplingLongPrimitiveIterator(it, 0.5);
+    assertFalse(sample.hasNext());
+    try {
+      sample.next();
+      fail("Should have thrown exception");
+    } catch (NoSuchElementException nsee) {
+      // good
+    }
+  }
+
+  public void testNext() {
+    LongPrimitiveArrayIterator it = new LongPrimitiveArrayIterator(new long[] {5,4,3,2,1});
+    SamplingLongPrimitiveIterator sample = new SamplingLongPrimitiveIterator(it, 0.5);
+    assertTrue(sample.hasNext());
+    assertEquals(4, (long) sample.next());
+    assertTrue(sample.hasNext());
+    assertEquals(2, sample.nextLong());
+    assertTrue(sample.hasNext());
+    assertEquals(1, (long) sample.next());
+    assertFalse(sample.hasNext());
+    try {
+      it.nextLong();
+      fail("Should have thrown exception");
+    } catch (NoSuchElementException nsee) {
+      // good
+    }
+  }
+
+  public void testPeekSkip() {
+    LongPrimitiveArrayIterator it = new LongPrimitiveArrayIterator(new long[] {8,7,6,5,4,3,2,1});
+    SamplingLongPrimitiveIterator sample = new SamplingLongPrimitiveIterator(it, 0.5);
+    assertEquals(7, sample.peek());
+    sample.skip(1);
+    assertEquals(4, sample.peek());
+    assertTrue(sample.hasNext());
+  }
+
+}
\ No newline at end of file

Added: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/BooleanItemPreferenceArrayTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/BooleanItemPreferenceArrayTest.java?rev=911810&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/BooleanItemPreferenceArrayTest.java (added)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/BooleanItemPreferenceArrayTest.java Fri Feb 19 13:45:18 2010
@@ -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.model;
+
+import org.apache.mahout.common.MahoutTestCase;
+
+public final class BooleanItemPreferenceArrayTest extends MahoutTestCase {
+
+  public void testUserID() {
+    BooleanItemPreferenceArray prefs = new BooleanItemPreferenceArray(3);
+    assertEquals(3, prefs.length());
+    prefs.setItemID(0, 1L);
+    assertEquals(1L, prefs.getItemID(0));
+    assertEquals(1L, prefs.getItemID(1));
+    assertEquals(1L, prefs.getItemID(2));
+  }
+
+  public void testItemID() {
+    BooleanItemPreferenceArray prefs = new BooleanItemPreferenceArray(3);
+    assertEquals(3, prefs.length());
+    prefs.setUserID(0, 1L);
+    prefs.setUserID(1, 2L);
+    prefs.setUserID(2, 3L);
+    assertEquals(1L, prefs.getUserID(0));
+    assertEquals(2L, prefs.getUserID(1));
+    assertEquals(3L, prefs.getUserID(2));
+  }
+
+  public void testSetValue() {
+    BooleanItemPreferenceArray prefs = new BooleanItemPreferenceArray(3);
+    assertEquals(3, prefs.length());
+    try {
+      prefs.setValue(0, 1.0f);
+      fail("Should have thrown exception");
+    } catch (UnsupportedOperationException uoe) {
+      // good
+    }
+    assertEquals(1.0f, prefs.getValue(2));
+  }
+
+  public void testHasPref() {
+    BooleanItemPreferenceArray prefs = new BooleanItemPreferenceArray(3);
+    prefs.set(0, new GenericPreference(1L, 3L, 5.0f));
+    assertTrue(prefs.hasPrefWithItemID(3L));
+    assertTrue(prefs.hasPrefWithUserID(1L));
+    assertFalse(prefs.hasPrefWithItemID(2L));
+    assertFalse(prefs.hasPrefWithUserID(2L));
+  }
+
+  public void testSort() {
+    BooleanItemPreferenceArray prefs = new BooleanItemPreferenceArray(3);
+    prefs.set(0, new GenericPreference(3L, 1L, 5.0f));
+    prefs.set(1, new GenericPreference(1L, 1L, 5.0f));
+    prefs.set(2, new GenericPreference(2L, 1L, 5.0f));
+    prefs.sortByUser();
+    assertEquals(1L, prefs.getUserID(0));
+    assertEquals(2L, prefs.getUserID(1));
+    assertEquals(3L, prefs.getUserID(2));
+  }
+
+  public void testClone() {
+    BooleanItemPreferenceArray prefs = new BooleanItemPreferenceArray(3);
+    prefs.set(0, new BooleanPreference(3L, 1L));
+    prefs.set(1, new BooleanPreference(1L, 1L));
+    prefs.set(2, new BooleanPreference(2L, 1L));
+    prefs = prefs.clone();
+    assertEquals(3L, prefs.getUserID(0));
+    assertEquals(1L, prefs.getItemID(1));
+  }
+
+}
\ No newline at end of file

Added: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/BooleanUserPreferenceArrayTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/BooleanUserPreferenceArrayTest.java?rev=911810&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/BooleanUserPreferenceArrayTest.java (added)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/BooleanUserPreferenceArrayTest.java Fri Feb 19 13:45:18 2010
@@ -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.model;
+
+import org.apache.mahout.common.MahoutTestCase;
+
+public final class BooleanUserPreferenceArrayTest extends MahoutTestCase {
+
+  public void testUserID() {
+    BooleanUserPreferenceArray prefs = new BooleanUserPreferenceArray(3);
+    assertEquals(3, prefs.length());
+    prefs.setUserID(0, 1L);
+    assertEquals(1L, prefs.getUserID(0));
+    assertEquals(1L, prefs.getUserID(1));
+    assertEquals(1L, prefs.getUserID(2));
+  }
+
+  public void testItemID() {
+    BooleanUserPreferenceArray prefs = new BooleanUserPreferenceArray(3);
+    assertEquals(3, prefs.length());
+    prefs.setItemID(0, 1L);
+    prefs.setItemID(1, 2L);
+    prefs.setItemID(2, 3L);
+    assertEquals(1L, prefs.getItemID(0));
+    assertEquals(2L, prefs.getItemID(1));
+    assertEquals(3L, prefs.getItemID(2));
+  }
+
+  public void testSetValue() {
+    BooleanUserPreferenceArray prefs = new BooleanUserPreferenceArray(3);
+    assertEquals(3, prefs.length());
+    try {
+      prefs.setValue(0, 1.0f);
+      fail("Should have thrown exception");
+    } catch (UnsupportedOperationException uoe) {
+      // good
+    }
+    assertEquals(1.0f, prefs.getValue(2));
+  }
+
+  public void testHasPref() {
+    BooleanUserPreferenceArray prefs = new BooleanUserPreferenceArray(3);
+    prefs.set(0, new GenericPreference(1L, 3L, 5.0f));
+    assertTrue(prefs.hasPrefWithItemID(3L));
+    assertTrue(prefs.hasPrefWithUserID(1L));
+    assertFalse(prefs.hasPrefWithItemID(2L));
+    assertFalse(prefs.hasPrefWithUserID(2L));
+  }
+
+  public void testSort() {
+    BooleanUserPreferenceArray prefs = new BooleanUserPreferenceArray(3);
+    prefs.set(0, new BooleanPreference(1L, 3L));
+    prefs.set(1, new BooleanPreference(1L, 1L));
+    prefs.set(2, new BooleanPreference(1L, 2L));
+    prefs.sortByItem();
+    assertEquals(1L, prefs.getItemID(0));
+    assertEquals(2L, prefs.getItemID(1));
+    assertEquals(3L, prefs.getItemID(2));
+  }
+
+  public void testClone() {
+    BooleanUserPreferenceArray prefs = new BooleanUserPreferenceArray(3);
+    prefs.set(0, new BooleanPreference(1L, 3L));
+    prefs.set(1, new BooleanPreference(1L, 1L));
+    prefs.set(2, new BooleanPreference(1L, 2L));
+    prefs = prefs.clone();
+    assertEquals(3L, prefs.getItemID(0));
+    assertEquals(1L, prefs.getUserID(1));
+  }
+
+}
\ No newline at end of file

Added: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/GenericItemPreferenceArrayTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/GenericItemPreferenceArrayTest.java?rev=911810&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/GenericItemPreferenceArrayTest.java (added)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/GenericItemPreferenceArrayTest.java Fri Feb 19 13:45:18 2010
@@ -0,0 +1,101 @@
+/**
+ * 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.model;
+
+import org.apache.mahout.common.MahoutTestCase;
+
+public final class GenericItemPreferenceArrayTest extends MahoutTestCase {
+
+  public void testUserID() {
+    GenericItemPreferenceArray prefs = new GenericItemPreferenceArray(3);
+    assertEquals(3, prefs.length());
+    prefs.setItemID(0, 1L);
+    assertEquals(1L, prefs.getItemID(0));
+    assertEquals(1L, prefs.getItemID(1));
+    assertEquals(1L, prefs.getItemID(2));
+  }
+
+  public void testItemID() {
+    GenericItemPreferenceArray prefs = new GenericItemPreferenceArray(3);
+    assertEquals(3, prefs.length());
+    prefs.setUserID(0, 1L);
+    prefs.setUserID(1, 2L);
+    prefs.setUserID(2, 3L);
+    assertEquals(1L, prefs.getUserID(0));
+    assertEquals(2L, prefs.getUserID(1));
+    assertEquals(3L, prefs.getUserID(2));    
+  }
+
+  public void testSetValue() {
+    GenericItemPreferenceArray prefs = new GenericItemPreferenceArray(3);
+    assertEquals(3, prefs.length());
+    prefs.setValue(0, 1.0f);
+    prefs.setValue(1, 2.0f);
+    prefs.setValue(2, 3.0f);
+    assertEquals(1.0f, prefs.getValue(0));
+    assertEquals(2.0f, prefs.getValue(1));
+    assertEquals(3.0f, prefs.getValue(2));
+  }
+
+  public void testHasPref() {
+    GenericItemPreferenceArray prefs = new GenericItemPreferenceArray(3);
+    prefs.set(0, new GenericPreference(1L, 3L, 5.0f));
+    assertTrue(prefs.hasPrefWithItemID(3L));
+    assertTrue(prefs.hasPrefWithUserID(1L));
+    assertFalse(prefs.hasPrefWithItemID(2L));
+    assertFalse(prefs.hasPrefWithUserID(2L));
+  }
+
+  public void testSort() {
+    GenericItemPreferenceArray prefs = new GenericItemPreferenceArray(3);
+    prefs.set(0, new GenericPreference(3L, 1L, 5.0f));
+    prefs.set(1, new GenericPreference(1L, 1L, 5.0f));
+    prefs.set(2, new GenericPreference(2L, 1L, 5.0f));
+    prefs.sortByUser();
+    assertEquals(1L, prefs.getUserID(0));
+    assertEquals(2L, prefs.getUserID(1));
+    assertEquals(3L, prefs.getUserID(2));
+  }
+
+  public void testSortValue() {
+    GenericItemPreferenceArray prefs = new GenericItemPreferenceArray(3);
+    prefs.set(0, new GenericPreference(3L, 1L, 5.0f));
+    prefs.set(1, new GenericPreference(1L, 1L, 4.0f));
+    prefs.set(2, new GenericPreference(2L, 1L, 3.0f));
+    prefs.sortByValue();
+    assertEquals(2L, prefs.getUserID(0));
+    assertEquals(1L, prefs.getUserID(1));
+    assertEquals(3L, prefs.getUserID(2));
+    prefs.sortByValueReversed();
+    assertEquals(3L, prefs.getUserID(0));
+    assertEquals(1L, prefs.getUserID(1));
+    assertEquals(2L, prefs.getUserID(2));
+  }
+
+  public void testClone() {
+    GenericItemPreferenceArray prefs = new GenericItemPreferenceArray(3);
+    prefs.set(0, new GenericPreference(3L, 1L, 5.0f));
+    prefs.set(1, new GenericPreference(1L, 1L, 4.0f));
+    prefs.set(2, new GenericPreference(2L, 1L, 3.0f));
+    prefs = prefs.clone();
+    assertEquals(3L, prefs.getUserID(0));
+    assertEquals(1L, prefs.getItemID(1));
+    assertEquals(3.0f, prefs.getValue(2));
+  }
+
+}
\ No newline at end of file

Added: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/GenericUserPreferenceArrayTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/GenericUserPreferenceArrayTest.java?rev=911810&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/GenericUserPreferenceArrayTest.java (added)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/GenericUserPreferenceArrayTest.java Fri Feb 19 13:45:18 2010
@@ -0,0 +1,101 @@
+/**
+ * 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.model;
+
+import org.apache.mahout.common.MahoutTestCase;
+
+public final class GenericUserPreferenceArrayTest extends MahoutTestCase {
+
+  public void testUserID() {
+    GenericUserPreferenceArray prefs = new GenericUserPreferenceArray(3);
+    assertEquals(3, prefs.length());
+    prefs.setUserID(0, 1L);
+    assertEquals(1L, prefs.getUserID(0));
+    assertEquals(1L, prefs.getUserID(1));
+    assertEquals(1L, prefs.getUserID(2));
+  }
+
+  public void testItemID() {
+    GenericUserPreferenceArray prefs = new GenericUserPreferenceArray(3);
+    assertEquals(3, prefs.length());
+    prefs.setItemID(0, 1L);
+    prefs.setItemID(1, 2L);
+    prefs.setItemID(2, 3L);
+    assertEquals(1L, prefs.getItemID(0));
+    assertEquals(2L, prefs.getItemID(1));
+    assertEquals(3L, prefs.getItemID(2));    
+  }
+
+  public void testSetValue() {
+    GenericUserPreferenceArray prefs = new GenericUserPreferenceArray(3);
+    assertEquals(3, prefs.length());
+    prefs.setValue(0, 1.0f);
+    prefs.setValue(1, 2.0f);
+    prefs.setValue(2, 3.0f);
+    assertEquals(1.0f, prefs.getValue(0));
+    assertEquals(2.0f, prefs.getValue(1));
+    assertEquals(3.0f, prefs.getValue(2));    
+  }
+
+  public void testHasPref() {
+    GenericUserPreferenceArray prefs = new GenericUserPreferenceArray(3);
+    prefs.set(0, new GenericPreference(1L, 3L, 5.0f));
+    assertTrue(prefs.hasPrefWithItemID(3L));
+    assertTrue(prefs.hasPrefWithUserID(1L));
+    assertFalse(prefs.hasPrefWithItemID(2L));
+    assertFalse(prefs.hasPrefWithUserID(2L));
+  }
+
+  public void testSort() {
+    GenericUserPreferenceArray prefs = new GenericUserPreferenceArray(3);
+    prefs.set(0, new GenericPreference(1L, 3L, 5.0f));
+    prefs.set(1, new GenericPreference(1L, 1L, 5.0f));
+    prefs.set(2, new GenericPreference(1L, 2L, 5.0f));
+    prefs.sortByItem();
+    assertEquals(1L, prefs.getItemID(0));
+    assertEquals(2L, prefs.getItemID(1));
+    assertEquals(3L, prefs.getItemID(2));    
+  }
+
+  public void testSortValue() {
+    GenericUserPreferenceArray prefs = new GenericUserPreferenceArray(3);
+    prefs.set(0, new GenericPreference(1L, 3L, 5.0f));
+    prefs.set(1, new GenericPreference(1L, 1L, 4.0f));
+    prefs.set(2, new GenericPreference(1L, 2L, 3.0f));
+    prefs.sortByValue();
+    assertEquals(2L, prefs.getItemID(0));
+    assertEquals(1L, prefs.getItemID(1));
+    assertEquals(3L, prefs.getItemID(2));
+    prefs.sortByValueReversed();
+    assertEquals(3L, prefs.getItemID(0));
+    assertEquals(1L, prefs.getItemID(1));
+    assertEquals(2L, prefs.getItemID(2));
+  }
+
+  public void testClone() {
+    GenericUserPreferenceArray prefs = new GenericUserPreferenceArray(3);
+    prefs.set(0, new GenericPreference(1L, 3L, 5.0f));
+    prefs.set(1, new GenericPreference(1L, 1L, 4.0f));
+    prefs.set(2, new GenericPreference(1L, 2L, 3.0f));
+    prefs = prefs.clone();
+    assertEquals(3L, prefs.getItemID(0));
+    assertEquals(1L, prefs.getUserID(1));
+    assertEquals(3.0f, prefs.getValue(2));
+  }
+
+}
\ No newline at end of file

Added: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/MemoryIDMigratorTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/MemoryIDMigratorTest.java?rev=911810&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/MemoryIDMigratorTest.java (added)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/MemoryIDMigratorTest.java Fri Feb 19 13:45:18 2010
@@ -0,0 +1,52 @@
+/**
+ * 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.model;
+
+import org.apache.mahout.cf.taste.model.IDMigrator;
+import org.apache.mahout.common.MahoutTestCase;
+
+import java.util.Collections;
+
+public final class MemoryIDMigratorTest extends MahoutTestCase {
+
+  private static final String DUMMY_STRING = "Mahout";
+  private static final long DUMMY_ID = -6311185995763544451L;
+
+  public void testToLong() {
+    IDMigrator migrator = new MemoryIDMigrator();
+    long id = migrator.toLongID(DUMMY_STRING);
+    assertEquals(DUMMY_ID, id);
+  }
+
+  public void testStore() throws Exception {
+    IDMigrator migrator = new MemoryIDMigrator();
+    long id = migrator.toLongID(DUMMY_STRING);
+    assertNull(migrator.toStringID(id));
+    migrator.storeMapping(id, DUMMY_STRING);
+    assertEquals(DUMMY_STRING, migrator.toStringID(id));
+  }
+
+  public void testInitialize() throws Exception {
+    IDMigrator migrator = new MemoryIDMigrator();
+    long id = migrator.toLongID(DUMMY_STRING);
+    assertNull(migrator.toStringID(id));
+    migrator.initialize(Collections.singleton(DUMMY_STRING));
+    assertEquals(DUMMY_STRING, migrator.toStringID(id));
+  }
+
+}

Copied: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/recommender/ItemAverageRecommenderTest.java (from r911233, lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/recommender/GenericUserBasedRecommenderTest.java)
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/recommender/ItemAverageRecommenderTest.java?p2=lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/recommender/ItemAverageRecommenderTest.java&p1=lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/recommender/GenericUserBasedRecommenderTest.java&r1=911233&r2=911810&rev=911810&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/recommender/GenericUserBasedRecommenderTest.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/recommender/ItemAverageRecommenderTest.java Fri Feb 19 13:45:18 2010
@@ -17,128 +17,25 @@
 
 package org.apache.mahout.cf.taste.impl.recommender;
 
-import org.apache.mahout.cf.taste.common.TasteException;
 import org.apache.mahout.cf.taste.impl.TasteTestCase;
-import org.apache.mahout.cf.taste.impl.neighborhood.NearestNUserNeighborhood;
-import org.apache.mahout.cf.taste.impl.similarity.PearsonCorrelationSimilarity;
-import org.apache.mahout.cf.taste.model.DataModel;
-import org.apache.mahout.cf.taste.neighborhood.UserNeighborhood;
 import org.apache.mahout.cf.taste.recommender.RecommendedItem;
 import org.apache.mahout.cf.taste.recommender.Recommender;
-import org.apache.mahout.cf.taste.recommender.UserBasedRecommender;
-import org.apache.mahout.cf.taste.similarity.UserSimilarity;
 
 import java.util.List;
 
-/** <p>Tests {@link GenericUserBasedRecommender}.</p> */
-public final class GenericUserBasedRecommenderTest extends TasteTestCase {
+public final class ItemAverageRecommenderTest extends TasteTestCase {
 
   public void testRecommender() throws Exception {
-    Recommender recommender = buildRecommender();
+    Recommender recommender = new ItemAverageRecommender(getDataModel());
     List<RecommendedItem> recommended = recommender.recommend(1, 1);
     assertNotNull(recommended);
     assertEquals(1, recommended.size());
     RecommendedItem firstRecommended = recommended.get(0);
     assertEquals(2, firstRecommended.getItemID());
-    assertEquals(0.3f, firstRecommended.getValue());
+    assertEquals(0.53333336f, firstRecommended.getValue());
     recommender.refresh(null);
     assertEquals(2, firstRecommended.getItemID());
-    assertEquals(0.3f, firstRecommended.getValue());
+    assertEquals(0.53333336f, firstRecommended.getValue());
   }
 
-  public void testHowMany() throws Exception {
-    DataModel dataModel = getDataModel(
-            new long[] {1, 2, 3, 4, 5},
-            new Double[][] {
-                    {0.1, 0.2},
-                    {0.2, 0.3, 0.3, 0.6},
-                    {0.4, 0.4, 0.5, 0.9},
-                    {0.1, 0.4, 0.5, 0.8, 0.9, 1.0},
-                    {0.2, 0.3, 0.6, 0.7, 0.1, 0.2},
-            });
-    UserSimilarity similarity = new PearsonCorrelationSimilarity(dataModel);
-    UserNeighborhood neighborhood = new NearestNUserNeighborhood(2, similarity, dataModel);
-    Recommender recommender = new GenericUserBasedRecommender(dataModel, neighborhood, similarity);
-    List<RecommendedItem> fewRecommended = recommender.recommend(1, 2);
-    List<RecommendedItem> moreRecommended = recommender.recommend(1, 4);
-    for (int i = 0; i < fewRecommended.size(); i++) {
-      assertEquals(fewRecommended.get(i).getItemID(), moreRecommended.get(i).getItemID());
-    }
-    recommender.refresh(null);
-    for (int i = 0; i < fewRecommended.size(); i++) {
-      assertEquals(fewRecommended.get(i).getItemID(), moreRecommended.get(i).getItemID());
-    }
-  }
-
-  public void testRescorer() throws Exception {
-    DataModel dataModel = getDataModel(
-            new long[] {1, 2, 3},
-            new Double[][] {
-                    {0.1, 0.2},
-                    {0.2, 0.3, 0.3, 0.6},
-                    {0.4, 0.4, 0.5, 0.9},
-            });
-    UserSimilarity similarity = new PearsonCorrelationSimilarity(dataModel);
-    UserNeighborhood neighborhood = new NearestNUserNeighborhood(1, similarity, dataModel);
-    Recommender recommender = new GenericUserBasedRecommender(dataModel, neighborhood, similarity);
-    List<RecommendedItem> originalRecommended = recommender.recommend(1, 2);
-    List<RecommendedItem> rescoredRecommended =
-        recommender.recommend(1, 2, new ReversingRescorer<Long>());
-    assertNotNull(originalRecommended);
-    assertNotNull(rescoredRecommended);
-    assertEquals(2, originalRecommended.size());
-    assertEquals(2, rescoredRecommended.size());
-    assertEquals(originalRecommended.get(0).getItemID(), rescoredRecommended.get(1).getItemID());
-    assertEquals(originalRecommended.get(1).getItemID(), rescoredRecommended.get(0).getItemID());
-  }
-
-  public void testEstimatePref() throws Exception {
-    Recommender recommender = buildRecommender();
-    assertEquals(0.3f, recommender.estimatePreference(1, 2));
-  }
-
-  public void testBestRating() throws Exception {
-    Recommender recommender = buildRecommender();
-    List<RecommendedItem> recommended = recommender.recommend(1, 1);
-    assertNotNull(recommended);
-    assertEquals(1, recommended.size());
-    RecommendedItem firstRecommended = recommended.get(0);
-    // item one should be recommended because it has a greater rating/score
-    assertEquals(2, firstRecommended.getItemID());
-    assertEquals(0.3f, firstRecommended.getValue(), EPSILON);
-  }
-
-  public void testMostSimilar() throws Exception {
-    UserBasedRecommender recommender = buildRecommender();
-    long[] similar = recommender.mostSimilarUserIDs(1, 2);
-    assertNotNull(similar);
-    assertEquals(2, similar.length);
-    assertEquals(2, similar[0]);
-    assertEquals(4, similar[1]);
-  }
-
-  public void testIsolatedUser() throws Exception {
-    DataModel dataModel = getDataModel(
-            new long[] {1, 2, 3, 4},
-            new Double[][] {
-                    {0.1, 0.2},
-                    {0.2, 0.3, 0.3, 0.6},
-                    {0.4, 0.4, 0.5, 0.9},
-                    {null, null, null, null, 1.0},
-            });
-    UserSimilarity similarity = new PearsonCorrelationSimilarity(dataModel);
-    UserNeighborhood neighborhood = new NearestNUserNeighborhood(3, similarity, dataModel);
-    UserBasedRecommender recommender = new GenericUserBasedRecommender(dataModel, neighborhood, similarity);
-    long[] mostSimilar = recommender.mostSimilarUserIDs(4, 3);
-    assertNotNull(mostSimilar);
-    assertEquals(0, mostSimilar.length);
-  }
-
-  private static UserBasedRecommender buildRecommender() throws TasteException {
-    DataModel dataModel = getDataModel();
-    UserSimilarity similarity = new PearsonCorrelationSimilarity(dataModel);
-    UserNeighborhood neighborhood = new NearestNUserNeighborhood(1, similarity, dataModel);
-    return new GenericUserBasedRecommender(dataModel, neighborhood, similarity);
-  }
-
-}
+}
\ No newline at end of file

Added: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/recommender/ItemUserAverageRecommenderTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/recommender/ItemUserAverageRecommenderTest.java?rev=911810&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/recommender/ItemUserAverageRecommenderTest.java (added)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/recommender/ItemUserAverageRecommenderTest.java Fri Feb 19 13:45:18 2010
@@ -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.impl.TasteTestCase;
+import org.apache.mahout.cf.taste.recommender.RecommendedItem;
+import org.apache.mahout.cf.taste.recommender.Recommender;
+
+import java.util.List;
+
+public final class ItemUserAverageRecommenderTest extends TasteTestCase {
+
+  public void testRecommender() throws Exception {
+    Recommender recommender = new ItemUserAverageRecommender(getDataModel());
+    List<RecommendedItem> recommended = recommender.recommend(1, 1);
+    assertNotNull(recommended);
+    assertEquals(1, recommended.size());
+    RecommendedItem firstRecommended = recommended.get(0);
+    assertEquals(2, firstRecommended.getItemID());
+    assertEquals(0.35151517f, firstRecommended.getValue());
+    recommender.refresh(null);
+    assertEquals(2, firstRecommended.getItemID());
+    assertEquals(0.35151517f, firstRecommended.getValue());
+  }
+
+}
\ No newline at end of file

Added: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/recommender/RandomRecommenderTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/recommender/RandomRecommenderTest.java?rev=911810&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/recommender/RandomRecommenderTest.java (added)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/recommender/RandomRecommenderTest.java Fri Feb 19 13:45:18 2010
@@ -0,0 +1,39 @@
+/**
+ * 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.impl.TasteTestCase;
+import org.apache.mahout.cf.taste.recommender.RecommendedItem;
+import org.apache.mahout.cf.taste.recommender.Recommender;
+
+import java.util.List;
+
+public final class RandomRecommenderTest extends TasteTestCase {
+
+  public void testRecommender() throws Exception {
+    Recommender recommender = new RandomRecommender(getDataModel());
+    List<RecommendedItem> recommended = recommender.recommend(1, 1);
+    assertNotNull(recommended);
+    assertEquals(1, recommended.size());
+    RecommendedItem firstRecommended = recommended.get(0);
+    assertEquals(2, firstRecommended.getItemID());
+    recommender.refresh(null);
+    assertEquals(2, firstRecommended.getItemID());
+  }
+
+}
\ No newline at end of file

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/hadoop/DatasetSplit.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/hadoop/DatasetSplit.java?rev=911810&r1=911809&r2=911810&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/hadoop/DatasetSplit.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/hadoop/DatasetSplit.java Fri Feb 19 13:45:18 2010
@@ -30,8 +30,8 @@
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.mahout.common.RandomUtils;
+import org.apache.mahout.common.RandomWrapper;
 import org.apache.mahout.common.StringUtils;
-import org.uncommons.maths.random.RepeatableRNG;
 
 /**
  * Separate the input data into a training and testing set.
@@ -63,7 +63,7 @@
   }
   
   public DatasetSplit(double threshold) {
-    this(RandomUtils.seedBytesToLong(((RepeatableRNG) RandomUtils.getRandom()).getSeed()), threshold);
+    this(((RandomWrapper) RandomUtils.getRandom()).getSeed(), threshold);
   }
   
   public DatasetSplit(JobConf conf) {

Modified: lucene/mahout/trunk/examples/src/test/java/org/apache/mahout/ga/watchmaker/cd/hadoop/DatasetSplitTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/test/java/org/apache/mahout/ga/watchmaker/cd/hadoop/DatasetSplitTest.java?rev=911810&r1=911809&r2=911810&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/test/java/org/apache/mahout/ga/watchmaker/cd/hadoop/DatasetSplitTest.java (original)
+++ lucene/mahout/trunk/examples/src/test/java/org/apache/mahout/ga/watchmaker/cd/hadoop/DatasetSplitTest.java Fri Feb 19 13:45:18 2010
@@ -22,9 +22,9 @@
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.mahout.common.MahoutTestCase;
+import org.apache.mahout.common.RandomWrapper;
 import org.apache.mahout.ga.watchmaker.cd.hadoop.DatasetSplit.RndLineRecordReader;
 import org.apache.mahout.common.RandomUtils;
-import org.uncommons.maths.random.MersenneTwisterRNG;
 
 import java.io.IOException;
 import java.util.HashSet;
@@ -87,7 +87,7 @@
     int n = 20;
 
     for (int nloop = 0; nloop < n; nloop++) {
-      MersenneTwisterRNG rng = (MersenneTwisterRNG) RandomUtils.getRandom();
+      RandomWrapper rng = (RandomWrapper) RandomUtils.getRandom();
       double threshold = rng.nextDouble();
 
       JobConf conf = new JobConf();
@@ -95,7 +95,7 @@
       LongWritable key = new LongWritable();
       Text value = new Text();
       
-      DatasetSplit split = new DatasetSplit(RandomUtils.seedBytesToLong(rng.getSeed()), threshold);
+      DatasetSplit split = new DatasetSplit(rng.getSeed(), threshold);
 
       // read the training set
       split.storeJobParameters(conf);
@@ -121,9 +121,9 @@
     int n = 20;
 
     for (int nloop = 0; nloop < n; nloop++) {
-      MersenneTwisterRNG rng = (MersenneTwisterRNG) RandomUtils.getRandom();
+      RandomWrapper rng = (RandomWrapper) RandomUtils.getRandom();
 
-      long seed = RandomUtils.seedBytesToLong(rng.getSeed());
+      long seed = rng.getSeed();
       double threshold = rng.nextDouble();
       boolean training = rng.nextBoolean();