You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2014/04/15 11:35:35 UTC

svn commit: r1587489 - in /lucene/dev/trunk/lucene: ./ analysis/common/src/java/org/apache/lucene/analysis/hunspell/ analysis/common/src/java/org/apache/lucene/analysis/synonym/ analysis/common/src/java/org/apache/lucene/collation/tokenattributes/ anal...

Author: mikemccand
Date: Tue Apr 15 09:35:34 2014
New Revision: 1587489

URL: http://svn.apache.org/r1587489
Log:
LUCENE-5604: switch BytesRef/BytesRefHash hashing to MurmurHash3

Added:
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestStringHelper.java   (with props)
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Dictionary.java
    lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymMap.java
    lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/collation/tokenattributes/CollatedTermAttributeImpl.java
    lucene/dev/trunk/lucene/analysis/icu/src/java/org/apache/lucene/collation/tokenattributes/ICUCollatedTermAttributeImpl.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/NumericTokenStream.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/CharTermAttributeImpl.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/TermToBytesRefAttribute.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRef.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/StringHelper.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/UnicodeUtil.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java
    lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/CellTokenStream.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/CannedBinaryTokenStream.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockUTF16TermAttributeImpl.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1587489&r1=1587488&r2=1587489&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Tue Apr 15 09:35:34 2014
@@ -84,6 +84,11 @@ API Changes
   IndexOutput.getFilePointer instead) and IndexOutput.setLength.
   (Mike McCandless)
 
+* LUCENE-5604: Switched BytesRef.hashCode to MurmurHash3 (32 bit).
+  TermToBytesRefAttribute.fillBytesRef no longer returns the hash
+  code.  BytesRefHash now uses MurmurHash3 for its hashing.  (Robert
+  Muir, Mike McCandless)
+
 Optimizations
 
 * LUCENE-5603: hunspell stemmer more efficiently strips prefixes

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Dictionary.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Dictionary.java?rev=1587489&r1=1587488&r2=1587489&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Dictionary.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Dictionary.java Tue Apr 15 09:35:34 2014
@@ -464,8 +464,8 @@ public class Dictionary {
         appendFlags = NOFLAGS;
       }
       
