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/07/17 23:17:46 UTC

svn commit: r965135 - in /mahout/trunk/core/src: main/java/org/apache/mahout/cf/taste/impl/model/ main/java/org/apache/mahout/cf/taste/impl/model/file/ main/java/org/apache/mahout/cf/taste/model/ test/java/org/apache/mahout/cf/taste/impl/model/ test/ja...

Author: srowen
Date: Sat Jul 17 21:17:45 2010
New Revision: 965135

URL: http://svn.apache.org/viewvc?rev=965135&view=rev
Log:
MAHOUT-441

Added:
    mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/impl/model/file/FileIDMigrator.java
    mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/model/UpdatableIDMigrator.java
    mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/file/FileIDMigratorTest.java
Modified:
    mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/impl/model/AbstractIDMigrator.java
    mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/impl/model/AbstractJDBCIDMigrator.java
    mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/impl/model/MemoryIDMigrator.java
    mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/model/IDMigrator.java
    mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/MemoryIDMigratorTest.java

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/impl/model/AbstractIDMigrator.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/impl/model/AbstractIDMigrator.java?rev=965135&r1=965134&r2=965135&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/impl/model/AbstractIDMigrator.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/impl/model/AbstractIDMigrator.java Sat Jul 17 21:17:45 2010
@@ -21,7 +21,8 @@ import java.nio.charset.Charset;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
 
-import org.apache.mahout.cf.taste.common.TasteException;
+import java.util.Collection;
+import org.apache.mahout.cf.taste.common.Refreshable;
 import org.apache.mahout.cf.taste.model.IDMigrator;
 
 public abstract class AbstractIDMigrator implements IDMigrator {
@@ -59,12 +60,9 @@ public abstract class AbstractIDMigrator
   public long toLongID(String stringID) {
     return hash(stringID);
   }
-  
+
   @Override
-  public void initialize(Iterable<String> stringIDs) throws TasteException {
-    for (String stringID : stringIDs) {
-      storeMapping(toLongID(stringID), stringID);
-    }
+  public void refresh(Collection<Refreshable> alreadyRefreshed) {
   }
   
 }
\ No newline at end of file

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/impl/model/AbstractJDBCIDMigrator.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/impl/model/AbstractJDBCIDMigrator.java?rev=965135&r1=965134&r2=965135&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/impl/model/AbstractJDBCIDMigrator.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/impl/model/AbstractJDBCIDMigrator.java Sat Jul 17 21:17:45 2010
@@ -25,6 +25,7 @@ import java.sql.SQLException;
 import javax.sql.DataSource;
 
 import org.apache.mahout.cf.taste.common.TasteException;
+import org.apache.mahout.cf.taste.model.UpdatableIDMigrator;
 import org.apache.mahout.common.IOUtils;
 
 /**
@@ -32,7 +33,7 @@ import org.apache.mahout.common.IOUtils;
  * configure the class to operate with particular databases by supplying appropriate SQL statements to the
  * constructor.
  */
-public abstract class AbstractJDBCIDMigrator extends AbstractIDMigrator {
+public abstract class AbstractJDBCIDMigrator extends AbstractIDMigrator implements UpdatableIDMigrator {
   
   public static final String DEFAULT_MAPPING_TABLE = "taste_id_mapping";
   public static final String DEFAULT_LONG_ID_COLUMN = "long_id";
@@ -96,5 +97,12 @@ public abstract class AbstractJDBCIDMigr
       IOUtils.quietClose(rs, stmt, conn);
     }
   }
-  
+
+  @Override
+  public void initialize(Iterable<String> stringIDs) throws TasteException {
+    for (String stringID : stringIDs) {
+      storeMapping(toLongID(stringID), stringID);
+    }
+  }
+
 }
\ No newline at end of file

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/impl/model/MemoryIDMigrator.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/impl/model/MemoryIDMigrator.java?rev=965135&r1=965134&r2=965135&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/impl/model/MemoryIDMigrator.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/impl/model/MemoryIDMigrator.java Sat Jul 17 21:17:45 2010
@@ -18,11 +18,12 @@
 package org.apache.mahout.cf.taste.impl.model;
 
 import org.apache.mahout.cf.taste.impl.common.FastByIDMap;
