You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dw...@apache.org on 2015/07/16 12:32:07 UTC

svn commit: r1691350 [1/3] - in /lucene/dev/branches/solr7787: ./ lucene/ solr/ solr/core/ solr/core/src/java/org/apache/solr/handler/component/ solr/core/src/java/org/apache/solr/search/facet/ solr/core/src/java/org/apache/solr/util/hll/ solr/core/src...

Author: dweiss
Date: Thu Jul 16 10:32:07 2015
New Revision: 1691350

URL: http://svn.apache.org/r1691350
Log:
SOLR-7787 (jhll integration).

Added:
    lucene/dev/branches/solr7787/   (props changed)
      - copied from r1691343, lucene/dev/trunk/
    lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/
    lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/BigEndianAscendingWordDeserializer.java   (with props)
    lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/BigEndianAscendingWordSerializer.java   (with props)
    lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/BitUtil.java   (with props)
    lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/BitVector.java   (with props)
    lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/HLL.java   (with props)
    lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/HLLMetadata.java   (with props)
    lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/HLLType.java   (with props)
    lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/HLLUtil.java   (with props)
    lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/IHLLMetadata.java   (with props)
    lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/ISchemaVersion.java   (with props)
    lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/IWordDeserializer.java   (with props)
    lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/IWordSerializer.java   (with props)
    lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/LongIterator.java   (with props)
    lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/NumberUtil.java   (with props)
    lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/SchemaVersionOne.java   (with props)
    lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/SerializationUtil.java   (with props)
    lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/package-info.java   (with props)
Modified:
    lucene/dev/branches/solr7787/lucene/ivy-versions.properties
    lucene/dev/branches/solr7787/solr/NOTICE.txt
    lucene/dev/branches/solr7787/solr/core/ivy.xml
    lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/handler/component/StatsField.java
    lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
    lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java
    lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/search/facet/UniqueSlotAcc.java
    lucene/dev/branches/solr7787/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java
    lucene/dev/branches/solr7787/solr/core/src/test/org/apache/solr/handler/component/TestDistributedStatsComponentCardinality.java
    lucene/dev/branches/solr7787/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java

Modified: lucene/dev/branches/solr7787/lucene/ivy-versions.properties
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr7787/lucene/ivy-versions.properties?rev=1691350&r1=1691343&r2=1691350&view=diff
==============================================================================
--- lucene/dev/branches/solr7787/lucene/ivy-versions.properties (original)
+++ lucene/dev/branches/solr7787/lucene/ivy-versions.properties Thu Jul 16 10:32:07 2015
@@ -73,7 +73,6 @@ com.sun.jersey.version = 1.9
 /hsqldb/hsqldb = 1.8.0.10
 /io.airlift/slice = 0.10
 /io.netty/netty = 3.7.0.Final
-/it.unimi.dsi/fastutil = 6.5.11
 /jakarta-regexp/jakarta-regexp = 1.4
 /javax.activation/activation = 1.1.1
 /javax.inject/javax.inject= 1
@@ -85,7 +84,6 @@ com.sun.jersey.version = 1.9
 /log4j/log4j = 1.2.17
 /mecab/mecab-ipadic = 2.7.0-20070801
 /mecab/mecab-naist-jdic = 0.6.3b-20111013
-/net.agkn/hll = 1.6.0
 /net.arnx/jsonic = 1.2.7
 /net.sf.ehcache/ehcache-core = 2.4.4
 /net.sf.saxon/Saxon-HE = 9.6.0-2

Modified: lucene/dev/branches/solr7787/solr/NOTICE.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr7787/solr/NOTICE.txt?rev=1691350&r1=1691343&r2=1691350&view=diff
==============================================================================
--- lucene/dev/branches/solr7787/solr/NOTICE.txt (original)
+++ lucene/dev/branches/solr7787/solr/NOTICE.txt Thu Jul 16 10:32:07 2015
@@ -13,6 +13,9 @@ including, but not limited to:
   - Apache Blur
   - Apache Hadoop
 
+This product includes code forked from the Java-HLL library.
+Copyright (c) 2013 Aggregate Knowledge, Inc., https://github.com/aggregateknowledge/java-hll/
+
 This product includes the JQuery JavaScript library created by John Resig.
 Copyright (c) 2010 John Resig, http://jquery.com/
 