-      final int hashCode = encodeFlagsWithHash(scratch, appendFlags);
-      int appendFlagsOrd = flagLookup.add(scratch, hashCode);
+      encodeFlags(scratch, appendFlags);
+      int appendFlagsOrd = flagLookup.add(scratch);
       if (appendFlagsOrd < 0) {
         // already exists in our hash
         appendFlagsOrd = (-appendFlagsOrd)-1;
@@ -765,8 +765,8 @@ public class Dictionary {
       if (cmp < 0) {
         throw new IllegalArgumentException("out of order: " + entry + " < " + currentEntry);
       } else {
-        final int hashCode = encodeFlagsWithHash(flagsScratch, wordForm);
-        int ord = flagLookup.add(flagsScratch, hashCode);
+        encodeFlags(flagsScratch, wordForm);
+        int ord = flagLookup.add(flagsScratch);
         if (ord < 0) {
           // already exists in our hash
           ord = (-ord)-1;
@@ -808,18 +808,16 @@ public class Dictionary {
     return flags;
   }
   
-  static int encodeFlagsWithHash(BytesRef b, char flags[]) {
-    int hash = 0;
+  static void encodeFlags(BytesRef b, char flags[]) {
     int len = flags.length << 1;
     b.grow(len);
     b.length = len;
     int upto = b.offset;
     for (int i = 0; i < flags.length; i++) {
       int flag = flags[i];
-      hash = 31*hash + (b.bytes[upto++] = (byte) ((flag >> 8) & 0xff));
-      hash = 31*hash + (b.bytes[upto++] = (byte) (flag & 0xff));
+      b.bytes[upto++] = (byte) ((flag >> 8) & 0xff);
+      b.bytes[upto++] = (byte) (flag & 0xff);
     }
-    return hash;
   }
 
   private void parseAlias(String line) {

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymMap.java?rev=1587489&r1=1587488&r2=1587489&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymMap.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymMap.java Tue Apr 15 09:35:34 2014
@@ -153,9 +153,9 @@ public class SynonymMap {
       assert !hasHoles(output): "output has holes: " + output;
 
       //System.out.println("fmap.add input=" + input + " numInputWords=" + numInputWords + " output=" + output + " numOutputWords=" + numOutputWords);
-      final int hashCode = UnicodeUtil.UTF16toUTF8WithHash(output.chars, output.offset, output.length, utf8Scratch);
+      UnicodeUtil.UTF16toUTF8(output.chars, output.offset, output.length, utf8Scratch);
       // lookup in hash
-      int ord = words.add(utf8Scratch, hashCode);
+      int ord = words.add(utf8Scratch);
       if (ord < 0) {
         // already exists in our hash
         ord = (-ord)-1;

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/collation/tokenattributes/CollatedTermAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/collation/tokenattributes/CollatedTermAttributeImpl.java?rev=1587489&r1=1587488&r2=1587489&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/collation/tokenattributes/CollatedTermAttributeImpl.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/collation/tokenattributes/CollatedTermAttributeImpl.java Tue Apr 15 09:35:34 2014
@@ -40,12 +40,11 @@ public class CollatedTermAttributeImpl e
   }
   
   @Override
-  public int fillBytesRef() {
+  public void fillBytesRef() {
     BytesRef bytes = getBytesRef();
     bytes.bytes = collator.getCollationKey(toString()).toByteArray();
     bytes.offset = 0;
     bytes.length = bytes.bytes.length;
-    return bytes.hashCode();
   }
 
 }

Modified: lucene/dev/trunk/lucene/analysis/icu/src/java/org/apache/lucene/collation/tokenattributes/ICUCollatedTermAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/icu/src/java/org/apache/lucene/collation/tokenattributes/ICUCollatedTermAttributeImpl.java?rev=1587489&r1=1587488&r2=1587489&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/icu/src/java/org/apache/lucene/collation/tokenattributes/ICUCollatedTermAttributeImpl.java (original)
+++ lucene/dev/trunk/lucene/analysis/icu/src/java/org/apache/lucene/collation/tokenattributes/ICUCollatedTermAttributeImpl.java Tue Apr 15 09:35:34 2014
@@ -45,12 +45,11 @@ public class ICUCollatedTermAttributeImp
   }
 
   @Override
-  public int fillBytesRef() {
+  public void fillBytesRef() {
     BytesRef bytes = getBytesRef();
     collator.getRawCollationKey(toString(), key);
     bytes.bytes = key.bytes;
     bytes.offset = 0;
     bytes.length = key.size;
-    return bytes.hashCode();
   }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/NumericTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/NumericTokenStream.java?rev=1587489&r1=1587488&r2=1587489&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/NumericTokenStream.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/NumericTokenStream.java Tue Apr 15 09:35:34 2014
@@ -159,11 +159,13 @@ public final class NumericTokenStream ex
     }
     
     @Override
-    public int fillBytesRef() {
+    public void fillBytesRef() {
       assert valueSize == 64 || valueSize == 32;
-      return (valueSize == 64) ? 
-        NumericUtils.longToPrefixCoded(value, shift, bytes) :
+      if (valueSize == 64) {
+        NumericUtils.longToPrefixCoded(value, shift, bytes);
+      } else {
         NumericUtils.intToPrefixCoded((int) value, shift, bytes);
+      }
     }
 
     @Override

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/CharTermAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/CharTermAttributeImpl.java?rev=1587489&r1=1587488&r2=1587489&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/CharTermAttributeImpl.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/CharTermAttributeImpl.java Tue Apr 15 09:35:34 2014
@@ -21,8 +21,8 @@ import java.nio.CharBuffer;
 
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.AttributeImpl;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.AttributeReflector;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.UnicodeUtil;
 
@@ -86,8 +86,8 @@ public class CharTermAttributeImpl exten
   private BytesRef bytes = new BytesRef(MIN_BUFFER_SIZE);
 
   @Override
-  public int fillBytesRef() {
-    return UnicodeUtil.UTF16toUTF8WithHash(termBuffer, 0, termLength, bytes);
+  public void fillBytesRef() {
+    UnicodeUtil.UTF16toUTF8(termBuffer, 0, termLength, bytes);
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/TermToBytesRefAttribute.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/TermToBytesRefAttribute.java?rev=1587489&r1=1587488&r2=1587489&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/TermToBytesRefAttribute.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/TermToBytesRefAttribute.java Tue Apr 15 09:35:34 2014
@@ -52,21 +52,12 @@ import org.apache.lucene.util.BytesRef;
  * for UTF-8 terms.
  */
 public interface TermToBytesRefAttribute extends Attribute {
+
   /** 
    * Updates the bytes {@link #getBytesRef()} to contain this term's
-   * final encoding, and returns its hashcode.
-   * @return the hashcode as defined by {@link BytesRef#hashCode}:
-   * <pre class="prettyprint">
-   *  int hash = 0;
-   *  for (int i = termBytes.offset; i &lt; termBytes.offset+termBytes.length; i++) {
-   *    hash = 31*hash + termBytes.bytes[i];
-   *  }
-   * </pre>
-   * Implement this for performance reasons, if your code can calculate
-   * the hash on-the-fly. If this is not the case, just return
-   * {@code termBytes.hashCode()}.
+   * final encoding.
    */
-  public int fillBytesRef();
+  public void fillBytesRef();
   
   /**
    * Retrieve this attribute's BytesRef. The bytes are updated 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java?rev=1587489&r1=1587488&r2=1587489&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java Tue Apr 15 09:35:34 2014
@@ -172,12 +172,14 @@ final class TermsHashPerField extends In
   @Override
   void add() throws IOException {
 
+    termAtt.fillBytesRef();
+
     // We are first in the chain so we must "intern" the
     // term text into textStart address
     // Get the text & hash of this term.
     int termID;
     try {
-      termID = bytesHash.add(termBytesRef, termAtt.fillBytesRef());
+      termID = bytesHash.add(termBytesRef);
     } catch (MaxBytesLengthExceededException e) {
       // Term is too large; record this here (can't throw an
       // exc because DocInverterPerField will then abort the

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRef.java?rev=1587489&r1=1587488&r2=1587489&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRef.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRef.java Tue Apr 15 09:35:34 2014
@@ -132,22 +132,13 @@ public final class BytesRef implements C
   }
   
   /** Calculates the hash code as required by TermsHash during indexing.
-   * <p>It is defined as:
-   * <pre class="prettyprint">
-   *  int hash = 0;
-   *  for (int i = offset; i &lt; offset + length; i++) {
-   *    hash = 31*hash + bytes[i];
-   *  }
-   * </pre>
-   */
+   *  <p> This is currently implemented as MurmurHash3 (32
+   *  bit), using the seed from {@link
+   *  StringHelper#GOOD_FAST_HASH_SEED}, but is subject to
+   *  change from release to release. */
   @Override
   public int hashCode() {
-    int hash = 0;
-    final int end = offset + length;
-    for(int i=offset;i<end;i++) {
-      hash = 31 * hash + bytes[i];
-    }
-    return hash;
+    return StringHelper.murmurhash3_x86_32(this, StringHelper.GOOD_FAST_HASH_SEED);
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java?rev=1587489&r1=1587488&r2=1587489&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java Tue Apr 15 09:35:34 2014
@@ -17,16 +17,16 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
-import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK;
-import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SHIFT;
-import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
-
 import java.util.Arrays;
 import java.util.Comparator;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.util.ByteBlockPool.DirectAllocator;
 
+import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK;
+import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SHIFT;
+import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
+
 /**
  * {@link BytesRefHash} is a special purpose hash-map like data-structure
  * optimized for {@link BytesRef} instances. BytesRefHash maintains mappings of
@@ -271,41 +271,10 @@ public final class BytesRefHash {
    *           {@link ByteBlockPool#BYTE_BLOCK_SIZE}
    */
   public int add(BytesRef bytes) {
-    return add(bytes, bytes.hashCode());
-  }
-
-  /**
-   * Adds a new {@link BytesRef} with a pre-calculated hash code.
-   * 
-   * @param bytes
-   *          the bytes to hash
-   * @param code
-   *          the bytes hash code
-   * 
-   *          <p>
-   *          Hashcode is defined as:
-   * 
-   *          <pre class="prettyprint">
-   * int hash = 0;
-   * for (int i = offset; i &lt; offset + length; i++) {
-   *   hash = 31 * hash + bytes[i];
-   * }
-   * </pre>
-   * 
-   * @return the id the given bytes are hashed if there was no mapping for the
-   *         given bytes, otherwise <code>(-(id)-1)</code>. This guarantees
-   *         that the return value will always be &gt;= 0 if the given bytes
-   *         haven't been hashed before.
-   * 
-   * @throws MaxBytesLengthExceededException
-   *           if the given bytes are >
-   *           {@link ByteBlockPool#BYTE_BLOCK_SIZE} - 2
-   */
-  public int add(BytesRef bytes, int code) {
     assert bytesStart != null : "Bytesstart is null - not initialized";
     final int length = bytes.length;
     // final position
-    final int hashPos = findHash(bytes, code);
+    final int hashPos = findHash(bytes);
     int e = ids[hashPos];
     
     if (e == -1) {
@@ -362,38 +331,32 @@ public final class BytesRefHash {
   /**
    * Returns the id of the given {@link BytesRef}.
    * 
-   * @see #find(BytesRef, int)
-   */
-  public int find(BytesRef bytes) {
-    return find(bytes, bytes.hashCode());
-  }
-
-  /**
-   * Returns the id of the given {@link BytesRef} with a pre-calculated hash code.
-   * 
    * @param bytes
    *          the bytes to look for
-   * @param code
-   *          the bytes hash code
    * 
    * @return the id of the given bytes, or {@code -1} if there is no mapping for the
    *         given bytes.
    */
-  public int find(BytesRef bytes, int code) {
-    return ids[findHash(bytes, code)];
+  public int find(BytesRef bytes) {
+    return ids[findHash(bytes)];
   }
-  
-  private int findHash(BytesRef bytes, int code) {
+
+  public static int totConflict;
+
+  private int findHash(BytesRef bytes) {
     assert bytesStart != null : "bytesStart is null - not initialized";
+
+    int code = doHash(bytes.bytes, bytes.offset, bytes.length);
+
     // final position
     int hashPos = code & hashMask;
     int e = ids[hashPos];
     if (e != -1 && !equals(e, bytes)) {
-      // Conflict: keep searching different locations in
-      // the hash table.
-      final int inc = ((code >> 8) + code) | 1;
+      // Conflict; use linear probe to find an open slot
+      // (see LUCENE-5604):
       do {
-        code += inc;
+        totConflict++;
+        code++;
         hashPos = code & hashMask;
         e = ids[hashPos];
       } while (e != -1 && !equals(e, bytes));
@@ -415,11 +378,10 @@ public final class BytesRefHash {
     int hashPos = offset & hashMask;
     int e = ids[hashPos];
     if (e != -1 && bytesStart[e] != offset) {
-      // Conflict: keep searching different locations in
-      // the hash table.
-      final int inc = ((code >> 8) + code) | 1;
+      // Conflict; use linear probe to find an open slot
+      // (see LUCENE-5604):
       do {
-        code += inc;
+        code++;
         hashPos = code & hashMask;
         e = ids[hashPos];
       } while (e != -1 && bytesStart[e] != offset);
@@ -461,7 +423,6 @@ public final class BytesRefHash {
           final int off = bytesStart[e0];
           final int start = off & BYTE_BLOCK_MASK;
           final byte[] bytes = pool.buffers[off >> BYTE_BLOCK_SHIFT];
-          code = 0;
           final int len;
           int pos;
           if ((bytes[start] & 0x80) == 0) {
@@ -472,11 +433,7 @@ public final class BytesRefHash {
             len = (bytes[start] & 0x7f) + ((bytes[start + 1] & 0xff) << 7);
             pos = start + 2;
           }
-
-          final int endPos = pos + len;
-          while (pos < endPos) {
-            code = 31 * code + bytes[pos++];
-          }
+          code = doHash(bytes, pos, len);
         } else {
           code = bytesStart[e0];
         }
@@ -484,9 +441,10 @@ public final class BytesRefHash {
         int hashPos = code & newMask;
         assert hashPos >= 0;
         if (newHash[hashPos] != -1) {
-          final int inc = ((code >> 8) + code) | 1;
+          // Conflict; use linear probe to find an open slot
+          // (see LUCENE-5604):
           do {
-            code += inc;
+            code++;
             hashPos = code & newMask;
           } while (newHash[hashPos] != -1);
         }
@@ -501,6 +459,11 @@ public final class BytesRefHash {
     hashHalfSize = newSize / 2;
   }
 
+  // TODO: maybe use long?  But our keys are typically short...
+  private int doHash(byte[] bytes, int offset, int length) {
+    return StringHelper.murmurhash3_x86_32(bytes, offset, length, StringHelper.GOOD_FAST_HASH_SEED);
+  }
+
   /**
    * reinitializes the {@link BytesRefHash} after a previous {@link #clear()}
    * call. If {@link #clear()} has not been called previously this method has no

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java?rev=1587489&r1=1587488&r2=1587489&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java Tue Apr 15 09:35:34 2014
@@ -108,11 +108,9 @@ public final class NumericUtils {
    * @param val the numeric value
    * @param shift how many bits to strip from the right
    * @param bytes will contain the encoded value
-   * @return the hash code for indexing (TermsHash)
    */
-  public static int longToPrefixCoded(final long val, final int shift, final BytesRef bytes) {
+  public static void longToPrefixCoded(final long val, final int shift, final BytesRef bytes) {
     longToPrefixCodedBytes(val, shift, bytes);
-    return bytes.hashCode();
   }
 
   /**
@@ -122,11 +120,9 @@ public final class NumericUtils {
    * @param val the numeric value
    * @param shift how many bits to strip from the right
    * @param bytes will contain the encoded value
-   * @return the hash code for indexing (TermsHash)
    */
-  public static int intToPrefixCoded(final int val, final int shift, final BytesRef bytes) {
+  public static void intToPrefixCoded(final int val, final int shift, final BytesRef bytes) {
     intToPrefixCodedBytes(val, shift, bytes);
-    return bytes.hashCode();
   }
 
   /**

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/StringHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/StringHelper.java?rev=1587489&r1=1587488&r2=1587489&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/StringHelper.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/StringHelper.java Tue Apr 15 09:35:34 2014
@@ -1,8 +1,5 @@
 package org.apache.lucene.util;
 
-import java.util.Comparator;
-import java.util.StringTokenizer;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -20,6 +17,8 @@ import java.util.StringTokenizer;
  * limitations under the License.
  */
 
+import java.util.Comparator;
+import java.util.StringTokenizer;
 
 /**
  * Methods for manipulating strings.
@@ -144,4 +143,85 @@ public abstract class StringHelper {
     
     return true;
   }
+
+  /** Pass this as the seed to {@link #murmurhash3_x86_32}. */
+
+  // Poached from Guava: set a different salt/seed
+  // for each JVM instance, to frustrate hash key collision
+  // denial of service attacks, and to catch any places that
+  // somehow rely on hash function/order across JVM
+  // instances:
+  public static final int GOOD_FAST_HASH_SEED;
+
+  static {
+    String prop = System.getProperty("tests.seed");
+    if (prop != null) {
+      // So if there is a test failure that relied on hash
+      // order, we remain reproducible based on the test seed:
+      if (prop.length() > 8) {
+        prop = prop.substring(prop.length()-8);
+      }
+      GOOD_FAST_HASH_SEED = (int) Long.parseLong(prop, 16);
+    } else {
+      GOOD_FAST_HASH_SEED = (int) System.currentTimeMillis();
+    }
+  }
+
+  /** Returns the MurmurHash3_x86_32 hash.
+   * Original source/tests at https://github.com/yonik/java_util/
+   */
+  public static int murmurhash3_x86_32(byte[] data, int offset, int len, int seed) {
+
+    final int c1 = 0xcc9e2d51;
+    final int c2 = 0x1b873593;
+
+    int h1 = seed;
+    int roundedEnd = offset + (len & 0xfffffffc);  // round down to 4 byte block
+
+    for (int i=offset; i<roundedEnd; i+=4) {
+      // little endian load order
+      int k1 = (data[i] & 0xff) | ((data[i+1] & 0xff) << 8) | ((data[i+2] & 0xff) << 16) | (data[i+3] << 24);
+      k1 *= c1;
+      k1 = Integer.rotateLeft(k1, 15);
+      k1 *= c2;
+
+      h1 ^= k1;
+      h1 = Integer.rotateLeft(h1, 13);
+      h1 = h1*5+0xe6546b64;
+    }
+
+    // tail
+    int k1 = 0;
+
+    switch(len & 0x03) {
+      case 3:
+        k1 = (data[roundedEnd + 2] & 0xff) << 16;
+        // fallthrough
+      case 2:
+        k1 |= (data[roundedEnd + 1] & 0xff) << 8;
+        // fallthrough
+      case 1:
+        k1 |= (data[roundedEnd] & 0xff);
+        k1 *= c1;
+        k1 = Integer.rotateLeft(k1, 15);
+        k1 *= c2;
+        h1 ^= k1;
+    }
+
+    // finalization
+    h1 ^= len;
+
+    // fmix(h1);
+    h1 ^= h1 >>> 16;
+    h1 *= 0x85ebca6b;
+    h1 ^= h1 >>> 13;
+    h1 *= 0xc2b2ae35;
+    h1 ^= h1 >>> 16;
+
+    return h1;
+  }
+
+  public static int murmurhash3_x86_32(BytesRef bytes, int seed) {
+    return murmurhash3_x86_32(bytes.bytes, bytes.offset, bytes.length, seed);
+  }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/UnicodeUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/UnicodeUtil.java?rev=1587489&r1=1587488&r2=1587489&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/UnicodeUtil.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/UnicodeUtil.java Tue Apr 15 09:35:34 2014
@@ -124,62 +124,6 @@ public final class UnicodeUtil {
     (UNI_SUR_HIGH_START << HALF_SHIFT) - UNI_SUR_LOW_START;
 
   /** Encode characters from a char[] source, starting at
-   *  offset for length chars.  Returns a hash of the resulting bytes.  After encoding, result.offset will always be 0. */
-  // TODO: broken if incoming result.offset != 0
-  public static int UTF16toUTF8WithHash(final char[] source, final int offset, final int length, BytesRef result) {
-    int hash = 0;
-    int upto = 0;
-    int i = offset;
-    final int end = offset + length;
-    byte[] out = result.bytes;
-    // Pre-allocate for worst case 4-for-1
-    final int maxLen = length * 4;
-    if (out.length < maxLen)
-      out = result.bytes = new byte[ArrayUtil.oversize(maxLen, 1)];
-    result.offset = 0;
-
-    while(i < end) {
-      
-      final int code = (int) source[i++];
-
-      if (code < 0x80) {
-        hash = 31*hash + (out[upto++] = (byte) code);
-      } else if (code < 0x800) {
-        hash = 31*hash + (out[upto++] = (byte) (0xC0 | (code >> 6)));
-        hash = 31*hash + (out[upto++] = (byte)(0x80 | (code & 0x3F)));
-      } else if (code < 0xD800 || code > 0xDFFF) {
-        hash = 31*hash + (out[upto++] = (byte)(0xE0 | (code >> 12)));
-        hash = 31*hash + (out[upto++] = (byte)(0x80 | ((code >> 6) & 0x3F)));
-        hash = 31*hash + (out[upto++] = (byte)(0x80 | (code & 0x3F)));
-      } else {
-        // surrogate pair
-        // confirm valid high surrogate
-        if (code < 0xDC00 && i < end) {
-          int utf32 = (int) source[i];
-          // confirm valid low surrogate and write pair
-          if (utf32 >= 0xDC00 && utf32 <= 0xDFFF) { 
-            utf32 = (code << 10) + utf32 + SURROGATE_OFFSET;
-            i++;
-            hash = 31*hash + (out[upto++] = (byte)(0xF0 | (utf32 >> 18)));
-            hash = 31*hash + (out[upto++] = (byte)(0x80 | ((utf32 >> 12) & 0x3F)));
-            hash = 31*hash + (out[upto++] = (byte)(0x80 | ((utf32 >> 6) & 0x3F)));
-            hash = 31*hash + (out[upto++] = (byte)(0x80 | (utf32 & 0x3F)));
-            continue;
-          }
-        }
-        // replace unpaired surrogate or out-of-order low surrogate
-        // with substitution character
-        hash = 31*hash + (out[upto++] = (byte) 0xEF);
-        hash = 31*hash + (out[upto++] = (byte) 0xBF);
-        hash = 31*hash + (out[upto++] = (byte) 0xBD);
-      }
-    }
-    //assert matches(source, offset, length, out, upto);
-    result.length = upto;
-    return hash;
-  }
-
-  /** Encode characters from a char[] source, starting at
    *  offset for length chars. After encoding, result.offset will always be 0.
    */
   // TODO: broken if incoming result.offset != 0

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java?rev=1587489&r1=1587488&r2=1587489&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java Tue Apr 15 09:35:34 2014
@@ -41,8 +41,7 @@ public class TestNumericTokenStream exte
     for (int shift=0; shift<64; shift+=NumericUtils.PRECISION_STEP_DEFAULT) {
       assertTrue("New token is available", stream.incrementToken());
       assertEquals("Shift value wrong", shift, numericAtt.getShift());
-      final int hash = bytesAtt.fillBytesRef();
-      assertEquals("Hash incorrect", bytes.hashCode(), hash);
+      bytesAtt.fillBytesRef();
       assertEquals("Term is incorrectly encoded", lvalue & ~((1L << shift) - 1L), NumericUtils.prefixCodedToLong(bytes));
       assertEquals("Term raw value is incorrectly encoded", lvalue & ~((1L << shift) - 1L), numericAtt.getRawValue());
       assertEquals("Type incorrect", (shift == 0) ? NumericTokenStream.TOKEN_TYPE_FULL_PREC : NumericTokenStream.TOKEN_TYPE_LOWER_PREC, typeAtt.type());
@@ -64,8 +63,7 @@ public class TestNumericTokenStream exte
     for (int shift=0; shift<32; shift+=NumericUtils.PRECISION_STEP_DEFAULT) {
       assertTrue("New token is available", stream.incrementToken());
       assertEquals("Shift value wrong", shift, numericAtt.getShift());
-      final int hash = bytesAtt.fillBytesRef();
-      assertEquals("Hash incorrect", bytes.hashCode(), hash);
+      bytesAtt.fillBytesRef();
       assertEquals("Term is incorrectly encoded", ivalue & ~((1 << shift) - 1), NumericUtils.prefixCodedToInt(bytes));
       assertEquals("Term raw value is incorrectly encoded", ((long) ivalue) & ~((1L << shift) - 1L), numericAtt.getRawValue());
       assertEquals("Type incorrect", (shift == 0) ? NumericTokenStream.TOKEN_TYPE_FULL_PREC : NumericTokenStream.TOKEN_TYPE_LOWER_PREC, typeAtt.type());

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java?rev=1587489&r1=1587488&r2=1587489&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java Tue Apr 15 09:35:34 2014
@@ -62,8 +62,8 @@ public final class BinaryTokenStream ext
     private BytesRef bytes;
     
     @Override
-    public int fillBytesRef() {
-      return bytes.hashCode();
+    public void fillBytesRef() {
+      // no-op: the bytes was already filled by our owner's incrementToken
     }
     
     @Override

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java?rev=1587489&r1=1587488&r2=1587489&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java Tue Apr 15 09:35:34 2014
@@ -87,8 +87,8 @@ public class Test2BTerms extends LuceneT
 
     private final static class MyTermAttributeImpl extends AttributeImpl implements TermToBytesRefAttribute {
       @Override
-      public int fillBytesRef() {
-        return bytes.hashCode();
+      public void fillBytesRef() {
+        // no-op: the bytes was already filled by our owner's incrementToken
       }
       
       @Override

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestStringHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestStringHelper.java?rev=1587489&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestStringHelper.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestStringHelper.java Tue Apr 15 09:35:34 2014
@@ -0,0 +1,29 @@
+package org.apache.lucene.util;
+
+/*
+ * 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.
+ */
+
+public class TestStringHelper extends LuceneTestCase {
+
+  public void testMurmurHash3() throws Exception {
+    // Hashes computed using murmur3_32 from https://code.google.com/p/pyfasthash
+    assertEquals(0xf6a5c420, StringHelper.murmurhash3_x86_32(new BytesRef("foo"), 0));
+    assertEquals(0xcd018ef6, StringHelper.murmurhash3_x86_32(new BytesRef("foo"), 16));
+    assertEquals(0x111e7435, StringHelper.murmurhash3_x86_32(new BytesRef("You want weapons? We're in a library! Books! The best weapons in the world!"), 0));
+    assertEquals(0x2c628cd0, StringHelper.murmurhash3_x86_32(new BytesRef("You want weapons? We're in a library! Books! The best weapons in the world!"), 3476));
+  }
+}

Modified: lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/CellTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/CellTokenStream.java?rev=1587489&r1=1587488&r2=1587489&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/CellTokenStream.java (original)
+++ lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/CellTokenStream.java Tue Apr 15 09:35:34 2014
@@ -107,12 +107,11 @@ class CellTokenStream extends TokenStrea
     }
 
     @Override
-    public int fillBytesRef() {
+    public void fillBytesRef() {
       if (omitLeafByte)
         cell.getTokenBytesNoLeaf(bytes);
       else
         cell.getTokenBytes(bytes);
-      return bytes.hashCode();
     }
 
     @Override

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/CannedBinaryTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/CannedBinaryTokenStream.java?rev=1587489&r1=1587488&r2=1587489&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/CannedBinaryTokenStream.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/CannedBinaryTokenStream.java Tue Apr 15 09:35:34 2014
@@ -72,8 +72,8 @@ public final class CannedBinaryTokenStre
     private final BytesRef bytes = new BytesRef();
 
     @Override
-    public int fillBytesRef() {
-      return bytes.hashCode();
+    public void fillBytesRef() {
+      // no-op: we already filled externally during owner's incrementToken
     }
       
     @Override

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockUTF16TermAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockUTF16TermAttributeImpl.java?rev=1587489&r1=1587488&r2=1587489&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockUTF16TermAttributeImpl.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/MockUTF16TermAttributeImpl.java Tue Apr 15 09:35:34 2014
@@ -30,12 +30,11 @@ public class MockUTF16TermAttributeImpl 
   static final Charset charset = Charset.forName("UTF-16LE");
   
   @Override
-  public int fillBytesRef() {
+  public void fillBytesRef() {
     BytesRef bytes = getBytesRef();
     byte[] utf16 = toString().getBytes(charset);
     bytes.bytes = utf16;
     bytes.offset = 0;
     bytes.length = utf16.length;
-    return bytes.hashCode();
   }
 }