You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mahout.apache.org by td...@apache.org on 2010/09/17 20:41:19 UTC

svn commit: r998241 - in /mahout/trunk/core/src: main/java/org/apache/mahout/vectors/ test/java/org/apache/mahout/vectors/

Author: tdunning
Date: Fri Sep 17 18:41:18 2010
New Revision: 998241

URL: http://svn.apache.org/viewvc?rev=998241&view=rev
Log:
Switched encoders to use byte[] instead of string
for internal communications.
Added cached encoders.

Added:
    mahout/trunk/core/src/main/java/org/apache/mahout/vectors/CachingContinuousValueEncoder.java
    mahout/trunk/core/src/main/java/org/apache/mahout/vectors/CachingStaticWordValueEncoder.java
    mahout/trunk/core/src/main/java/org/apache/mahout/vectors/CachingTextValueEncoder.java
Modified:
    mahout/trunk/core/src/main/java/org/apache/mahout/vectors/AdaptiveWordValueEncoder.java
    mahout/trunk/core/src/main/java/org/apache/mahout/vectors/ConstantValueEncoder.java
    mahout/trunk/core/src/main/java/org/apache/mahout/vectors/ContinuousValueEncoder.java
    mahout/trunk/core/src/main/java/org/apache/mahout/vectors/FeatureVectorEncoder.java
    mahout/trunk/core/src/main/java/org/apache/mahout/vectors/InteractionValueEncoder.java
    mahout/trunk/core/src/main/java/org/apache/mahout/vectors/LuceneTextValueEncoder.java
    mahout/trunk/core/src/main/java/org/apache/mahout/vectors/StaticWordValueEncoder.java
    mahout/trunk/core/src/main/java/org/apache/mahout/vectors/TextValueEncoder.java
    mahout/trunk/core/src/main/java/org/apache/mahout/vectors/WordValueEncoder.java
    mahout/trunk/core/src/test/java/org/apache/mahout/vectors/InteractionValueEncoderTest.java
    mahout/trunk/core/src/test/java/org/apache/mahout/vectors/MurmurHashTest.java
    mahout/trunk/core/src/test/java/org/apache/mahout/vectors/TextValueEncoderTest.java
    mahout/trunk/core/src/test/java/org/apache/mahout/vectors/WordLikeValueEncoderTest.java

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/vectors/AdaptiveWordValueEncoder.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/vectors/AdaptiveWordValueEncoder.java?rev=998241&r1=998240&r2=998241&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/vectors/AdaptiveWordValueEncoder.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/vectors/AdaptiveWordValueEncoder.java Fri Sep 17 18:41:18 2010
@@ -47,22 +47,33 @@ public class AdaptiveWordValueEncoder ex
     super.addToVector(originalForm, weight, data);
   }
 
+  /**
+   * Adds a value to a vector.
+   *
+   * @param originalForm The original form of the value as a byte array.
+   * @param data         The vector to which the value should be added.
+   */
   @Override