Modified: lucene/dev/branches/solr7787/solr/core/ivy.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr7787/solr/core/ivy.xml?rev=1691350&r1=1691343&r2=1691350&view=diff
==============================================================================
--- lucene/dev/branches/solr7787/solr/core/ivy.xml (original)
+++ lucene/dev/branches/solr7787/solr/core/ivy.xml Thu Jul 16 10:32:07 2015
@@ -134,10 +134,6 @@
     <dependency org="org.antlr" name="antlr4-runtime" rev="${/org.antlr/antlr4-runtime}"/>
     <dependency org="io.airlift" name="slice" rev="${/io.airlift/slice}"/>
 
-    <!-- StatsComponents HLL Dependencies-->
-    <dependency org="net.agkn" name="hll" rev="${/net.agkn/hll}" conf="compile->*"/>
-    <dependency org="it.unimi.dsi" name="fastutil" rev="${/it.unimi.dsi/fastutil}" conf="compile->*"/>
-
     <exclude org="*" ext="*" matcher="regexp" type="${ivy.exclude.types}"/> 
   </dependencies>
 </ivy-module>

Modified: lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/handler/component/StatsField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/handler/component/StatsField.java?rev=1691350&r1=1691343&r2=1691350&view=diff
==============================================================================
--- lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/handler/component/StatsField.java (original)
+++ lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/handler/component/StatsField.java Thu Jul 16 10:32:07 2015
@@ -55,9 +55,9 @@ import org.apache.solr.search.QParserPlu
 import org.apache.solr.search.QueryParsing;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.search.SyntaxError;
+import org.apache.solr.util.hll.HLL;
+import org.apache.solr.util.hll.HLLType;
 
-import net.agkn.hll.HLL;
-import net.agkn.hll.HLLType;
 import com.google.common.hash.Hashing;
 import com.google.common.hash.HashFunction;
 