+import org.apache.mahout.cf.taste.model.UpdatableIDMigrator;
 
 /**
  * Implementation which stores the reverse long-to-String mapping in memory.
  */
-public final class MemoryIDMigrator extends AbstractIDMigrator {
+public final class MemoryIDMigrator extends AbstractIDMigrator implements UpdatableIDMigrator {
   
   private final FastByIDMap<String> longToString;
   
@@ -43,5 +44,12 @@ public final class MemoryIDMigrator exte
       return longToString.get(longID);
     }
   }
+
+  @Override
+  public void initialize(Iterable<String> stringIDs) {
+    for (String stringID : stringIDs) {
+      storeMapping(toLongID(stringID), stringID);
+    }
+  }
   
 }
\ No newline at end of file

Added: mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/impl/model/file/FileIDMigrator.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/impl/model/file/FileIDMigrator.java?rev=965135&view=auto
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/impl/model/file/FileIDMigrator.java (added)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/impl/model/file/FileIDMigrator.java Sat Jul 17 21:17:45 2010
@@ -0,0 +1,122 @@
+/*
+ * 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.file;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.mahout.cf.taste.common.Refreshable;
+import org.apache.mahout.cf.taste.impl.common.FastByIDMap;
+import org.apache.mahout.cf.taste.impl.model.AbstractIDMigrator;
+import org.apache.mahout.cf.taste.model.IDMigrator;
+import org.apache.mahout.common.FileLineIterable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>
+ * An {@link IDMigrator} backed by a file. This class typically expects a file where each line
+ * contains a single stringID to be stored in this migrator.
+ * </p>
+ *
+ * <p>
+ * This class will reload data from the data file when {@link #refresh(Collection)} is called, unless the file
+ * has been reloaded very recently already.
+ * </p>
+ */
+public class FileIDMigrator extends AbstractIDMigrator {
+
+  public static final long DEFAULT_MIN_RELOAD_INTERVAL_MS = 60 * 1000L; // 1 minute?
+
+  private final File dataFile;
+  private final FastByIDMap<String> longToString;
+  private final ReentrantLock reloadLock;
+
+  private long lastModified;
+  private boolean loaded;
+  private long minReloadIntervalMS;
+
+  private static final Logger log = LoggerFactory.getLogger(FileIDMigrator.class);
+
+  public FileIDMigrator(File dataFile) throws FileNotFoundException {
+    this(dataFile, DEFAULT_MIN_RELOAD_INTERVAL_MS);
+  }
+
+  public FileIDMigrator(File dataFile, long minReloadIntervalMS) throws FileNotFoundException {
+    super();
+    longToString = new FastByIDMap<String>(100);
+    if (dataFile == null) {
+      throw new IllegalArgumentException("dataFile is null");
+    }
+    if (!dataFile.exists() || dataFile.isDirectory()) {
+      throw new FileNotFoundException(dataFile.toString());
+    }
+
+    log.info("Creating FileReadonlyIDMigrator for file {}", dataFile);
+
+    this.dataFile = dataFile;
+    this.reloadLock = new ReentrantLock();
+    this.lastModified = dataFile.lastModified();
+    this.loaded = false;
+    this.minReloadIntervalMS = minReloadIntervalMS;
+  }
+
+  @Override
+  public String toStringID(long longID) {
+    if (!loaded) {
+      reload();
+    }
+    synchronized (longToString) {
+      return longToString.get(longID);
+    }
+  }
+
+  private void reload() {
+    if (!reloadLock.isLocked()) {
+      reloadLock.lock();
+      try {
+        longToString.clear();
+        FileLineIterable lines = new FileLineIterable(dataFile);
+        for (String line : lines) {
+          longToString.put(toLongID(line), line);
+        }
+        lastModified = dataFile.lastModified();
+        loaded = true;
+      } catch(IOException ioe) {
+        throw new RuntimeException("Unable to reload dataFile [" + dataFile.getAbsolutePath() + "]", ioe);
+      } finally {
+        reloadLock.unlock();
+      }
+    }
+  }
+
+  @Override
+  public void refresh(Collection<Refreshable> alreadyRefreshed) {
+    if (!loaded || dataFile.lastModified() > lastModified + minReloadIntervalMS) {
+      log.debug("File has changed; reloading...");
+      reload();
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "FileIDMigrator[dataFile:" + dataFile + ']';
+  }
+}

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/model/IDMigrator.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/model/IDMigrator.java?rev=965135&r1=965134&r2=965135&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/model/IDMigrator.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/model/IDMigrator.java Sat Jul 17 21:17:45 2010
@@ -17,6 +17,7 @@
 
 package org.apache.mahout.cf.taste.model;
 
+import org.apache.mahout.cf.taste.common.Refreshable;
 import org.apache.mahout.cf.taste.common.TasteException;
 
 /**
@@ -44,7 +45,7 @@ import org.apache.mahout.cf.taste.common
  * 
  * @since 0.2
  */
-public interface IDMigrator {
+public interface IDMigrator extends Refreshable {
   
   /**
    * @return the top 8 bytes of the MD5 hash of the bytes of the given {@link String}'s UTF-8 encoding as a
@@ -61,27 +62,4 @@ public interface IDMigrator {
    */
   String toStringID(long longID) throws TasteException;
   
-  /**
-   * Stores the reverse long-to-String mapping in some kind of backing store. Note that this must be called
-   * directly (or indirectly through {@link #initialize(Iterable)}) for every String that might be encountered
-   * in the application, or else the mapping will not be known.
-   * 
-   * @param longID
-   *          long ID
-   * @param stringID
-   *          string ID that maps to/from that long ID
-   * @throws TasteException
-   *           if an error occurs while saving the mapping
-   */
-  void storeMapping(long longID, String stringID) throws TasteException;
-  
-  /**
-   * Make the mapping aware of the given string IDs. This must be called initially before the implementation
-   * is used, or else it will not be aware of reverse long-to-String mappings.
-   * 
-   * @throws TasteException
-   *           if an error occurs while storing the mappings
-   */
-  void initialize(Iterable<String> stringIDs) throws TasteException;
-  
 }

Added: mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/model/UpdatableIDMigrator.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/model/UpdatableIDMigrator.java?rev=965135&view=auto
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/model/UpdatableIDMigrator.java (added)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/model/UpdatableIDMigrator.java Sat Jul 17 21:17:45 2010
@@ -0,0 +1,47 @@
+/**
+ * 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.model;
+
+import org.apache.mahout.cf.taste.common.TasteException;
+
+public interface UpdatableIDMigrator extends IDMigrator {
+  
+  /**
+   * Stores the reverse long-to-String mapping in some kind of backing store. Note that this must be called
+   * directly (or indirectly through {@link #initialize(Iterable)}) for every String that might be encountered
+   * in the application, or else the mapping will not be known.
+   *
+   * @param longID
+   *          long ID
+   * @param stringID
+   *          string ID that maps to/from that long ID
+   * @throws TasteException
+   *           if an error occurs while saving the mapping
+   */
+  void storeMapping(long longID, String stringID) throws TasteException;
+
+  /**
+   * Make the mapping aware of the given string IDs. This must be called initially before the implementation
+   * is used, or else it will not be aware of reverse long-to-String mappings.
+   *
+   * @throws TasteException
+   *           if an error occurs while storing the mappings
+   */
+  void initialize(Iterable<String> stringIDs) throws TasteException;
+  
+}

Modified: mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/MemoryIDMigratorTest.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/MemoryIDMigratorTest.java?rev=965135&r1=965134&r2=965135&view=diff
==============================================================================
--- mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/MemoryIDMigratorTest.java (original)
+++ mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/MemoryIDMigratorTest.java Sat Jul 17 21:17:45 2010
@@ -21,6 +21,7 @@ import org.apache.mahout.cf.taste.model.
 import org.apache.mahout.common.MahoutTestCase;
 
 import java.util.Collections;
+import org.apache.mahout.cf.taste.model.UpdatableIDMigrator;
 
 public final class MemoryIDMigratorTest extends MahoutTestCase {
 
@@ -34,7 +35,7 @@ public final class MemoryIDMigratorTest 
   }
 
   public void testStore() throws Exception {
-    IDMigrator migrator = new MemoryIDMigrator();
+    UpdatableIDMigrator migrator = new MemoryIDMigrator();
     long id = migrator.toLongID(DUMMY_STRING);
     assertNull(migrator.toStringID(id));
     migrator.storeMapping(id, DUMMY_STRING);
@@ -42,7 +43,7 @@ public final class MemoryIDMigratorTest 
   }
 
   public void testInitialize() throws Exception {
-    IDMigrator migrator = new MemoryIDMigrator();
+    UpdatableIDMigrator migrator = new MemoryIDMigrator();
     long id = migrator.toLongID(DUMMY_STRING);
     assertNull(migrator.toStringID(id));
     migrator.initialize(Collections.singleton(DUMMY_STRING));

Added: mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/file/FileIDMigratorTest.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/file/FileIDMigratorTest.java?rev=965135&view=auto
==============================================================================
--- mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/file/FileIDMigratorTest.java (added)
+++ mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/file/FileIDMigratorTest.java Sat Jul 17 21:17:45 2010
@@ -0,0 +1,97 @@
+/*
+ * 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.file;
+
+import java.io.File;
+import org.apache.mahout.cf.taste.impl.TasteTestCase;
+import org.apache.mahout.cf.taste.model.IDMigrator;
+
+/**
+ * Tests {@link FileIDMigrator}
+ */
+public class FileIDMigratorTest extends TasteTestCase {
+
+  private static final String[] STRING_IDS = {
+      "dog",
+      "cow" };
+
+  private static final String[] UPDATED_STRING_IDS = {
+      "dog",
+      "cow",
+      "donkey" };
+
+  private File testFile;
+
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    testFile = getTestTempFile("test.txt");
+    writeLines(testFile, STRING_IDS);
+  }
+
+  public void testLoadFromFile() throws Exception {
+    IDMigrator migrator = new FileIDMigrator(testFile);
+    long dogAsLong = migrator.toLongID("dog");
+    long cowAsLong = migrator.toLongID("cow");
+    long donkeyAsLong = migrator.toLongID("donkey");
+    assertEquals("dog", migrator.toStringID(dogAsLong));
+    assertEquals("cow", migrator.toStringID(cowAsLong));
+    assertNull(migrator.toStringID(donkeyAsLong));
+  }
+
+  public void testNoRefreshAfterFileUpdate() throws Exception {
+    IDMigrator migrator = new FileIDMigrator(testFile, 0L);
+
+    /* call a method to make sure the original file is loaded */
+    long dogAsLong = migrator.toLongID("dog");
+    migrator.toStringID(dogAsLong);
+
+    /* change the underlying file,
+     * we have to wait at least a second to see the change in the file's lastModified timestamp */
+    Thread.sleep(2000L);
+    writeLines(testFile, UPDATED_STRING_IDS);
+
+    /* we shouldn't see any changes in the data as we have not yet refreshed */
+    long cowAsLong = migrator.toLongID("cow");
+    long donkeyAsLong = migrator.toLongID("donkey");
+    assertEquals("dog", migrator.toStringID(dogAsLong));
+    assertEquals("cow", migrator.toStringID(cowAsLong));
+    assertNull(migrator.toStringID(donkeyAsLong));
+  }
+
+  public void testRefreshAfterFileUpdate() throws Exception {
+    IDMigrator migrator = new FileIDMigrator(testFile, 0L);
+
+    /* call a method to make sure the original file is loaded */
+    long dogAsLong = migrator.toLongID("dog");
+    migrator.toStringID(dogAsLong);
+
+    /* change the underlying file,
+     * we have to wait at least a second to see the change in the file's lastModified timestamp */
+    Thread.sleep(2000L);
+    writeLines(testFile, UPDATED_STRING_IDS);
+
+    migrator.refresh(null);
+
+    long cowAsLong = migrator.toLongID("cow");
+    long donkeyAsLong = migrator.toLongID("donkey");
+    assertEquals("dog", migrator.toStringID(dogAsLong));
+    assertEquals("cow", migrator.toStringID(cowAsLong));
+    assertEquals("donkey", migrator.toStringID(donkeyAsLong));
+  }
+}