-  protected int hashForProbe(String originalForm, int dataSize, String name, int probe) {
+  public void addToVector(byte[] originalForm, double weight, Vector data) {
+    super.addToVector(originalForm, weight, data);
+  }
+
+  @Override
+  protected int hashForProbe(byte[] originalForm, int dataSize, String name, int probe) {
     return super.hashForProbe(originalForm, dataSize, name, probe);
   }
 
   @Override
-  protected double getWeight(String originalForm, double w) {
+  protected double getWeight(byte[] originalForm, double w) {
     return w * weight(originalForm);
   }
 
   @Override
-  protected double weight(String originalForm) {
+  protected double weight(byte[] originalForm) {
     // the counts here are adjusted so that every observed value has an extra 0.5 count
     // as does a hypothetical unobserved value.  This smooths our estimates a bit and
     // allows the first word seen to have a non-zero weight of -log(1.5 / 2)
-    double thisWord = dictionary.count(originalForm) + 0.5;
+    double thisWord = dictionary.count(new String(originalForm)) + 0.5;
     double allWords = dictionary.size() + dictionary.elementSet().size() * 0.5 + 0.5;
     return -Math.log(thisWord / allWords);
   }

Added: mahout/trunk/core/src/main/java/org/apache/mahout/vectors/CachingContinuousValueEncoder.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/vectors/CachingContinuousValueEncoder.java?rev=998241&view=auto
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/vectors/CachingContinuousValueEncoder.java (added)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/vectors/CachingContinuousValueEncoder.java Fri Sep 17 18:41:18 2010
@@ -0,0 +1,61 @@
+/*
+ * 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.vectors;
+
+import org.apache.mahout.math.map.OpenIntIntHashMap;
+
+
+public class CachingContinuousValueEncoder extends ContinuousValueEncoder {
+  private int dataSize;
+  private OpenIntIntHashMap[] caches;
+
+  public CachingContinuousValueEncoder(String name, int dataSize) {
+    super(name);
+    this.dataSize = dataSize;
+    initCaches();
+  }
+
+  private void initCaches() {
+    this.caches = new OpenIntIntHashMap[getProbes()];
+    for (int ii = 0; ii < getProbes(); ii++) {
+      caches[ii] = new OpenIntIntHashMap();
+    }
+  }
+
+  protected OpenIntIntHashMap[] getCaches() {
+    return caches;
+  }
+
+  @Override
+  public void setProbes(int probes) {
+    super.setProbes(probes);
+    initCaches();
+  }
+
+  protected int hashForProbe(String originalForm, int dataSize, String name, int probe) {
+    if (dataSize != this.dataSize) {
+      throw new IllegalArgumentException("dataSize argument [" + dataSize + "] does not match expected dataSize [" + this.dataSize + "]");
+    }
+    if (caches[probe].containsKey(originalForm.hashCode())) {
+      return caches[probe].get(originalForm.hashCode());
+    }
+    int hash = super.hashForProbe(originalForm.getBytes(), dataSize, name, probe);
+    caches[probe].put(originalForm.hashCode(), hash);
+    return hash;
+  }
+}

Added: mahout/trunk/core/src/main/java/org/apache/mahout/vectors/CachingStaticWordValueEncoder.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/vectors/CachingStaticWordValueEncoder.java?rev=998241&view=auto
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/vectors/CachingStaticWordValueEncoder.java (added)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/vectors/CachingStaticWordValueEncoder.java Fri Sep 17 18:41: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.vectors;
+
+import org.apache.mahout.math.map.OpenIntIntHashMap;
+
+public class CachingStaticWordValueEncoder extends StaticWordValueEncoder {
+  private int dataSize;
+  private OpenIntIntHashMap[] caches;
+//  private TIntIntHashMap[] caches;
+
+  public CachingStaticWordValueEncoder(String name, int dataSize) {
+    super(name);
+    this.dataSize = dataSize;
+    initCaches();
+  }
+
+  private void initCaches() {
+    this.caches = new OpenIntIntHashMap[getProbes()];
+    for (int ii = 0; ii < getProbes(); ii++) {
+      caches[ii] = new OpenIntIntHashMap();
+    }
+  }
+
+  protected OpenIntIntHashMap[] getCaches() {
+    return caches;
+  }
+
+  @Override
+  public void setProbes(int probes) {
+    super.setProbes(probes);
+    initCaches();
+  }
+
+  protected int hashForProbe(String originalForm, int dataSize, String name, int probe) {
+    if (dataSize != this.dataSize) {
+      throw new IllegalArgumentException("dataSize argument [" + dataSize + "] does not match expected dataSize [" + this.dataSize + "]");
+    }
+    if (caches[probe].containsKey(originalForm.hashCode())) {
+      return caches[probe].get(originalForm.hashCode());
+    }
+    int hash = super.hashForProbe(originalForm.getBytes(), dataSize, name, probe);
+    caches[probe].put(originalForm.hashCode(), hash);
+    return hash;
+  }
+}
+

Added: mahout/trunk/core/src/main/java/org/apache/mahout/vectors/CachingTextValueEncoder.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/vectors/CachingTextValueEncoder.java?rev=998241&view=auto
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/vectors/CachingTextValueEncoder.java (added)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/vectors/CachingTextValueEncoder.java Fri Sep 17 18:41:18 2010
@@ -0,0 +1,25 @@
+/*
+ * 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.vectors;
+
+public class CachingTextValueEncoder extends TextValueEncoder {
+  public CachingTextValueEncoder(String name, int dataSize) {
+    super(name);
+    setWordEncoder(new CachingStaticWordValueEncoder(name, dataSize));
+  }
+}

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/vectors/ConstantValueEncoder.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/vectors/ConstantValueEncoder.java?rev=998241&r1=998240&r2=998241&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/vectors/ConstantValueEncoder.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/vectors/ConstantValueEncoder.java Fri Sep 17 18:41:18 2010
@@ -28,18 +28,20 @@ public class ConstantValueEncoder extend
   }
 
   @Override
-  public void addToVector(String originalForm, double weight, Vector data) {
+  public void addToVector(byte[] originalForm, double weight, Vector data) {
     int probes = getProbes();
     String name = getName();
     for (int i = 0; i < probes; i++) {
-        int n = hashForProbe(originalForm, data.size(), name, i);
-        trace(null, n);
+      int n = hashForProbe(originalForm, data.size(), name, i);
+      if(isTraceEnabled()){
+        trace((byte[]) null, n);                
+      }
       data.set(n, data.get(n) + getWeight(originalForm,weight));
     }
   }
 
   @Override
-  protected double getWeight(String originalForm, double w) {
+  protected double getWeight(byte[] originalForm, double w) {
     return w;
   }
 
@@ -49,7 +51,7 @@ public class ConstantValueEncoder extend
   }
 
   @Override
-  protected int hashForProbe(String originalForm, int dataSize, String name, int probe){
+  protected int hashForProbe(byte[] originalForm, int dataSize, String name, int probe){
     return hash(name, probe, dataSize);
   }
 

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/vectors/ContinuousValueEncoder.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/vectors/ContinuousValueEncoder.java?rev=998241&r1=998240&r2=998241&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/vectors/ContinuousValueEncoder.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/vectors/ContinuousValueEncoder.java Fri Sep 17 18:41:18 2010
@@ -35,24 +35,26 @@ public class ContinuousValueEncoder exte
    * @param data         The vector to which the value should be added.
    */
   @Override
-  public void addToVector(String originalForm, double weight, Vector data) {
+  public void addToVector(byte[] originalForm, double weight, Vector data) {
     int probes = getProbes();
     String name = getName();
     for (int i = 0; i < probes; i++) {
       int n = hashForProbe(originalForm, data.size(), name, i);
-      trace(null, n);
+      if(isTraceEnabled()){
+        trace((byte[]) null, n);        
+      }
       data.set(n, data.get(n) + getWeight(originalForm,weight));
     }
   }
 
   @Override
-  protected int hashForProbe(String originalForm, int dataSize, String name, int probe) {
+  protected int hashForProbe(byte[] originalForm, int dataSize, String name, int probe) {
     return hash(name, CONTINUOUS_VALUE_HASH_SEED + probe, dataSize);
   }
 
   @Override
-  protected double getWeight(String originalForm, double w) {
-    return w * Double.parseDouble(originalForm);
+  protected double getWeight(byte[] originalForm, double w) {
+    return w * Double.parseDouble(new String(originalForm));
   }
 
   /**

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/vectors/FeatureVectorEncoder.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/vectors/FeatureVectorEncoder.java?rev=998241&r1=998240&r2=998241&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/vectors/FeatureVectorEncoder.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/vectors/FeatureVectorEncoder.java Fri Sep 17 18:41:18 2010
@@ -17,6 +17,7 @@
 
 package org.apache.mahout.vectors;
 
+import com.google.common.base.Charsets;
 import com.google.common.collect.Sets;
 import org.apache.mahout.math.Vector;
 
@@ -62,6 +63,16 @@ public abstract class FeatureVectorEncod
   }
 
   /**
+   * Adds a value expressed in byte array form to a vector.
+   *
+   * @param originalForm The original form of the value as a byte array.
+   * @param data         The vector to which the value should be added.
+   */
+  public void addToVector(byte[] originalForm, Vector data) {
+    addToVector(originalForm, 1.0, data);
+  }
+
+  /**
    * Adds a weighted value expressed in string form to a vector.  In some cases it is convenient to
    * use this method to encode continuous values using the weight as the value.  In such cases, the
    * string value should typically be set to null.
@@ -70,7 +81,11 @@ public abstract class FeatureVectorEncod
    * @param weight       The weight to be applied to this feature.
    * @param data         The vector to which the value should be added.
    */
-  public abstract void addToVector(String originalForm, double weight, Vector data);
+  public void addToVector(String originalForm, double weight, Vector data){
+    addToVector(bytesForString(originalForm), weight, data);        
+  }
+
+  public abstract void addToVector(byte[] originalForm, double weight, Vector data);
 
   /**
    * Provides the unique hash for a particular probe.  For all encoders except text, this
@@ -78,30 +93,30 @@ public abstract class FeatureVectorEncod
    * thing.  For text and similar values, hashesForProbe should be over-ridden and this method
    * should not be used.
    *
-   * @param originalForm  The original string value
+   * @param originalForm  The original byte array value
    * @param dataSize      The length of hte vector being encoded
    * @param name          The name of the variable being encoded
    * @param probe             The probe number
    * @return              The hash of the current probe
    */
-  protected abstract int hashForProbe(String originalForm, int dataSize, String name, int probe);
+  protected abstract int hashForProbe(byte[] originalForm, int dataSize, String name, int probe);
 
   /**
    * Returns all of the hashes for this probe.  For most encoders, this is a singleton, but
    * for text, many hashes are returned, one for each word (unique or not).  Most implementations
    * should only implement hashForProbe for simplicity.
    *
-   * @param originalForm The original string value.
+   * @param originalForm The original byte array value.
    * @param dataSize     The length of the vector being encoded
    * @param name         The name of the variable being encoded
    * @param probe        The probe number
    * @return an Iterable of the hashes
    */
-  protected Iterable<Integer> hashesForProbe(String originalForm, int dataSize, String name, int probe) {
+  protected Iterable<Integer> hashesForProbe(byte[] originalForm, int dataSize, String name, int probe) {
     return Collections.singletonList(hashForProbe(originalForm, dataSize, name, probe));
   }
 
-  protected double getWeight(String originalForm, double w){
+  protected double getWeight(byte[] originalForm, double w){
     return 1.0;
   }
 
@@ -117,7 +132,7 @@ public abstract class FeatureVectorEncod
    *         term and probe.
    */
   protected int hash(String term, int probe, int numFeatures) {
-    long r = MurmurHash.hash64A(term.getBytes(Charset.forName("UTF-8")), probe) % numFeatures;
+    long r = MurmurHash.hash64A(bytesForString(term), probe) % numFeatures;
     if (r < 0) {
       r += numFeatures;
     }
@@ -125,6 +140,24 @@ public abstract class FeatureVectorEncod
   }
 
   /**
+    * Hash a byte array and an integer into the range [0..numFeatures-1].
+    *
+    * @param term        The bytes.
+    * @param probe       An integer that modifies the resulting hash.
+    * @param numFeatures The range into which the resulting hash must fit.
+    * @return An integer in the range [0..numFeatures-1] that has good spread for small changes in
+    *         term and probe.
+    */
+   protected int hash(byte[] term, int probe, int numFeatures) {
+     long r = MurmurHash.hash64A(term, probe) % numFeatures;
+     if (r < 0) {
+       r += numFeatures;
+     }
+     return (int) r;
+   }
+
+
+  /**
    * Hash two strings and an integer into the range [0..numFeatures-1].
    *
    * @param term1       The first string.
@@ -135,8 +168,27 @@ public abstract class FeatureVectorEncod
    *         term and probe.
    */
   protected int hash(String term1, String term2, int probe, int numFeatures) {
-    long r = MurmurHash.hash64A(term1.getBytes(Charset.forName("UTF-8")), probe);
-    r = MurmurHash.hash64A(term2.getBytes(Charset.forName("UTF-8")), (int) r) % numFeatures;
+    long r = MurmurHash.hash64A(bytesForString(term1), probe);
+    r = MurmurHash.hash64A(bytesForString(term2), (int) r) % numFeatures;
+    if (r < 0) {
+      r += numFeatures;
+    }
+    return (int) r;
+  }
+
+  /**
+   * Hash two byte arrays and an integer into the range [0..numFeatures-1].
+   *
+   * @param term1       The first string.
+   * @param term2       The second string.
+   * @param probe       An integer that modifies the resulting hash.
+   * @param numFeatures The range into which the resulting hash must fit.
+   * @return An integer in the range [0..numFeatures-1] that has good spread for small changes in
+   *         term and probe.
+   */
+  protected int hash(byte[] term1, byte[] term2, int probe, int numFeatures) {
+    long r = MurmurHash.hash64A(term1, probe);
+    r = MurmurHash.hash64A(term2, (int) r) % numFeatures;
     if (r < 0) {
       r += numFeatures;
     }
@@ -156,10 +208,10 @@ public abstract class FeatureVectorEncod
    *         term and probe.
    */
   protected int hash(String term1, String term2, String term3, String term4, int probe, int numFeatures) {
-    long r = MurmurHash.hash64A(term1.getBytes(Charset.forName("UTF-8")), probe);
-    r = MurmurHash.hash64A(term2.getBytes(Charset.forName("UTF-8")), (int) r) % numFeatures;
-    r = MurmurHash.hash64A(term3.getBytes(Charset.forName("UTF-8")), (int) r) % numFeatures;
-    r = MurmurHash.hash64A(term4.getBytes(Charset.forName("UTF-8")), (int) r) % numFeatures;
+    long r = MurmurHash.hash64A(bytesForString(term1), probe);
+    r = MurmurHash.hash64A(bytesForString(term2), (int) r) % numFeatures;
+    r = MurmurHash.hash64A(bytesForString(term3), (int) r) % numFeatures;
+    r = MurmurHash.hash64A(bytesForString(term4), (int) r) % numFeatures;
     if (r < 0) {
       r += numFeatures;
     }
@@ -193,6 +245,10 @@ public abstract class FeatureVectorEncod
     return name;
   }
 
+  protected boolean isTraceEnabled(){
+    return traceDictionary != null;
+  }
+
   protected void trace(String subName, int n) {
     if (traceDictionary != null) {
       String key = name;
@@ -209,7 +265,15 @@ public abstract class FeatureVectorEncod
     }
   }
 
+  protected void trace(byte[] subName, int n) {
+    trace(new String(subName), n);
+  }
+
   public void setTraceDictionary(Map<String, Set<Integer>> traceDictionary) {
     this.traceDictionary = traceDictionary;
   }
+
+  protected byte[] bytesForString(String x){
+    return x.getBytes(Charsets.UTF_8);
+  }
 }

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/vectors/InteractionValueEncoder.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/vectors/InteractionValueEncoder.java?rev=998241&r1=998240&r2=998241&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/vectors/InteractionValueEncoder.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/vectors/InteractionValueEncoder.java Fri Sep 17 18:41:18 2010
@@ -39,25 +39,53 @@ public class InteractionValueEncoder ext
    */
   @Override
   public void addToVector(String originalForm, double w, Vector data) {
+    throw new IllegalArgumentException("addToVector is not supported for InteractionVectorEncoder");
+  }
+
+  /**
+   * Adds a value to a vector. (Unsupported)
+   *
+   * @param originalForm The original form of the first value as a byte array.
+   * @param data          The vector to which the value should be added.
+   */
+  @Override
+  public void addToVector(byte[] originalForm, double w, Vector data) {
+    throw new IllegalArgumentException("addToVector is not supported for InteractionVectorEncoder");
+  }
+
+  /**
+   * Adds a value to a vector.
+   *
+   * @param original1 The original form of the first value as a string.
+   * @param original2 The original form of the second value as a string.
+   * @param weight        How much to weight this interaction
+   * @param data          The vector to which the value should be added.
+   */
+  public void addInteractionToVector(String original1, String original2, double weight, Vector data) {
+    byte[] originalForm1 = bytesForString(original1);
+    byte[] originalForm2 = bytesForString(original2);
+    addInteractionToVector(originalForm1, originalForm2, weight, data);
   }
 
   /**
    * Adds a value to a vector.
    *
-   * @param originalForm1 The original form of the first value as a string.
-   * @param originalForm2 The original form of the second value as a string.
+   * @param originalForm1 The original form of the first value as a byte array.
+   * @param originalForm2 The original form of the second value as a byte array.
    * @param weight        How much to weight this interaction
    * @param data          The vector to which the value should be added.
    */
-  public void addInteractionToVector(String originalForm1, String originalForm2, double weight, Vector data) {
+  public void addInteractionToVector(byte[] originalForm1, byte[] originalForm2, double weight, Vector data) {
     String name = getName();
     double w = getWeight(originalForm1, originalForm2, weight);
     for (int i = 0; i < probes(); i++) {
-      Iterable<Integer> jValues = secondEncoder.hashesForProbe(originalForm2, data.size(), name, i);
-      for(Integer k : firstEncoder.hashesForProbe(originalForm1, data.size(), name, i)){
+      Iterable<Integer> jValues = secondEncoder.hashesForProbe(originalForm2, data.size(), name, i % secondEncoder.getProbes());
+      for(Integer k : firstEncoder.hashesForProbe(originalForm1, data.size(), name, i % firstEncoder.getProbes())){
         for(Integer j : jValues) {
           int n = (k + j) % data.size();
-          trace(String.format("%s:%s", originalForm1, originalForm2), n);
+          if(isTraceEnabled()){
+            trace(String.format("%s:%s", originalForm1, originalForm2), n);            
+          }
           data.set(n, data.get(n) + w);
         }
       }
@@ -68,7 +96,7 @@ public class InteractionValueEncoder ext
     return getProbes();
   }
 
-  protected double getWeight(String originalForm1, String originalForm2, double w) {
+  protected double getWeight(byte[] originalForm1, byte[] originalForm2, double w) {
     return firstEncoder.getWeight(originalForm1, 1.0) * secondEncoder.getWeight(originalForm2, 1.0) * w;
   }
 
@@ -86,7 +114,7 @@ public class InteractionValueEncoder ext
   }
 
   @Override
-  protected int hashForProbe(String originalForm, int dataSize, String name, int probe) {
+  protected int hashForProbe(byte[] originalForm, int dataSize, String name, int probe) {
     return hash(name, probe, dataSize);
   }
 }

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/vectors/LuceneTextValueEncoder.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/vectors/LuceneTextValueEncoder.java?rev=998241&r1=998240&r2=998241&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/vectors/LuceneTextValueEncoder.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/vectors/LuceneTextValueEncoder.java Fri Sep 17 18:41:18 2010
@@ -142,7 +142,7 @@ public class LuceneTextValueEncoder exte
           throw new TokenizationException("IO error while tokenizing", e);
         }
         if (r) {
-          bufferedToken = tokenStream.getAttribute(TermAttribute.class).toString();
+          bufferedToken = tokenStream.getAttribute(TermAttribute.class).term();
         }
         return r;
       } else {

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/vectors/StaticWordValueEncoder.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/vectors/StaticWordValueEncoder.java?rev=998241&r1=998240&r2=998241&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/vectors/StaticWordValueEncoder.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/vectors/StaticWordValueEncoder.java Fri Sep 17 18:41:18 2010
@@ -30,16 +30,18 @@ import java.util.Map;
  * of probes should probably be increased to 3.
  */
 public class StaticWordValueEncoder extends WordValueEncoder {
-  private Map<String, Double> dictionary;
+  private Map<byte[], Double> dictionary;
   private double missingValueWeight = 1;
+  private byte[] nameBytes;
 
   public StaticWordValueEncoder(String name) {
     super(name);
+    nameBytes = bytesForString(name);
   }
 
   @Override
-  protected int hashForProbe(String originalForm, int dataSize, String name, int probe) {
-    return hash(name, originalForm, WORD_LIKE_VALUE_HASH_SEED + probe, dataSize);
+  protected int hashForProbe(byte[] originalForm, int dataSize, String name, int probe) {
+    return hash(nameBytes, originalForm, WORD_LIKE_VALUE_HASH_SEED + probe, dataSize);
   }
 
    /**
@@ -47,7 +49,7 @@ public class StaticWordValueEncoder exte
    * the missing value weight to be half the smallest weight in the dictionary.
    * @param dictionary  The dictionary to use to look up weights.
    */
-  public void setDictionary(Map<String, Double> dictionary) {
+  public void setDictionary(Map<byte[], Double> dictionary) {
     this.dictionary = dictionary;
     missingValueWeight = Collections.min(dictionary.values()) / 2;
   }
@@ -61,7 +63,7 @@ public class StaticWordValueEncoder exte
   }
 
   @Override
-  protected double weight(String originalForm) {
+  protected double weight(byte[] originalForm) {
     double weight = missingValueWeight;
     if (dictionary != null && dictionary.containsKey(originalForm)) {
       weight = dictionary.get(originalForm);

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/vectors/TextValueEncoder.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/vectors/TextValueEncoder.java?rev=998241&r1=998240&r2=998241&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/vectors/TextValueEncoder.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/vectors/TextValueEncoder.java Fri Sep 17 18:41:18 2010
@@ -17,6 +17,7 @@
 
 package org.apache.mahout.vectors;
 
+import com.google.common.base.Charsets;
 import com.google.common.base.Splitter;
 import com.google.common.collect.HashMultiset;
 import com.google.common.collect.Multiset;
@@ -33,10 +34,10 @@ import java.util.regex.Pattern;
  * @see LuceneTextValueEncoder
  */
 public class TextValueEncoder extends FeatureVectorEncoder {
-
   private final Splitter onNonWord = Splitter.on(Pattern.compile("\\W+")).omitEmptyStrings();
-  private FeatureVectorEncoder wordEncoder;
   private static final double LOG_2 = Math.log(2);
+
+  private FeatureVectorEncoder wordEncoder;
   private Multiset<String> counts;
 
   public TextValueEncoder(String name) {
@@ -52,17 +53,9 @@ public class TextValueEncoder extends Fe
    * @param data         The vector to which the value should be added.
    */
   @Override
-  public void addToVector(String originalForm, double weight, Vector data) {
-    add(originalForm);
-    flush(weight, data);
-  }
-
-  /**
-   * Counts up lokens in a char sequence for later addition to a vector.
-   * @param originalForm  The string to tokenize.
-   */
-  public void add(CharSequence originalForm) {
-    for (String word : tokenize(originalForm)) {
+  public void addToVector(byte[] originalForm, double weight, Vector data) {
+    Multiset<String> counts = HashMultiset.create();
+    for (String word : tokenize(new String(originalForm))) {
       counts.add(word);
     }
   }
@@ -81,15 +74,15 @@ public class TextValueEncoder extends Fe
   }
 
   @Override
-  protected int hashForProbe(String originalForm, int dataSize, String name, int probe) {
+  protected int hashForProbe(byte[] originalForm, int dataSize, String name, int probe) {
     return 0;
   }
 
   @Override
-  protected Iterable<Integer> hashesForProbe(String originalForm, int dataSize, String name, int probe) {
+  protected Iterable<Integer> hashesForProbe(byte[] originalForm, int dataSize, String name, int probe){
     List<Integer> hashes = new ArrayList<Integer>();
-    for (String word : tokenize(originalForm)) {
-      hashes.add(hashForProbe(word, dataSize, name, probe));
+    for (String word : tokenize(new String(originalForm, Charsets.UTF_8))){
+      hashes.add(hashForProbe(bytesForString(word), dataSize, name, probe));
     }
     return hashes;
   }
@@ -127,7 +120,7 @@ public class TextValueEncoder extends Fe
     return r.toString();
   }
 
-  public void setWordEncoder(FeatureVectorEncoder wordEncoder) {
+  public final void setWordEncoder(FeatureVectorEncoder wordEncoder) {
     this.wordEncoder = wordEncoder;
   }
 }

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/vectors/WordValueEncoder.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/vectors/WordValueEncoder.java?rev=998241&r1=998240&r2=998241&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/vectors/WordValueEncoder.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/vectors/WordValueEncoder.java Fri Sep 17 18:41:18 2010
@@ -26,9 +26,11 @@ import java.util.Locale;
  * sub-class.
  */
 public abstract class WordValueEncoder extends FeatureVectorEncoder {
+  private byte[] nameBytes;
 
   protected WordValueEncoder(String name) {
     super(name, 2);
+    nameBytes = bytesForString(name);
   }
 
   /**
@@ -38,26 +40,28 @@ public abstract class WordValueEncoder e
    * @param data         The vector to which the value should be added.
    */
   @Override
-  public void addToVector(String originalForm, double w, Vector data) {
+  public void addToVector(byte[] originalForm, double w, Vector data) {
     int probes = getProbes();
     String name = getName();
     double weight = getWeight(originalForm,w);
     for (int i = 0; i < probes; i++) {
       int n = hashForProbe(originalForm, data.size(), name, i);
-      trace(originalForm, n);
+      if(isTraceEnabled()){
+        trace(originalForm, n);        
+      }
       data.set(n, data.get(n) + weight);
     }
   }
 
 
   @Override
-  protected double getWeight(String originalForm, double w) {
+  protected double getWeight(byte[] originalForm, double w) {
     return w * weight(originalForm);
   }
 
   @Override
-  protected int hashForProbe(String originalForm, int dataSize, String name, int probe) {
-    return hash(name, originalForm, WORD_LIKE_VALUE_HASH_SEED + probe, dataSize);
+  protected int hashForProbe(byte[] originalForm, int dataSize, String name, int probe) {
+    return hash(nameBytes, originalForm, WORD_LIKE_VALUE_HASH_SEED + probe, dataSize);
   }
 
     /**
@@ -70,8 +74,8 @@ public abstract class WordValueEncoder e
    */
   @Override
   public String asString(String originalForm) {
-    return String.format(Locale.ENGLISH, "%s:%s:%.4f", getName(), originalForm, weight(originalForm));
+    return String.format(Locale.ENGLISH, "%s:%s:%.4f", getName(), originalForm, weight(bytesForString(originalForm)));
   }
 
-  protected abstract double weight(String originalForm);
+  protected abstract double weight(byte[] originalForm);
 }

Modified: mahout/trunk/core/src/test/java/org/apache/mahout/vectors/InteractionValueEncoderTest.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/test/java/org/apache/mahout/vectors/InteractionValueEncoderTest.java?rev=998241&r1=998240&r2=998241&view=diff
==============================================================================
--- mahout/trunk/core/src/test/java/org/apache/mahout/vectors/InteractionValueEncoderTest.java (original)
+++ mahout/trunk/core/src/test/java/org/apache/mahout/vectors/InteractionValueEncoderTest.java Fri Sep 17 18:41:18 2010
@@ -22,8 +22,7 @@ import org.apache.mahout.math.DenseVecto
 import org.apache.mahout.math.Vector;
 import org.junit.Test;
 
-public final class InteractionValueEncoderTest extends MahoutTestCase {
-
+public class InteractionValueEncoderTest extends MahoutTestCase {
   @Test
   public void testAddToVector() {
     WordValueEncoder wv = new StaticWordValueEncoder("word");
@@ -72,7 +71,7 @@ public final class InteractionValueEncod
     enc.addInteractionToVector("a","some text here",1.0, v1);
     int k = enc.getProbes();
     // should interact "a" with each of "some","text" and "here"
-    assertEquals((float) k*3, v1.norm(1), 0);    
+    assertEquals((float) k*3, v1.norm(1), 0);
   }
 
 }

Modified: mahout/trunk/core/src/test/java/org/apache/mahout/vectors/MurmurHashTest.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/test/java/org/apache/mahout/vectors/MurmurHashTest.java?rev=998241&r1=998240&r2=998241&view=diff
==============================================================================
--- mahout/trunk/core/src/test/java/org/apache/mahout/vectors/MurmurHashTest.java (original)
+++ mahout/trunk/core/src/test/java/org/apache/mahout/vectors/MurmurHashTest.java Fri Sep 17 18:41:18 2010
@@ -17,25 +17,28 @@
 
 package org.apache.mahout.vectors;
 
+import com.google.common.base.Charsets;
 import org.apache.mahout.common.MahoutTestCase;
 import org.junit.Test;
 
-public final class MurmurHashTest extends MahoutTestCase {
-
-  @Test
-  public void testHash64() throws Exception {
-      // test data generated by running MurmurHash2_64.cpp
-      assertEquals(0x9cc9c33498a95efbL, MurmurHash.hash64A("abc".getBytes("UTF-8"), 0));
-      assertEquals(0xd2c8c9b470122bddL, MurmurHash.hash64A("abc def ghi jkl ".getBytes("UTF-8"), 0));
-      assertEquals(0xcd37895736a81cbcL, MurmurHash.hash64A("abc def ghi jkl moreGoo".getBytes("UTF-8"), 0));
-  }
-
-  @Test
-  public void testHash() throws Exception {
-      // test data generated by running MurmurHashNeutral2.cpp
-      assertEquals(0x13577c9b, MurmurHash.hash("abc".getBytes("UTF-8"), 0));
-      assertEquals(0x6fec441b, MurmurHash.hash("abc def ghi jkl ".getBytes("UTF-8"), 0));
-      assertEquals(0x7e953277, MurmurHash.hash("abc def ghi jkl moreGoo".getBytes("UTF-8"), 0));
-  }
+import java.io.UnsupportedEncodingException;
+import java.nio.ByteBuffer;
 
+public class MurmurHashTest extends MahoutTestCase {
+    @Test
+    public void testForLotsOfChange() throws UnsupportedEncodingException {
+        long h1 = MurmurHash.hash64A("abc".getBytes(Charsets.UTF_8), 0);
+        long h2 = MurmurHash.hash64A("abc ".getBytes(Charsets.UTF_8), 0);
+        int flipCount = Long.bitCount(h1 ^ h2);
+        assertTrue("Small changes should result in lots of bit flips, only found " + flipCount, flipCount > 25);
+    }
+
+    @Test
+    public void testHash64() {
+        // test data generated by running MurmurHash2_64.cpp
+        assertEquals(0x9cc9c33498a95efbL, MurmurHash.hash64A("abc".getBytes(Charsets.UTF_8), 0));
+        assertEquals(0xd2c8c9b470122bddL, MurmurHash.hash64A("abc def ghi jkl ".getBytes(Charsets.UTF_8), 0));
+        assertEquals(0xcd37895736a81cbcL, MurmurHash.hash64A("abc def ghi jkl moreGoo".getBytes(Charsets.UTF_8), 0));
+    }
 }
+

Modified: mahout/trunk/core/src/test/java/org/apache/mahout/vectors/TextValueEncoderTest.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/test/java/org/apache/mahout/vectors/TextValueEncoderTest.java?rev=998241&r1=998240&r2=998241&view=diff
==============================================================================
--- mahout/trunk/core/src/test/java/org/apache/mahout/vectors/TextValueEncoderTest.java (original)
+++ mahout/trunk/core/src/test/java/org/apache/mahout/vectors/TextValueEncoderTest.java Fri Sep 17 18:41:18 2010
@@ -17,6 +17,7 @@
 
 package org.apache.mahout.vectors;
 
+import com.google.common.base.Charsets;
 import com.google.common.collect.ImmutableMap;
 import org.apache.mahout.common.MahoutTestCase;
 import org.apache.mahout.math.DenseVector;
@@ -38,7 +39,7 @@ public final class TextValueEncoderTest 
 
     // now some fancy weighting
     StaticWordValueEncoder w = new StaticWordValueEncoder("text");
-    w.setDictionary(ImmutableMap.<String, Double>of("word1", 3.0, "word2", 1.5));
+    w.setDictionary(ImmutableMap.<byte[], Double>of("word1".getBytes(Charsets.UTF_8), 3.0, "word2".getBytes(Charsets.UTF_8), 1.5));
     enc.setWordEncoder(w);
 
     // should set 6 locations to something

Modified: mahout/trunk/core/src/test/java/org/apache/mahout/vectors/WordLikeValueEncoderTest.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/test/java/org/apache/mahout/vectors/WordLikeValueEncoderTest.java?rev=998241&r1=998240&r2=998241&view=diff
==============================================================================
--- mahout/trunk/core/src/test/java/org/apache/mahout/vectors/WordLikeValueEncoderTest.java (original)
+++ mahout/trunk/core/src/test/java/org/apache/mahout/vectors/WordLikeValueEncoderTest.java Fri Sep 17 18:41:18 2010
@@ -17,6 +17,7 @@
 
 package org.apache.mahout.vectors;
 
+import com.google.common.base.Charsets;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import org.apache.mahout.common.MahoutTestCase;
@@ -55,7 +56,7 @@ public final class WordLikeValueEncoderT
   @Test
   public void testStaticWeights() {
     StaticWordValueEncoder enc = new StaticWordValueEncoder("word");
-    enc.setDictionary(ImmutableMap.<String, Double>of("word1", 3.0, "word2", 1.5));
+    enc.setDictionary(ImmutableMap.<byte[], Double>of("word1".getBytes(Charsets.UTF_8), 3.0, "word2".getBytes(Charsets.UTF_8), 1.5));
     Vector v = new DenseVector(200);
     enc.addToVector("word1", v);
     enc.addToVector("word2", v);