@@ -625,8 +625,8 @@ public class StatsField {
      * Creates an HllOptions based on the (local) params specified (if appropriate).
      *
      * @param localParams the LocalParams for this {@link StatsField}
-     * @param field the field corrisponding to this {@link StatsField}, may be null if these stats are over a value source
-     * @return the {@link HllOptions} to use basd on the params, or null if no {@link HLL} should be computed
+     * @param field the field corresponding to this {@link StatsField}, may be null if these stats are over a value source
+     * @return the {@link HllOptions} to use based on the params, or null if no {@link HLL} should be computed
      * @throws SolrException if there are invalid options
      */
     public static HllOptions parseHllOptions(SolrParams localParams, SchemaField field) 

Modified: lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java?rev=1691350&r1=1691343&r2=1691350&view=diff
==============================================================================
--- lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java (original)
+++ lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java Thu Jul 16 10:32:07 2015
@@ -33,12 +33,12 @@ import org.apache.solr.handler.component
 import org.apache.solr.schema.*;
 
 import com.tdunning.math.stats.AVLTreeDigest;
-
-import net.agkn.hll.HLL;
-import net.agkn.hll.HLLType;
 import com.google.common.hash.Hashing;
 import com.google.common.hash.HashFunction;
 
+import org.apache.solr.util.hll.HLL;
+import org.apache.solr.util.hll.HLLType;
+
 /**
  * Factory class for creating instance of 
  * {@link org.apache.solr.handler.component.StatsValues}

Modified: lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java?rev=1691350&r1=1691343&r2=1691350&view=diff
==============================================================================
--- lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java (original)
+++ lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/search/facet/HLLAgg.java Thu Jul 16 10:32:07 2015
@@ -23,8 +23,8 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
-import net.agkn.hll.HLL;
-import net.agkn.hll.HLLType;
+import org.apache.solr.util.hll.HLL;
+import org.apache.solr.util.hll.HLLType;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;

Modified: lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/search/facet/UniqueSlotAcc.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/search/facet/UniqueSlotAcc.java?rev=1691350&r1=1691343&r2=1691350&view=diff
==============================================================================
--- lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/search/facet/UniqueSlotAcc.java (original)
+++ lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/search/facet/UniqueSlotAcc.java Thu Jul 16 10:32:07 2015
@@ -21,7 +21,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
-import net.agkn.hll.HLL;
+import org.apache.solr.util.hll.HLL;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.MultiDocValues;
 import org.apache.lucene.index.SortedDocValues;

Added: lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/BigEndianAscendingWordDeserializer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/BigEndianAscendingWordDeserializer.java?rev=1691350&view=auto
==============================================================================
--- lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/BigEndianAscendingWordDeserializer.java (added)
+++ lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/BigEndianAscendingWordDeserializer.java Thu Jul 16 10:32:07 2015
@@ -0,0 +1,173 @@
+package org.apache.solr.util.hll;
+
+/*
+ * 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.
+ */
+
+/**
+ * A corresponding deserializer for {@link BigEndianAscendingWordSerializer}.
+ */
+class BigEndianAscendingWordDeserializer implements IWordDeserializer {
+    // The number of bits per byte.
+    private static final int BITS_PER_BYTE = 8;
+
+    // long mask for the maximum value stored in a byte
+    private static final long BYTE_MASK = (1L << BITS_PER_BYTE) - 1L;
+
+    // ************************************************************************
+    // The length in bits of the words to be read.
+    private final int wordLength;
+
+    // The byte array to which the words are serialized.
+    private final byte[] bytes;
+
+    // The number of leading padding bytes in 'bytes' to be ignored.
+    private final int bytePadding;
+
+    // The number of words that the byte array contains.
+    private final int wordCount;
+
+    // The current read state.
+    private int currentWordIndex;
+
+    // ========================================================================
+    /**
+     * @param wordLength the length in bits of the words to be deserialized. Must
+     *        be less than or equal to 64 and greater than or equal to 1.
+     * @param bytePadding the number of leading bytes that pad the serialized words.
+     *        Must be greater than or equal to zero.
+     * @param bytes the byte array containing the serialized words. Cannot be
+     *        <code>null</code>.
+     */
+    public BigEndianAscendingWordDeserializer(final int wordLength, final int bytePadding, final byte[] bytes) {
+        if((wordLength < 1) || (wordLength > 64)) {
+            throw new IllegalArgumentException("Word length must be >= 1 and <= 64. (was: " + wordLength + ")");
+        }
+
+        if(bytePadding < 0) {
+            throw new IllegalArgumentException("Byte padding must be >= zero. (was: " + bytePadding + ")");
+        }
+
+        this.wordLength = wordLength;
+        this.bytes = bytes;
+        this.bytePadding = bytePadding;
+
+        final int dataBytes = (bytes.length - bytePadding);
+        final long dataBits = (dataBytes * BITS_PER_BYTE);
+
+        this.wordCount = (int)(dataBits/wordLength);
+
+        currentWordIndex = 0;
+    }
+
+    // ========================================================================
+    /* (non-Javadoc)
+     * @see net.agkn.hll.serialization.IWordDeserializer#readWord()
+     */
+    @Override
+    public long readWord() {
+        final long word = readWord(currentWordIndex);
+        currentWordIndex++;
+
+        return word;
+    }
+
+    // ------------------------------------------------------------------------
+    /**
+     * Reads the word at the specified sequence position (zero-indexed).
+     *
+     * @param  position the zero-indexed position of the word to be read. This
+     *         must be greater than or equal to zero.
+     * @return the value of the serialized word at the specified position.
+     */
+    private long readWord(final int position) {
+        if(position < 0) {
+            throw new ArrayIndexOutOfBoundsException(position);
+        }
+
+        // First bit of the word
+        final long firstBitIndex = (position * wordLength);
+        final int firstByteIndex = (bytePadding + (int)(firstBitIndex / BITS_PER_BYTE));
+        final int firstByteSkipBits = (int)(firstBitIndex % BITS_PER_BYTE);
+
+        // Last bit of the word
+        final long lastBitIndex = (firstBitIndex + wordLength - 1);
+        final int lastByteIndex = (bytePadding + (int)(lastBitIndex / BITS_PER_BYTE));
+        final int lastByteBitsToConsume;
+
+        final int bitsAfterByteBoundary = (int)((lastBitIndex + 1) % BITS_PER_BYTE);
+        // If the word terminates at the end of the last byte, consume the whole
+        // last byte.
+        if(bitsAfterByteBoundary == 0) {
+            lastByteBitsToConsume = BITS_PER_BYTE;
+        } else {
+            // Otherwise, only consume what is necessary.
+            lastByteBitsToConsume = bitsAfterByteBoundary;
+        }
+
+        if(lastByteIndex >= bytes.length) {
+            throw new ArrayIndexOutOfBoundsException("Word out of bounds of backing array.");
+        }
+
+        // Accumulator
+        long value = 0;
+
+        // --------------------------------------------------------------------
+        // First byte
+        final int bitsRemainingInFirstByte = (BITS_PER_BYTE - firstByteSkipBits);
+        final int bitsToConsumeInFirstByte = Math.min(bitsRemainingInFirstByte, wordLength);
+        long firstByte = (long)bytes[firstByteIndex];
+
+        // Mask off the bits to skip in the first byte.
+        final long firstByteMask = ((1L << bitsRemainingInFirstByte) - 1L);
+        firstByte &= firstByteMask;
+        // Right-align relevant bits of first byte.
+        firstByte >>>= (bitsRemainingInFirstByte - bitsToConsumeInFirstByte);
+
+        value |= firstByte;
+
+        // If the first byte contains the whole word, short-circuit.
+        if(firstByteIndex == lastByteIndex) {
+            return value;
+        }
+
+        // --------------------------------------------------------------------
+        // Middle bytes
+        final int middleByteCount = (lastByteIndex - firstByteIndex - 1);
+        for(int i=0; i<middleByteCount; i++) {
+            final long middleByte = (bytes[firstByteIndex + i + 1] & BYTE_MASK);
+            // Push middle byte onto accumulator.
+            value <<= BITS_PER_BYTE;
+            value |= middleByte;
+        }
+
+        // --------------------------------------------------------------------
+        // Last byte
+        long lastByte = (bytes[lastByteIndex] & BYTE_MASK);
+        lastByte >>= (BITS_PER_BYTE - lastByteBitsToConsume);
+        value <<= lastByteBitsToConsume;
+        value |= lastByte;
+        return value;
+    }
+
+    /* (non-Javadoc)
+     * @see net.agkn.hll.serialization.IWordDeserializer#totalWordCount()
+     */
+    @Override
+    public int totalWordCount() {
+        return wordCount;
+    }
+}

Added: lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/BigEndianAscendingWordSerializer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/BigEndianAscendingWordSerializer.java?rev=1691350&view=auto
==============================================================================
--- lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/BigEndianAscendingWordSerializer.java (added)
+++ lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/BigEndianAscendingWordSerializer.java Thu Jul 16 10:32:07 2015
@@ -0,0 +1,174 @@
+package org.apache.solr.util.hll;
+
+/*
+ * 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.
+ */
+
+/**
+ * A serializer that writes a sequence of fixed bit-width 'words' to a byte array.
+ * Bitwise OR is used to write words into bytes, so a low bit in a word is also
+ * a low bit in a byte. However, a high byte in a word is written at a lower index
+ * in the array than a low byte in a word. The first word is written at the lowest
+ * array index. Each serializer is one time use and returns its backing byte
+ * array.<p/>
+ *
+ * This encoding was chosen so that when reading bytes as octets in the typical
+ * first-octet-is-the-high-nibble fashion, an octet-to-binary conversion
+ * would yield a high-to-low, left-to-right view of the "short words".<p/>
+ *
+ * Example:<p/>
+ *
+ * Say short words are 5 bits wide. Our word sequence is the values
+ * <code>[31, 1, 5]</code>. In big-endian binary format, the values are
+ * <code>[0b11111, 0b00001, 0b00101]</code>. We use 15 of 16 bits in two bytes
+ * and pad the last (lowest) bit of the last byte with a zero:
+ *
+ * <code>
+ *  [0b11111000, 0b01001010] = [0xF8, 0x4A]
+ * </code>.
+ */
+class BigEndianAscendingWordSerializer implements IWordSerializer {
+    // The number of bits per byte.
+    private static final int BITS_PER_BYTE = 8;
+
+    // ************************************************************************
+    // The length in bits of the words to be written.
+    private final int wordLength;
+    // The number of words to be written.
+    private final int wordCount;
+
+    // The byte array to which the words are serialized.
+    private final byte[] bytes;
+
+    // ------------------------------------------------------------------------
+    // Write state
+    // Number of bits that remain writable in the current byte.
+    private int bitsLeftInByte;
+    // Index of byte currently being written to.
+    private int byteIndex;
+    // Number of words written.
+    private int wordsWritten;
+
+    // ========================================================================
+    /**
+     * @param wordLength the length in bits of the words to be serialized. Must
+     *        be greater than or equal to 1 and less than or equal to 64.
+     * @param wordCount the number of words to be serialized. Must be greater than
+     *        or equal to zero.
+     * @param bytePadding the number of leading bytes that should pad the
+     *        serialized words. Must be greater than or equal to zero.
+     */
+    public BigEndianAscendingWordSerializer(final int wordLength, final int wordCount, final int bytePadding) {
+        if((wordLength < 1) || (wordLength > 64)) {
+            throw new IllegalArgumentException("Word length must be >= 1 and <= 64. (was: " + wordLength + ")");
+        }
+        if(wordCount < 0) {
+            throw new IllegalArgumentException("Word count must be >= 0. (was: " + wordCount + ")");
+        }
+        if(bytePadding < 0) {
+            throw new IllegalArgumentException("Byte padding must be must be >= 0. (was: " + bytePadding + ")");
+        }
+
+        this.wordLength = wordLength;
+        this.wordCount = wordCount;
+
+        final long bitsRequired = (wordLength * wordCount);
+        final boolean leftoverBits = ((bitsRequired % BITS_PER_BYTE) != 0);
+        final int bytesRequired = (int)(bitsRequired / BITS_PER_BYTE) + (leftoverBits ? 1 : 0) + bytePadding;
+        bytes = new byte[bytesRequired];
+
+        bitsLeftInByte = BITS_PER_BYTE;
+        byteIndex = bytePadding;
+        wordsWritten = 0;
+    }
+
+    /* (non-Javadoc)
+     * @see net.agkn.hll.serialization.IWordSerializer#writeWord(long)
+     * @throws RuntimeException if the number of words written is greater than the
+     *         <code>wordCount</code> parameter in the constructor.
+     */
+    @Override
+    public void writeWord(final long word) {
+        if(wordsWritten == wordCount) {
+            throw new RuntimeException("Cannot write more words, backing array full!");
+        }
+
+        int bitsLeftInWord = wordLength;
+
+        while(bitsLeftInWord > 0) {
+            // Move to the next byte if the current one is fully packed.
+            if(bitsLeftInByte == 0) {
+                byteIndex++;
+                bitsLeftInByte = BITS_PER_BYTE;
+            }
+
+            final long consumedMask;
+            if(bitsLeftInWord == 64) {
+                consumedMask = ~0L;
+            } else {
+                consumedMask = ((1L << bitsLeftInWord) - 1L);
+            }
+
+            // Fix how many bits will be written in this cycle. Choose the
+            // smaller of the remaining bits in the word or byte.
+            final int numberOfBitsToWrite = Math.min(bitsLeftInByte, bitsLeftInWord);
+            final int bitsInByteRemainingAfterWrite = (bitsLeftInByte - numberOfBitsToWrite);
+
+            // In general, we write the highest bits of the word first, so we
+            // strip the highest bits that were consumed in previous cycles.
+            final long remainingBitsOfWordToWrite = (word & consumedMask);
+
+            final long bitsThatTheByteCanAccept;
+            // If there is more left in the word than can be written to this
+            // byte, shift off the bits that can't be written off the bottom.
+            if(bitsLeftInWord > numberOfBitsToWrite) {
+                bitsThatTheByteCanAccept = (remainingBitsOfWordToWrite >>> (bitsLeftInWord - bitsLeftInByte));
+            } else {
+                // If the byte can accept all remaining bits, there is no need
+                // to shift off the bits that won't be written in this cycle.
+                bitsThatTheByteCanAccept = remainingBitsOfWordToWrite;
+            }
+
+            // Align the word bits to write up against the byte bits that have
+            // already been written. This shift may do nothing if the remainder
+            // of the byte is being consumed in this cycle.
+            final long alignedBits = (bitsThatTheByteCanAccept << bitsInByteRemainingAfterWrite);
+
+            // Update the byte with the alignedBits.
+            bytes[byteIndex] |= (byte)alignedBits;
+
+            // Update state with bit count written.
+            bitsLeftInWord -= numberOfBitsToWrite;
+            bitsLeftInByte = bitsInByteRemainingAfterWrite;
+        }
+
+        wordsWritten ++;
+    }
+
+    /* (non-Javadoc)
+     * @see net.agkn.hll.serialization.IWordSerializer#getBytes()
+     * @throws RuntimeException if the number of words written is fewer than the
+     *         <code>wordCount</code> parameter in the constructor.
+     */
+    @Override
+    public byte[] getBytes() {
+        if(wordsWritten < wordCount) {
+            throw new RuntimeException("Not all words have been written! (" + wordsWritten + "/" + wordCount + ")");
+        }
+
+        return bytes;
+    }
+}

Added: lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/BitUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/BitUtil.java?rev=1691350&view=auto
==============================================================================
--- lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/BitUtil.java (added)
+++ lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/BitUtil.java Thu Jul 16 10:32:07 2015
@@ -0,0 +1,71 @@
+package org.apache.solr.util.hll;
+
+/*
+ * 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.
+ */
+
+/**
+ * A collection of bit utilities.
+ */
+class BitUtil {
+    /**
+     * The set of least-significant bits for a given <code>byte</code>.  <code>-1</code>
+     * is used if no bits are set (so as to not be confused with "index of zero"
+     * meaning that the least significant bit is the 0th (1st) bit).
+     *
+     * @see #leastSignificantBit(long)
+     */
+    private static final int[] LEAST_SIGNIFICANT_BIT = {
+       -1, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0,
+        4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0,
+        5, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0,
+        4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0,
+        6, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0,
+        4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0,
+        5, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0,
+        4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0,
+        7, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0,
+        4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0,
+        5, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0,
+        4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0,
+        6, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0,
+        4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0,
+        5, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0,
+        4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0
+    };
+
+    /**
+     * Computes the least-significant bit of the specified <code>long</code>
+     * that is set to <code>1</code>. Zero-indexed.
+     *
+     * @param  value the <code>long</code> whose least-significant bit is desired.
+     * @return the least-significant bit of the specified <code>long</code>.
+     *         <code>-1</code> is returned if there are no bits set.
+     */
+    // REF:  http://stackoverflow.com/questions/757059/position-of-least-significant-bit-that-is-set
+    // REF:  http://www-graphics.stanford.edu/~seander/bithacks.html
+    public static int leastSignificantBit(final long value) {
+        if(value == 0L) return -1/*by contract*/;
+        if((value & 0xFFL) != 0) return LEAST_SIGNIFICANT_BIT[(int)( (value >>>  0) & 0xFF)] +  0;
+        if((value & 0xFFFFL) != 0) return LEAST_SIGNIFICANT_BIT[(int)( (value >>>  8) & 0xFF)] +  8;
+        if((value & 0xFFFFFFL) != 0) return LEAST_SIGNIFICANT_BIT[(int)( (value >>> 16) & 0xFF)] + 16;
+        if((value & 0xFFFFFFFFL) != 0) return LEAST_SIGNIFICANT_BIT[(int)( (value >>> 24) & 0xFF)] + 24;
+        if((value & 0xFFFFFFFFFFL) != 0) return LEAST_SIGNIFICANT_BIT[(int)( (value >>> 32) & 0xFF)] + 32;
+        if((value & 0xFFFFFFFFFFFFL) != 0) return LEAST_SIGNIFICANT_BIT[(int)( (value >>> 40) & 0xFF)] + 40;
+        if((value & 0xFFFFFFFFFFFFFFL) != 0) return LEAST_SIGNIFICANT_BIT[(int)( (value >>> 48) & 0xFF)] + 48;
+        return LEAST_SIGNIFICANT_BIT[(int)( (value >>> 56) & 0xFFL)] + 56;
+    }
+}
\ No newline at end of file

Added: lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/BitVector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/BitVector.java?rev=1691350&view=auto
==============================================================================
--- lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/BitVector.java (added)
+++ lucene/dev/branches/solr7787/solr/core/src/java/org/apache/solr/util/hll/BitVector.java Thu Jul 16 10:32:07 2015
@@ -0,0 +1,259 @@
+package org.apache.solr.util.hll;
+
+/*
+ * 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.
+ */
+
+/**
+ * A vector (array) of bits that is accessed in units ("registers") of <code>width</code>
+ * bits which are stored as 64bit "words" (<code>long</code>s).  In this context
+ * a register is at most 64bits.
+ */
+class BitVector implements Cloneable {
+    // NOTE:  in this context, a word is 64bits
+
+    // rather than doing division to determine how a bit index fits into 64bit
+    // words (i.e. longs), bit shifting is used
+    private static final int LOG2_BITS_PER_WORD = 6/*=>64bits*/;
+    private static final int BITS_PER_WORD = 1 << LOG2_BITS_PER_WORD;
+    private static final int BITS_PER_WORD_MASK = BITS_PER_WORD - 1;
+
+    // ditto from above but for bytes (for output)
+    private static final int LOG2_BITS_PER_BYTE = 3/*=>8bits*/;
+    public static final int BITS_PER_BYTE = 1 << LOG2_BITS_PER_BYTE;
+
+    // ========================================================================
+    public static final int BYTES_PER_WORD = 8/*8 bytes in a long*/;
+
+    // ************************************************************************
+    // 64bit words
+    private final long[] words;
+    public final long[] words() { return words; }
+    public final int wordCount() { return words.length; }
+    public final int byteCount() { return wordCount() * BYTES_PER_WORD; }
+
+    // the width of a register in bits (this cannot be more than 64 (the word size))
+    private final int registerWidth;
+    public final int registerWidth() { return registerWidth; }
+
+    private final long count;
+
+    // ------------------------------------------------------------------------
+    private final long registerMask;
+
+    // ========================================================================
+    /**
+     * @param  width the width of each register.  This cannot be negative or
+     *         zero or greater than 63 (the signed word size).
+     * @param  count the number of registers.  This cannot be negative or zero
+     */
+    public BitVector(final int width, final long count) {
+        // ceil((width * count)/BITS_PER_WORD)
+        this.words = new long[(int)(((width * count) + BITS_PER_WORD_MASK) >>> LOG2_BITS_PER_WORD)];
+        this.registerWidth = width;
+        this.count = count;
+
+        this.registerMask = (1L << width) - 1;
+    }
+
+    // ========================================================================
+    /**
+     * @param  registerIndex the index of the register whose value is to be
+     *         retrieved.  This cannot be negative.
+     * @return the value at the specified register index
+     * @see #setRegister(long, long)
+     * @see #setMaxRegister(long, long)
+     */
+    // NOTE:  if this changes then setMaxRegister() must change
+    public long getRegister(final long registerIndex) {
+        final long bitIndex = registerIndex * registerWidth;
+        final int firstWordIndex = (int)(bitIndex >>> LOG2_BITS_PER_WORD)/*aka (bitIndex / BITS_PER_WORD)*/;
+        final int secondWordIndex = (int)((bitIndex + registerWidth - 1) >>> LOG2_BITS_PER_WORD)/*see above*/;
+        final int bitRemainder = (int)(bitIndex & BITS_PER_WORD_MASK)/*aka (bitIndex % BITS_PER_WORD)*/;
+
+        if(firstWordIndex == secondWordIndex)
+            return ((words[firstWordIndex] >>> bitRemainder) & registerMask);
+        /* else -- register spans words */
+        return (words[firstWordIndex] >>> bitRemainder)/*no need to mask since at top of word*/
+             | (words[secondWordIndex] << (BITS_PER_WORD - bitRemainder)) & registerMask;
+    }
+
+    /**
+     * @param registerIndex the index of the register whose value is to be set.
+     *        This cannot be negative
+     * @param value the value to set in the register
+     * @see #getRegister(long)
+     * @see #setMaxRegister(long, long)
+     */
+    // NOTE:  if this changes then setMaxRegister() must change
+    public void setRegister(final long registerIndex, final long value) {
+        final long bitIndex = registerIndex * registerWidth;
+        final int firstWordIndex = (int)(bitIndex >>> LOG2_BITS_PER_WORD)/*aka (bitIndex / BITS_PER_WORD)*/;
+        final int secondWordIndex = (int)((bitIndex + registerWidth - 1) >>> LOG2_BITS_PER_WORD)/*see above*/;
+        final int bitRemainder = (int)(bitIndex & BITS_PER_WORD_MASK)/*aka (bitIndex % BITS_PER_WORD)*/;
+
+        final long words[] = this.words/*for convenience/performance*/;
+        if(firstWordIndex == secondWordIndex) {
+            // clear then set
+            words[firstWordIndex] &= ~(registerMask << bitRemainder);
+            words[firstWordIndex] |= (value << bitRemainder);
+        } else {/*register spans words*/
+            // clear then set each partial word
+            words[firstWordIndex] &= (1L << bitRemainder) - 1;
+            words[firstWordIndex] |= (value << bitRemainder);
+
+            words[secondWordIndex] &= ~(registerMask >>> (BITS_PER_WORD - bitRemainder));
+            words[secondWordIndex] |= (value >>> (BITS_PER_WORD - bitRemainder));
+        }
+    }
+
+    // ------------------------------------------------------------------------
+    /**
+     * @return a <code>LongIterator</code> for iterating starting at the register
+     *         with index zero. This will never be <code>null</code>.
+     */
+    public LongIterator registerIterator() {
+        return new LongIterator() {
+            final int registerWidth = BitVector.this.registerWidth;
+            final long[] words = BitVector.this.words;
+            final long registerMask = BitVector.this.registerMask;
+
+            // register setup
+            long registerIndex = 0;
+            int wordIndex = 0;
+            int remainingWordBits = BITS_PER_WORD;
+            long word = words[wordIndex];
+
+            @Override public long next() {
+                long register;
+                if(remainingWordBits >= registerWidth) {
+                    register = word & registerMask;
+
+                    // shift to the next register
+                    word >>>= registerWidth;
+                    remainingWordBits -= registerWidth;
+                } else { /*insufficient bits remaining in current word*/
+                    wordIndex++/*move to the next word*/;
+
+                    register = (word | (words[wordIndex] << remainingWordBits)) & registerMask;
+
+                    // shift to the next partial register (word)
+                    word = words[wordIndex] >>> (registerWidth - remainingWordBits);
+                    remainingWordBits += BITS_PER_WORD - registerWidth;
+                }
+                registerIndex++;
+                return register;
+            }
+
+            @Override public boolean hasNext() {
+                return registerIndex < count;
+            }
+        };
+    }
+
+    // ------------------------------------------------------------------------
+    // composite accessors
+    /**
+     * Sets the value of the specified index register if and only if the specified
+     * value is greater than the current value in the register.  This is equivalent
+     * to but much more performant than:<p/>
+     *
+     * <pre>vector.setRegister(index, Math.max(vector.getRegister(index), value));</pre>
+     *
+     * @param  registerIndex the index of the register whose value is to be set.
+     *         This cannot be negative
+     * @param  value the value to set in the register if and only if this value
+     *         is greater than the current value in the register
+     * @return <code>true</code> if and only if the specified value is greater
+     *         than or equal to the current register value.  <code>false</code>
+     *         otherwise.
+     * @see #getRegister(long)
+     * @see #setRegister(long, long)
+     * @see java.lang.Math#max(long, long)
+     */
+    // NOTE:  if this changes then setRegister() must change
+    public boolean setMaxRegister(final long registerIndex, final long value) {
+        final long bitIndex = registerIndex * registerWidth;
+        final int firstWordIndex = (int)(bitIndex >>> LOG2_BITS_PER_WORD)/*aka (bitIndex / BITS_PER_WORD)*/;
+        final int secondWordIndex = (int)((bitIndex + registerWidth - 1) >>> LOG2_BITS_PER_WORD)/*see above*/;
+        final int bitRemainder = (int)(bitIndex & BITS_PER_WORD_MASK)/*aka (bitIndex % BITS_PER_WORD)*/;
+
+        // NOTE:  matches getRegister()
+        final long registerValue;
+        final long words[] = this.words/*for convenience/performance*/;
+        if(firstWordIndex == secondWordIndex)
+            registerValue = ((words[firstWordIndex] >>> bitRemainder) & registerMask);
+        else /*register spans words*/
+            registerValue = (words[firstWordIndex] >>> bitRemainder)/*no need to mask since at top of word*/
+                          | (words[secondWordIndex] << (BITS_PER_WORD - bitRemainder)) & registerMask;
+
+        // determine which is the larger and update as necessary
+        if(value > registerValue) {
+            // NOTE:  matches setRegister()
+            if(firstWordIndex == secondWordIndex) {
+                // clear then set
+                words[firstWordIndex] &= ~(registerMask << bitRemainder);
+                words[firstWordIndex] |= (value << bitRemainder);
+            } else {/*register spans words*/
+                // clear then set each partial word
+                words[firstWordIndex] &= (1L << bitRemainder) - 1;
+                words[firstWordIndex] |= (value << bitRemainder);
+
+                words[secondWordIndex] &= ~(registerMask >>> (BITS_PER_WORD - bitRemainder));
+                words[secondWordIndex] |= (value >>> (BITS_PER_WORD - bitRemainder));
+            }
+        } /* else -- the register value is greater (or equal) so nothing needs to be done */
+
+        return (value >= registerValue);
+    }
+
+    // ========================================================================
+    /**
+     * Fills this bit vector with the specified bit value.  This can be used to
+     * clear the vector by specifying <code>0</code>.
+     *
+     * @param  value the value to set all bits to (only the lowest bit is used)
+     */
+    public void fill(final long value) {
+        for(long i=0; i<count; i++) {
+            setRegister(i, value);
+        }
+    }
+
+    // ------------------------------------------------------------------------
+    /**
+     * Serializes the registers of the vector using the specified serializer.
+     *
+     * @param serializer the serializer to use. This cannot be <code>null</code>.
+     */
+    public void getRegisterContents(final IWordSerializer serializer) {
+        for(final LongIterator iter = registerIterator(); iter.hasNext();) {
+            serializer.writeWord(iter.next());
+        }
+    }
+
+    /**
+     * Creates a deep copy of this vector.
+     *
+     * @see java.lang.Object#clone()
+     */
+    @Override
+    public BitVector clone() {
+        final BitVector copy = new BitVector(registerWidth, count);
+        System.arraycopy(words, 0, copy.words, 0, words.length);
+        return copy;
+    }
+}
\ No newline at end of file