Return-Path: Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: (qmail 85500 invoked from network); 24 Dec 2010 17:58:54 -0000 Received: from unknown (HELO mail.apache.org) (140.211.11.3) by 140.211.11.9 with SMTP; 24 Dec 2010 17:58:54 -0000 Received: (qmail 36197 invoked by uid 500); 24 Dec 2010 17:58:54 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 36180 invoked by uid 500); 24 Dec 2010 17:58:54 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 36172 invoked by uid 99); 24 Dec 2010 17:58:54 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 24 Dec 2010 17:58:54 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 24 Dec 2010 17:58:46 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id 9F1AE238897D; Fri, 24 Dec 2010 17:58:23 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1052542 [1/2] - in /cassandra/branches/cassandra-0.7: ./ src/java/org/apache/cassandra/db/columniterator/ src/java/org/apache/cassandra/db/filter/ src/java/org/apache/cassandra/io/sstable/ src/java/org/apache/cassandra/utils/ src/java/org/... Date: Fri, 24 Dec 2010 17:58:23 -0000 To: commits@cassandra.apache.org From: jbellis@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20101224175823.9F1AE238897D@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: jbellis Date: Fri Dec 24 17:58:22 2010 New Revision: 1052542 URL: http://svn.apache.org/viewvc?rev=1052542&view=rev Log: add OpenBitSet to support larger bloom filters patch by Ryan King, Stu Hood, and tjake for CASSANDRA-1555 Added: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/LegacyBloomFilter.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/LegacyBloomFilterSerializer.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/obs/ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/obs/ArrayUtil.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/obs/BitUtil.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/obs/OpenBitSet.java cassandra/branches/cassandra-0.7/test/long/org/apache/cassandra/utils/LongLegacyBloomFilterTest.java cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/utils/FilterTestHelper.java cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/utils/LegacyBloomFilterTest.java Removed: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/BitSetSerializer.java cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/utils/FilterTest.java Modified: cassandra/branches/cassandra-0.7/CHANGES.txt cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/columniterator/IndexedSliceReader.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/columniterator/SimpleSliceReader.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/filter/IFilter.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/filter/QueryFilter.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/Descriptor.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/IndexHelper.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/BloomCalculations.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/BloomFilter.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/Filter.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/MurmurHash.java cassandra/branches/cassandra-0.7/test/long/org/apache/cassandra/utils/LongBloomFilterTest.java cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/utils/BloomFilterTest.java Modified: cassandra/branches/cassandra-0.7/CHANGES.txt URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/CHANGES.txt?rev=1052542&r1=1052541&r2=1052542&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/CHANGES.txt (original) +++ cassandra/branches/cassandra-0.7/CHANGES.txt Fri Dec 24 17:58:22 2010 @@ -8,6 +8,7 @@ dev * fix CLI get recognition of supercolumns (CASSANDRA-1899) * avoid polluting page cache with commitlog or sstable writes and seq scan operations (CASSANDRA-1470) + * add OpenBitSet to support larger bloom filters (CASSANDRA-1555) 0.7.0-rc3 Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/columniterator/IndexedSliceReader.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/columniterator/IndexedSliceReader.java?rev=1052542&r1=1052541&r2=1052542&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/columniterator/IndexedSliceReader.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/columniterator/IndexedSliceReader.java Fri Dec 24 17:58:22 2010 @@ -38,6 +38,7 @@ import org.apache.cassandra.db.marshal.A import org.apache.cassandra.io.sstable.IndexHelper; import org.apache.cassandra.io.util.FileDataInput; import org.apache.cassandra.io.util.FileMark; +import org.apache.cassandra.io.sstable.SSTableReader; /** * This is a reader that finds the block for a starting column and returns @@ -58,19 +59,19 @@ class IndexedSliceReader extends Abstrac private Deque blockColumns = new ArrayDeque(); private AbstractType comparator; - public IndexedSliceReader(CFMetaData metadata, FileDataInput input, ByteBuffer startColumn, ByteBuffer finishColumn, boolean reversed) + public IndexedSliceReader(SSTableReader sstable, FileDataInput input, ByteBuffer startColumn, ByteBuffer finishColumn, boolean reversed) { this.file = input; this.startColumn = startColumn; this.finishColumn = finishColumn; this.reversed = reversed; - comparator = metadata.comparator; + comparator = sstable.metadata.comparator; try { IndexHelper.skipBloomFilter(file); indexes = IndexHelper.deserializeIndex(file); - emptyColumnFamily = ColumnFamily.serializer().deserializeFromSSTableNoColumns(ColumnFamily.create(metadata), file); + emptyColumnFamily = ColumnFamily.serializer().deserializeFromSSTableNoColumns(ColumnFamily.create(sstable.metadata), file); fetcher = indexes == null ? new SimpleBlockFetcher() : new IndexedBlockFetcher(); } catch (IOException e) Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java?rev=1052542&r1=1052541&r2=1052542&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java Fri Dec 24 17:58:22 2010 @@ -39,7 +39,7 @@ import org.apache.cassandra.io.sstable.S import org.apache.cassandra.io.util.FileDataInput; import org.apache.cassandra.io.util.FileMark; import org.apache.cassandra.io.util.FileUtils; -import org.apache.cassandra.utils.BloomFilter; +import org.apache.cassandra.utils.Filter; import org.apache.cassandra.utils.FBUtilities; public class SSTableNamesIterator extends SimpleAbstractColumnIterator implements IColumnIterator @@ -68,7 +68,7 @@ public class SSTableNamesIterator extend FBUtilities.readShortByteArray(file)); assert keyInDisk.equals(key) : String.format("%s != %s in %s", keyInDisk, key, file.getPath()); SSTableReader.readRowSize(file, sstable.descriptor); - read(sstable.metadata, file); + read(sstable, file); } catch (IOException e) { @@ -80,7 +80,7 @@ public class SSTableNamesIterator extend } } - public SSTableNamesIterator(CFMetaData metadata, FileDataInput file, DecoratedKey key, SortedSet columns) + public SSTableNamesIterator(SSTableReader sstable, FileDataInput file, DecoratedKey key, SortedSet columns) { assert columns != null; this.columns = columns; @@ -88,7 +88,7 @@ public class SSTableNamesIterator extend try { - read(metadata, file); + read(sstable, file); } catch (IOException ioe) { @@ -96,18 +96,19 @@ public class SSTableNamesIterator extend } } - private void read(CFMetaData metadata, FileDataInput file) + private void read(SSTableReader sstable, FileDataInput file) throws IOException { // read the requested columns into `cf` /* Read the bloom filter summarizing the columns */ - BloomFilter bf = IndexHelper.defreezeBloomFilter(file); + + Filter bf = IndexHelper.defreezeBloomFilter(file, sstable.descriptor.usesOldBloomFilter); List indexList = IndexHelper.deserializeIndex(file); // we can stop early if bloom filter says none of the columns actually exist -- but, // we can't stop before initializing the cf above, in case there's a relevant tombstone - cf = ColumnFamily.serializer().deserializeFromSSTableNoColumns(ColumnFamily.create(metadata), file); + cf = ColumnFamily.serializer().deserializeFromSSTableNoColumns(ColumnFamily.create(sstable.metadata), file); List filteredColumnNames = new ArrayList(columns.size()); for (ByteBuffer name : columns) @@ -123,7 +124,7 @@ public class SSTableNamesIterator extend if (indexList == null) readSimpleColumns(file, columns, filteredColumnNames); else - readIndexedColumns(metadata, file, columns, filteredColumnNames, indexList); + readIndexedColumns(sstable.metadata, file, columns, filteredColumnNames, indexList); // create an iterator view of the columns we read iter = cf.getSortedColumns().iterator(); Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java?rev=1052542&r1=1052541&r2=1052542&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java Fri Dec 24 17:58:22 2010 @@ -64,7 +64,7 @@ public class SSTableSliceIterator implem throw new IOError(e); } - reader = createReader(sstable.metadata, fileToClose, startColumn, finishColumn, reversed); + reader = createReader(sstable, fileToClose, startColumn, finishColumn, reversed); } /** @@ -79,18 +79,18 @@ public class SSTableSliceIterator implem * @param finishColumn The end of the slice * @param reversed Results are returned in reverse order iff reversed is true. */ - public SSTableSliceIterator(CFMetaData metadata, FileDataInput file, DecoratedKey key, ByteBuffer startColumn, ByteBuffer finishColumn, boolean reversed) + public SSTableSliceIterator(SSTableReader sstable, FileDataInput file, DecoratedKey key, ByteBuffer startColumn, ByteBuffer finishColumn, boolean reversed) { this.key = key; fileToClose = null; - reader = createReader(metadata, file, startColumn, finishColumn, reversed); + reader = createReader(sstable, file, startColumn, finishColumn, reversed); } - private static IColumnIterator createReader(CFMetaData metadata, FileDataInput file, ByteBuffer startColumn, ByteBuffer finishColumn, boolean reversed) + private static IColumnIterator createReader(SSTableReader sstable, FileDataInput file, ByteBuffer startColumn, ByteBuffer finishColumn, boolean reversed) { return startColumn.remaining() == 0 && !reversed - ? new SimpleSliceReader(metadata, file, finishColumn) - : new IndexedSliceReader(metadata, file, startColumn, finishColumn, reversed); + ? new SimpleSliceReader(sstable, file, finishColumn) + : new IndexedSliceReader(sstable, file, startColumn, finishColumn, reversed); } public DecoratedKey getKey() Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/columniterator/SimpleSliceReader.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/columniterator/SimpleSliceReader.java?rev=1052542&r1=1052541&r2=1052542&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/columniterator/SimpleSliceReader.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/columniterator/SimpleSliceReader.java Fri Dec 24 17:58:22 2010 @@ -33,6 +33,7 @@ import org.apache.cassandra.db.Decorated import org.apache.cassandra.db.IColumn; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.io.sstable.IndexHelper; +import org.apache.cassandra.io.sstable.SSTableReader; import org.apache.cassandra.io.util.FileDataInput; import org.apache.cassandra.io.util.FileMark; @@ -46,17 +47,17 @@ class SimpleSliceReader extends Abstract private int i; private FileMark mark; - public SimpleSliceReader(CFMetaData metadata, FileDataInput input, ByteBuffer finishColumn) + public SimpleSliceReader(SSTableReader sstable, FileDataInput input, ByteBuffer finishColumn) { this.file = input; this.finishColumn = finishColumn; - comparator = metadata.comparator; + comparator = sstable.metadata.comparator; try { IndexHelper.skipBloomFilter(file); IndexHelper.skipIndex(file); - emptyColumnFamily = ColumnFamily.serializer().deserializeFromSSTableNoColumns(ColumnFamily.create(metadata), file); + emptyColumnFamily = ColumnFamily.serializer().deserializeFromSSTableNoColumns(ColumnFamily.create(sstable.metadata), file); columns = file.readInt(); mark = file.mark(); } Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/filter/IFilter.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/filter/IFilter.java?rev=1052542&r1=1052541&r2=1052542&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/filter/IFilter.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/filter/IFilter.java Fri Dec 24 17:58:22 2010 @@ -48,11 +48,11 @@ public interface IFilter /** * Get an iterator that returns columns from the given SSTable using the opened file * matching the Filter criteria in sorted order. - * @param metadata + * @param sstable * @param file Already opened file data input, saves us opening another one * @param key The key of the row we are about to iterate over */ - public abstract IColumnIterator getSSTableColumnIterator(CFMetaData metadata, FileDataInput file, DecoratedKey key); + public abstract IColumnIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput file, DecoratedKey key); /** * returns an iterator that returns columns from the given SSTable Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java?rev=1052542&r1=1052541&r2=1052542&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java Fri Dec 24 17:58:22 2010 @@ -59,9 +59,9 @@ public class NamesQueryFilter implements return new SSTableNamesIterator(sstable, key, columns); } - public IColumnIterator getSSTableColumnIterator(CFMetaData metadata, FileDataInput file, DecoratedKey key) + public IColumnIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput file, DecoratedKey key) { - return new SSTableNamesIterator(metadata, file, key, columns); + return new SSTableNamesIterator(sstable, file, key, columns); } public SuperColumn filterSuperColumn(SuperColumn superColumn, int gcBefore) Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/filter/QueryFilter.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/filter/QueryFilter.java?rev=1052542&r1=1052541&r2=1052542&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/filter/QueryFilter.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/filter/QueryFilter.java Fri Dec 24 17:58:22 2010 @@ -84,8 +84,8 @@ public class QueryFilter public IColumnIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput file, DecoratedKey key) { if (path.superColumnName == null) - return filter.getSSTableColumnIterator(sstable.metadata, file, key); - return superFilter.getSSTableColumnIterator(sstable.metadata, file, key); + return filter.getSSTableColumnIterator(sstable, file, key); + return superFilter.getSSTableColumnIterator(sstable, file, key); } // here so it can be used by SQF and NQF. non-package callers should call IFilter.getColumnComparator Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java?rev=1052542&r1=1052541&r2=1052542&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java Fri Dec 24 17:58:22 2010 @@ -68,9 +68,9 @@ public class SliceQueryFilter implements return new SSTableSliceIterator(sstable, key, start, finish, reversed); } - public IColumnIterator getSSTableColumnIterator(CFMetaData metadata, FileDataInput file, DecoratedKey key) + public IColumnIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput file, DecoratedKey key) { - return new SSTableSliceIterator(metadata, file, key, start, finish, reversed); + return new SSTableSliceIterator(sstable, file, key, start, finish, reversed); } public SuperColumn filterSuperColumn(SuperColumn superColumn, int gcBefore) Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/Descriptor.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/Descriptor.java?rev=1052542&r1=1052541&r2=1052542&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/Descriptor.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/Descriptor.java Fri Dec 24 17:58:22 2010 @@ -38,7 +38,7 @@ import org.apache.cassandra.utils.Pair; public class Descriptor { public static final String LEGACY_VERSION = "a"; - public static final String CURRENT_VERSION = "e"; + public static final String CURRENT_VERSION = "f"; public final File directory; public final String version; @@ -52,6 +52,7 @@ public class Descriptor public final boolean hasIntRowSize; public final boolean hasEncodedKeys; public final boolean isLatestVersion; + public final boolean usesOldBloomFilter; /** * A descriptor that assumes CURRENT_VERSION. @@ -76,6 +77,7 @@ public class Descriptor hasIntRowSize = version.compareTo("d") < 0; hasEncodedKeys = version.compareTo("e") < 0; isLatestVersion = version.compareTo(CURRENT_VERSION) == 0; + usesOldBloomFilter = version.compareTo("f") < 0; } public String filenameFor(Component component) Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/IndexHelper.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/IndexHelper.java?rev=1052542&r1=1052541&r2=1052542&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/IndexHelper.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/IndexHelper.java Fri Dec 24 17:58:22 2010 @@ -28,7 +28,9 @@ import java.util.List; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.io.util.FileDataInput; import org.apache.cassandra.io.util.FileMark; +import org.apache.cassandra.utils.LegacyBloomFilter; import org.apache.cassandra.utils.BloomFilter; +import org.apache.cassandra.utils.Filter; import org.apache.cassandra.utils.FBUtilities; /** @@ -92,14 +94,16 @@ public class IndexHelper * @return bloom filter summarizing the column information * @throws java.io.IOException */ - public static BloomFilter defreezeBloomFilter(DataInput file) throws IOException + public static Filter defreezeBloomFilter(DataInput file, boolean useOldBF) throws IOException { int size = file.readInt(); byte[] bytes = new byte[size]; file.readFully(bytes); - + ByteArrayInputStream bufIn = new ByteArrayInputStream(bytes); - return BloomFilter.serializer().deserialize(new DataInputStream(bufIn)); + return useOldBF + ? LegacyBloomFilter.serializer().deserialize(new DataInputStream(bufIn)) + : BloomFilter.serializer().deserialize(new DataInputStream(bufIn)); } /** Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java?rev=1052542&r1=1052541&r2=1052542&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java Fri Dec 24 17:58:22 2010 @@ -112,7 +112,7 @@ public class SSTableReader extends SSTab private SegmentedFile dfile; private IndexSummary indexSummary; - private BloomFilter bf; + private Filter bf; private InstrumentedCache, Long> keyCache; @@ -203,7 +203,7 @@ public class SSTableReader extends SSTab /** * Open a RowIndexedReader which already has its state initialized (by SSTableWriter). */ - static SSTableReader internalOpen(Descriptor desc, Set components, CFMetaData metadata, IPartitioner partitioner, SegmentedFile ifile, SegmentedFile dfile, IndexSummary isummary, BloomFilter bf, long maxDataAge, EstimatedHistogram rowsize, + static SSTableReader internalOpen(Descriptor desc, Set components, CFMetaData metadata, IPartitioner partitioner, SegmentedFile ifile, SegmentedFile dfile, IndexSummary isummary, Filter bf, long maxDataAge, EstimatedHistogram rowsize, EstimatedHistogram columncount) throws IOException { assert desc != null && partitioner != null && ifile != null && dfile != null && isummary != null && bf != null; @@ -217,7 +217,7 @@ public class SSTableReader extends SSTab SegmentedFile ifile, SegmentedFile dfile, IndexSummary indexSummary, - BloomFilter bloomFilter, + Filter bloomFilter, long maxDataAge, EstimatedHistogram rowSizes, EstimatedHistogram columnCounts) @@ -248,7 +248,14 @@ public class SSTableReader extends SSTab try { stream = new DataInputStream(new BufferedInputStream(new FileInputStream(descriptor.filenameFor(Component.FILTER)))); - bf = BloomFilter.serializer().deserialize(stream); + if (descriptor.usesOldBloomFilter) + { + bf = LegacyBloomFilter.serializer().deserialize(stream); + } + else + { + bf = BloomFilter.serializer().deserialize(stream); + } } finally { @@ -280,7 +287,7 @@ public class SSTableReader extends SSTab indexSummary = new IndexSummary(estimatedKeys); if (recreatebloom) // estimate key count based on index length - bf = BloomFilter.getFilter(estimatedKeys, 15); + bf = LegacyBloomFilter.getFilter(estimatedKeys, 15); while (true) { long indexPosition = input.getFilePointer(); @@ -344,10 +351,10 @@ public class SSTableReader extends SSTab */ public void forceFilterFailures() { - bf = BloomFilter.alwaysMatchingBloomFilter(); + bf = LegacyBloomFilter.alwaysMatchingBloomFilter(); } - public BloomFilter getBloomFilter() + public Filter getBloomFilter() { return bf; } Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/BloomCalculations.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/BloomCalculations.java?rev=1052542&r1=1052541&r2=1052542&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/BloomCalculations.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/BloomCalculations.java Fri Dec 24 17:58:22 2010 @@ -32,6 +32,8 @@ class BloomCalculations { private static final int minBuckets = 2; private static final int minK = 1; + private static final int EXCESS = 20; + /** * In the following table, the row 'i' shows false positive rates if i buckets * per element are used. Column 'j' shows false positive rates if j hash @@ -158,4 +160,20 @@ class BloomCalculations { return new BloomSpecification(K, bucketsPerElement); } + + /** + * Calculates the maximum number of buckets per element that this implementation + * can support. Crucially, it will lower the bucket count if necessary to meet + * BitSet's size restrictions. + */ + public static int maxBucketsPerElement(long numElements) + { + numElements = Math.max(1, numElements); + double v = (Long.MAX_VALUE - EXCESS) / (double)numElements; + if (v < 1.0) + { + throw new UnsupportedOperationException("Cannot compute probabilities for " + numElements + " elements."); + } + return Math.min(BloomCalculations.probs.length - 1, (int)v); + } } Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/BloomFilter.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/BloomFilter.java?rev=1052542&r1=1052541&r2=1052542&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/BloomFilter.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/BloomFilter.java Fri Dec 24 17:58:22 2010 @@ -18,175 +18,133 @@ package org.apache.cassandra.utils; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; import java.nio.ByteBuffer; -import java.util.BitSet; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.cassandra.io.ICompactSerializer; +import org.apache.cassandra.utils.obs.OpenBitSet; public class BloomFilter extends Filter { - private static final Logger logger = LoggerFactory.getLogger(BloomFilter.class); - static ICompactSerializer serializer_ = new BloomFilterSerializer(); + private static final Logger logger = LoggerFactory.getLogger(BloomFilter.class); private static final int EXCESS = 20; + static ICompactSerializer serializer_ = new BloomFilterSerializer(); - public static ICompactSerializer serializer() - { - return serializer_; - } - - private BitSet filter_; + public OpenBitSet bitset; - BloomFilter(int hashes, BitSet filter) + BloomFilter(int hashes, OpenBitSet bs) { hashCount = hashes; - filter_ = filter; + bitset = bs; } - private static BitSet bucketsFor(long numElements, int bucketsPer) + public static ICompactSerializer serializer() { - long numBits = numElements * bucketsPer + EXCESS; - return new BitSet((int)Math.min(Integer.MAX_VALUE, numBits)); + return serializer_; } - /** - * Calculates the maximum number of buckets per element that this implementation - * can support. Crucially, it will lower the bucket count if necessary to meet - * BitSet's size restrictions. - */ - private static int maxBucketsPerElement(long numElements) - { - numElements = Math.max(1, numElements); - double v = (Integer.MAX_VALUE - EXCESS) / (double)numElements; - if (v < 1.0) + long emptyBuckets() + { + long n = 0; + for (long i = 0; i < buckets(); i++) { - throw new UnsupportedOperationException("Cannot compute probabilities for " + numElements + " elements."); + if (!bitset.get(i)) + { + n++; + } } - return Math.min(BloomCalculations.probs.length - 1, (int)v); + return n; + } + + private static OpenBitSet bucketsFor(long numElements, int bucketsPer) + { + long numBits = numElements * bucketsPer + EXCESS; //TODO overflow? + return new OpenBitSet((long)Math.min(Long.MAX_VALUE, numBits)); } /** - * @return A BloomFilter with the lowest practical false positive probability - * for the given number of elements. - */ + * @return A BloomFilter with the lowest practical false positive probability + * for the given number of elements. + */ public static BloomFilter getFilter(long numElements, int targetBucketsPerElem) { - int maxBucketsPerElement = Math.max(1, maxBucketsPerElement(numElements)); + int maxBucketsPerElement = Math.max(1, BloomCalculations.maxBucketsPerElement(numElements)); int bucketsPerElement = Math.min(targetBucketsPerElem, maxBucketsPerElement); if (bucketsPerElement < targetBucketsPerElem) { logger.warn(String.format("Cannot provide an optimal BloomFilter for %d elements (%d/%d buckets per element).", - numElements, bucketsPerElement, targetBucketsPerElem)); + numElements, bucketsPerElement, targetBucketsPerElem)); } BloomCalculations.BloomSpecification spec = BloomCalculations.computeBloomSpec(bucketsPerElement); return new BloomFilter(spec.K, bucketsFor(numElements, spec.bucketsPerElement)); } /** - * @return The smallest BloomFilter that can provide the given false positive - * probability rate for the given number of elements. - * - * Asserts that the given probability can be satisfied using this filter. - */ + * @return The smallest BloomFilter that can provide the given false positive + * probability rate for the given number of elements. + * + * Asserts that the given probability can be satisfied using this filter. + */ public static BloomFilter getFilter(long numElements, double maxFalsePosProbability) { assert maxFalsePosProbability <= 1.0 : "Invalid probability"; - int bucketsPerElement = maxBucketsPerElement(numElements); + int bucketsPerElement = BloomCalculations.maxBucketsPerElement(numElements); BloomCalculations.BloomSpecification spec = BloomCalculations.computeBloomSpec(bucketsPerElement, maxFalsePosProbability); return new BloomFilter(spec.K, bucketsFor(numElements, spec.bucketsPerElement)); } - public void clear() - { - filter_.clear(); - } - - int buckets() + private long buckets() { - return filter_.size(); + return bitset.size(); } - BitSet filter() + private long[] getHashBuckets(ByteBuffer key) { - return filter_; + return BloomFilter.getHashBuckets(key, hashCount, buckets()); } - public boolean isPresent(ByteBuffer key) + // Murmur is faster than an SHA-based approach and provides as-good collision + // resistance. The combinatorial generation approach described in + // http://www.eecs.harvard.edu/~kirsch/pubs/bbbf/esa06.pdf + // does prove to work in actual tests, and is obviously faster + // than performing further iterations of murmur. + static long[] getHashBuckets(ByteBuffer b, int hashCount, long max) { - for (int bucketIndex : getHashBuckets(key)) + long[] result = new long[hashCount]; + long hash1 = MurmurHash.hash64(b.array(), b.position()+b.arrayOffset(), b.remaining(), 0L); + long hash2 = MurmurHash.hash64(b.array(), b.position()+b.arrayOffset(), b.remaining(), hash1); + for (int i = 0; i < hashCount; ++i) { - if (!filter_.get(bucketIndex)) - { - return false; - } + result[i] = Math.abs((hash1 + (long)i * hash2) % max); } - return true; + return result; } - /* - @param key -- value whose hash is used to fill - the filter_. - This is a general purpose API. - */ public void add(ByteBuffer key) { - for (int bucketIndex : getHashBuckets(key)) - { - filter_.set(bucketIndex); - } - } - - public String toString() - { - return filter_.toString(); - } - - ICompactSerializer tserializer() - { - return serializer_; - } - - int emptyBuckets() - { - int n = 0; - for (int i = 0; i < buckets(); i++) + for (long bucketIndex : getHashBuckets(key)) { - if (!filter_.get(i)) - { - n++; - } + bitset.set(bucketIndex); } - return n; - } - - /** @return a BloomFilter that always returns a positive match, for testing */ - public static BloomFilter alwaysMatchingBloomFilter() - { - BitSet set = new BitSet(64); - set.set(0, 64); - return new BloomFilter(1, set); } -} -class BloomFilterSerializer implements ICompactSerializer -{ - public void serialize(BloomFilter bf, DataOutputStream dos) - throws IOException + public boolean isPresent(ByteBuffer key) { - dos.writeInt(bf.getHashCount()); - BitSetSerializer.serialize(bf.filter(), dos); + for (long bucketIndex : getHashBuckets(key)) + { + if (!bitset.get(bucketIndex)) + { + return false; + } + } + return true; } - public BloomFilter deserialize(DataInputStream dis) throws IOException + public void clear() { - int hashes = dis.readInt(); - BitSet bs = BitSetSerializer.deserialize(dis); - return new BloomFilter(hashes, bs); + bitset.clear(0, bitset.size()); } } Added: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java?rev=1052542&view=auto ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java (added) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java Fri Dec 24 17:58:22 2010 @@ -0,0 +1,36 @@ +package org.apache.cassandra.utils; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import org.apache.cassandra.utils.obs.OpenBitSet; +import org.apache.cassandra.io.ICompactSerializer; + +class BloomFilterSerializer implements ICompactSerializer +{ + public void serialize(BloomFilter bf, DataOutputStream dos) throws IOException + { + long[] bits = bf.bitset.getBits(); + int bitLength = bits.length; + + dos.writeInt(bf.getHashCount()); + dos.writeInt(bitLength); + + for (int i = 0; i < bitLength; i++) + dos.writeLong(bits[i]); + dos.flush(); + } + + public BloomFilter deserialize(DataInputStream dis) throws IOException + { + int hashes = dis.readInt(); + int bitLength = dis.readInt(); + long[] bits = new long[bitLength]; + for (int i = 0; i < bitLength; i++) + bits[i] = dis.readLong(); + OpenBitSet bs = new OpenBitSet(bits, bitLength); + return new BloomFilter(hashes, bs); + } +} + + Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/Filter.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/Filter.java?rev=1052542&r1=1052541&r2=1052542&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/Filter.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/Filter.java Fri Dec 24 17:58:22 2010 @@ -18,64 +18,18 @@ */ package org.apache.cassandra.utils; -import java.lang.reflect.Method; import java.nio.ByteBuffer; -import org.apache.cassandra.io.ICompactSerializer; - public abstract class Filter { int hashCount; - private static MurmurHash hasher = new MurmurHash(); - int getHashCount() { return hashCount; } - public int[] getHashBuckets(ByteBuffer key) - { - return Filter.getHashBuckets(key, hashCount, buckets()); - } - - abstract int buckets(); - public abstract void add(ByteBuffer key); public abstract boolean isPresent(ByteBuffer key); - - // for testing - abstract int emptyBuckets(); - - ICompactSerializer getSerializer() - { - Method method = null; - try - { - method = getClass().getMethod("serializer"); - return (ICompactSerializer) method.invoke(null); - } - catch (Exception e) - { - throw new RuntimeException(e); - } - } - - // Murmur is faster than an SHA-based approach and provides as-good collision - // resistance. The combinatorial generation approach described in - // http://www.eecs.harvard.edu/~kirsch/pubs/bbbf/esa06.pdf - // does prove to work in actual tests, and is obviously faster - // than performing further iterations of murmur. - static int[] getHashBuckets(ByteBuffer b, int hashCount, int max) - { - int[] result = new int[hashCount]; - int hash1 = hasher.hash(b.array(), b.position()+b.arrayOffset(), b.remaining(), 0); - int hash2 = hasher.hash(b.array(), b.position()+b.arrayOffset(), b.remaining(), hash1); - for (int i = 0; i < hashCount; i++) - { - result[i] = Math.abs((hash1 + i * hash2) % max); - } - return result; - } } Added: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/LegacyBloomFilter.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/LegacyBloomFilter.java?rev=1052542&view=auto ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/LegacyBloomFilter.java (added) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/LegacyBloomFilter.java Fri Dec 24 17:58:22 2010 @@ -0,0 +1,176 @@ +/** + * 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.cassandra.utils; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.BitSet; + +import org.apache.cassandra.io.ICompactSerializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class LegacyBloomFilter extends Filter +{ + private static final int EXCESS = 20; + private static final Logger logger = LoggerFactory.getLogger(LegacyBloomFilter.class); + static ICompactSerializer serializer_ = new LegacyBloomFilterSerializer(); + + public static ICompactSerializer serializer() + { + return serializer_; + } + + private BitSet filter_; + + LegacyBloomFilter(int hashes, BitSet filter) + { + hashCount = hashes; + filter_ = filter; + } + + private static BitSet bucketsFor(long numElements, int bucketsPer) + { + long numBits = numElements * bucketsPer + EXCESS; + return new BitSet((int)Math.min(Integer.MAX_VALUE, numBits)); + } + + /** + * @return A LegacyBloomFilter with the lowest practical false positive probability + * for the given number of elements. + */ + public static LegacyBloomFilter getFilter(long numElements, int targetBucketsPerElem) + { + int maxBucketsPerElement = Math.max(1, BloomCalculations.maxBucketsPerElement(numElements)); + int bucketsPerElement = Math.min(targetBucketsPerElem, maxBucketsPerElement); + if (bucketsPerElement < targetBucketsPerElem) + { + logger.warn(String.format("Cannot provide an optimal LegacyBloomFilter for %d elements (%d/%d buckets per element).", + numElements, bucketsPerElement, targetBucketsPerElem)); + } + BloomCalculations.BloomSpecification spec = BloomCalculations.computeBloomSpec(bucketsPerElement); + return new LegacyBloomFilter(spec.K, bucketsFor(numElements, spec.bucketsPerElement)); + } + + /** + * @return The smallest LegacyBloomFilter that can provide the given false positive + * probability rate for the given number of elements. + * + * Asserts that the given probability can be satisfied using this filter. + */ + public static LegacyBloomFilter getFilter(long numElements, double maxFalsePosProbability) + { + assert maxFalsePosProbability <= 1.0 : "Invalid probability"; + int bucketsPerElement = BloomCalculations.maxBucketsPerElement(numElements); + BloomCalculations.BloomSpecification spec = BloomCalculations.computeBloomSpec(bucketsPerElement, maxFalsePosProbability); + return new LegacyBloomFilter(spec.K, bucketsFor(numElements, spec.bucketsPerElement)); + } + + public void clear() + { + filter_.clear(); + } + + int buckets() + { + return filter_.size(); + } + + public boolean isPresent(ByteBuffer key) + { + for (int bucketIndex : getHashBuckets(key)) + { + if (!filter_.get(bucketIndex)) + { + return false; + } + } + return true; + } + + /* + @param key -- value whose hash is used to fill + the filter_. + This is a general purpose API. + */ + public void add(ByteBuffer key) + { + for (int bucketIndex : getHashBuckets(key)) + { + filter_.set(bucketIndex); + } + } + + public String toString() + { + return filter_.toString(); + } + + ICompactSerializer tserializer() + { + return serializer_; + } + + int emptyBuckets() + { + int n = 0; + for (int i = 0; i < buckets(); i++) + { + if (!filter_.get(i)) + { + n++; + } + } + return n; + } + + /** @return a LegacyBloomFilter that always returns a positive match, for testing */ + public static LegacyBloomFilter alwaysMatchingBloomFilter() + { + BitSet set = new BitSet(64); + set.set(0, 64); + return new LegacyBloomFilter(1, set); + } + + public int[] getHashBuckets(ByteBuffer key) + { + return LegacyBloomFilter.getHashBuckets(key, hashCount, buckets()); + } + + // Murmur is faster than an SHA-based approach and provides as-good collision + // resistance. The combinatorial generation approach described in + // http://www.eecs.harvard.edu/~kirsch/pubs/bbbf/esa06.pdf + // does prove to work in actual tests, and is obviously faster + // than performing further iterations of murmur. + static int[] getHashBuckets(ByteBuffer b, int hashCount, int max) + { + int[] result = new int[hashCount]; + int hash1 = MurmurHash.hash32(b.array(), b.position()+b.arrayOffset(), b.remaining(), 0); + int hash2 = MurmurHash.hash32(b.array(), b.position()+b.arrayOffset(), b.remaining(), hash1); + for (int i = 0; i < hashCount; i++) + { + result[i] = Math.abs((hash1 + i * hash2) % max); + } + return result; + } + + public BitSet getBitSet(){ + return filter_; + } +} \ No newline at end of file Added: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/LegacyBloomFilterSerializer.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/LegacyBloomFilterSerializer.java?rev=1052542&view=auto ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/LegacyBloomFilterSerializer.java (added) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/LegacyBloomFilterSerializer.java Fri Dec 24 17:58:22 2010 @@ -0,0 +1,36 @@ +package org.apache.cassandra.utils; + +import java.util.BitSet; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.ObjectOutputStream; +import java.io.ObjectInputStream; +import org.apache.cassandra.utils.obs.OpenBitSet; +import org.apache.cassandra.io.ICompactSerializer; + +class LegacyBloomFilterSerializer implements ICompactSerializer +{ + public void serialize(LegacyBloomFilter bf, DataOutputStream dos) + throws IOException + { + dos.writeInt(bf.getHashCount()); + ObjectOutputStream oos = new ObjectOutputStream(dos); + oos.writeObject(bf.getBitSet()); + oos.flush(); + } + + public LegacyBloomFilter deserialize(DataInputStream dis) throws IOException + { + int hashes = dis.readInt(); + ObjectInputStream ois = new ObjectInputStream(dis); + try + { + BitSet bs = (BitSet) ois.readObject(); + return new LegacyBloomFilter(hashes, bs); + } catch (ClassNotFoundException e) + { + throw new RuntimeException(e); + } + } +} Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/MurmurHash.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/MurmurHash.java?rev=1052542&r1=1052541&r2=1052542&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/MurmurHash.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/MurmurHash.java Fri Dec 24 17:58:22 2010 @@ -20,58 +20,123 @@ package org.apache.cassandra.utils; /** * This is a very fast, non-cryptographic hash suitable for general hash-based - * lookup. See http://murmurhash.googlepages.com/ for more details. + * lookup. See http://murmurhash.googlepages.com/ for more details. * - *

The C version of MurmurHash 2.0 found at that site was ported - * to Java by Andrzej Bialecki (ab at getopt org).

+ *

+ * The C version of MurmurHash 2.0 found at that site was ported to Java by + * Andrzej Bialecki (ab at getopt org). + *

*/ -public class MurmurHash { - public int hash(byte[] data, int offset, int length, int seed) { - int m = 0x5bd1e995; - int r = 24; - - int h = seed ^ length; - - int len_4 = length >> 2; - - for (int i = 0; i < len_4; i++) { - int i_4 = i << 2; - int k = data[offset + i_4 + 3]; - k = k << 8; - k = k | (data[offset + i_4 + 2] & 0xff); - k = k << 8; - k = k | (data[offset + i_4 + 1] & 0xff); - k = k << 8; - k = k | (data[offset + i_4 + 0] & 0xff); - k *= m; - k ^= k >>> r; - k *= m; - h *= m; - h ^= k; - } - - // avoid calculating modulo - int len_m = len_4 << 2; - int left = length - len_m; - - if (left != 0) { - if (left >= 3) { - h ^= (int) data[offset + length - 3] << 16; - } - if (left >= 2) { - h ^= (int) data[offset + length - 2] << 8; - } - if (left >= 1) { - h ^= (int) data[offset + length - 1]; - } +public class MurmurHash +{ + public static int hash32(byte[] data, int offset, int length, int seed) + { + int m = 0x5bd1e995; + int r = 24; + + int h = seed ^ length; + + int len_4 = length >> 2; + + for (int i = 0; i < len_4; i++) + { + int i_4 = i << 2; + int k = data[offset + i_4 + 3]; + k = k << 8; + k = k | (data[offset + i_4 + 2] & 0xff); + k = k << 8; + k = k | (data[offset + i_4 + 1] & 0xff); + k = k << 8; + k = k | (data[offset + i_4 + 0] & 0xff); + k *= m; + k ^= k >>> r; + k *= m; + h *= m; + h ^= k; + } + + // avoid calculating modulo + int len_m = len_4 << 2; + int left = length - len_m; + + if (left != 0) + { + if (left >= 3) + { + h ^= (int) data[offset + length - 3] << 16; + } + if (left >= 2) + { + h ^= (int) data[offset + length - 2] << 8; + } + if (left >= 1) + { + h ^= (int) data[offset + length - 1]; + } + + h *= m; + } + + h ^= h >>> 13; + h *= m; + h ^= h >>> 15; - h *= m; + return h; } - h ^= h >>> 13; - h *= m; - h ^= h >>> 15; + public static long hash64(byte[] key, int offset, int length, long seed) + { + long m64 = 0xc6a4a7935bd1e995L; + int r64 = 47; + + long h64 = (seed & 0xffffffffL) ^ (m64 * length); + + int lenLongs = length >> 3; + + for (int i = 0; i < lenLongs; ++i) + { + int i_8 = i << 3; + + long k64 = ((long)key[offset+i_8+0]&0xff) + (((long)key[offset+i_8+1]&0xff)<<8) + + (((long)key[offset+i_8+2]&0xff)<<16) + (((long)key[offset+i_8+3]&0xff)<<24) + + (((long)key[offset+i_8+4]&0xff)<<32) + (((long)key[offset+i_8+5]&0xff)<<40) + + (((long)key[offset+i_8+6]&0xff)<<48) + (((long)key[offset+i_8+7]&0xff)<<56); + + k64 *= m64; + k64 ^= k64 >>> r64; + k64 *= m64; + + h64 ^= k64; + h64 *= m64; + } + + int rem = length & 0x7; + + switch (rem) + { + case 0: + break; + case 7: + h64 ^= (long) key[offset + length - rem + 6] << 48; + case 6: + h64 ^= (long) key[offset + length - rem + 5] << 40; + case 5: + h64 ^= (long) key[offset + length - rem + 4] << 32; + case 4: + h64 ^= (long) key[offset + length - rem + 3] << 24; + case 3: + h64 ^= (long) key[offset + length - rem + 2] << 16; + case 2: + h64 ^= (long) key[offset + length - rem + 1] << 8; + case 1: + h64 ^= (long) key[offset + length - rem]; + h64 *= m64; + } + + h64 ^= h64 >>> r64; + h64 *= m64; + h64 ^= h64 >>> r64; - return h; - } + return h64; + } } Added: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/obs/ArrayUtil.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/obs/ArrayUtil.java?rev=1052542&view=auto ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/obs/ArrayUtil.java (added) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/obs/ArrayUtil.java Fri Dec 24 17:58:22 2010 @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils.obs; + +import java.util.Collection; + +/** + * Methods for manipulating arrays. + * + * @lucene.internal + */ + +final class ArrayUtil { + public static long[] grow(long[] array, int minSize) { + if (array.length < minSize) { + long[] newArray = new long[Math.max(array.length << 1, minSize)]; + System.arraycopy(array, 0, newArray, 0, array.length); + return newArray; + } else + return array; + } + + public static long[] grow(long[] array) { + return grow(array, 1 + array.length); + } +} Added: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/obs/BitUtil.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/obs/BitUtil.java?rev=1052542&view=auto ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/obs/BitUtil.java (added) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/obs/BitUtil.java Fri Dec 24 17:58:22 2010 @@ -0,0 +1,815 @@ +/** + * 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.cassandra.utils.obs; + +/** A variety of high efficiency bit twiddling routines. + * @lucene.internal + */ +final class BitUtil { + + /** Returns the number of bits set in the long */ + public static int pop(long x) { + /* Hacker's Delight 32 bit pop function: + * http://www.hackersdelight.org/HDcode/newCode/pop_arrayHS.cc + * + int pop(unsigned x) { + x = x - ((x >> 1) & 0x55555555); + x = (x & 0x33333333) + ((x >> 2) & 0x33333333); + x = (x + (x >> 4)) & 0x0F0F0F0F; + x = x + (x >> 8); + x = x + (x >> 16); + return x & 0x0000003F; + } + ***/ + + // 64 bit java version of the C function from above + x = x - ((x >>> 1) & 0x5555555555555555L); + x = (x & 0x3333333333333333L) + ((x >>>2 ) & 0x3333333333333333L); + x = (x + (x >>> 4)) & 0x0F0F0F0F0F0F0F0FL; + x = x + (x >>> 8); + x = x + (x >>> 16); + x = x + (x >>> 32); + return ((int)x) & 0x7F; + } + + /*** Returns the number of set bits in an array of longs. */ + public static long pop_array(long A[], int wordOffset, int numWords) { + /* + * Robert Harley and David Seal's bit counting algorithm, as documented + * in the revisions of Hacker's Delight + * http://www.hackersdelight.org/revisions.pdf + * http://www.hackersdelight.org/HDcode/newCode/pop_arrayHS.cc + * + * This function was adapted to Java, and extended to use 64 bit words. + * if only we had access to wider registers like SSE from java... + * + * This function can be transformed to compute the popcount of other functions + * on bitsets via something like this: + * sed 's/A\[\([^]]*\)\]/\(A[\1] \& B[\1]\)/g' + * + */ + int n = wordOffset+numWords; + long tot=0, tot8=0; + long ones=0, twos=0, fours=0; + + int i; + for (i = wordOffset; i <= n - 8; i+=8) { + /*** C macro from Hacker's Delight + #define CSA(h,l, a,b,c) \ + {unsigned u = a ^ b; unsigned v = c; \ + h = (a & b) | (u & v); l = u ^ v;} + ***/ + + long twosA,twosB,foursA,foursB,eights; + + // CSA(twosA, ones, ones, A[i], A[i+1]) + { + long b=A[i], c=A[i+1]; + long u=ones ^ b; + twosA=(ones & b)|( u & c); + ones=u^c; + } + // CSA(twosB, ones, ones, A[i+2], A[i+3]) + { + long b=A[i+2], c=A[i+3]; + long u=ones^b; + twosB =(ones&b)|(u&c); + ones=u^c; + } + //CSA(foursA, twos, twos, twosA, twosB) + { + long u=twos^twosA; + foursA=(twos&twosA)|(u&twosB); + twos=u^twosB; + } + //CSA(twosA, ones, ones, A[i+4], A[i+5]) + { + long b=A[i+4], c=A[i+5]; + long u=ones^b; + twosA=(ones&b)|(u&c); + ones=u^c; + } + // CSA(twosB, ones, ones, A[i+6], A[i+7]) + { + long b=A[i+6], c=A[i+7]; + long u=ones^b; + twosB=(ones&b)|(u&c); + ones=u^c; + } + //CSA(foursB, twos, twos, twosA, twosB) + { + long u=twos^twosA; + foursB=(twos&twosA)|(u&twosB); + twos=u^twosB; + } + + //CSA(eights, fours, fours, foursA, foursB) + { + long u=fours^foursA; + eights=(fours&foursA)|(u&foursB); + fours=u^foursB; + } + tot8 += pop(eights); + } + + // handle trailing words in a binary-search manner... + // derived from the loop above by setting specific elements to 0. + // the original method in Hackers Delight used a simple for loop: + // for (i = i; i < n; i++) // Add in the last elements + // tot = tot + pop(A[i]); + + if (i<=n-4) { + long twosA, twosB, foursA, eights; + { + long b=A[i], c=A[i+1]; + long u=ones ^ b; + twosA=(ones & b)|( u & c); + ones=u^c; + } + { + long b=A[i+2], c=A[i+3]; + long u=ones^b; + twosB =(ones&b)|(u&c); + ones=u^c; + } + { + long u=twos^twosA; + foursA=(twos&twosA)|(u&twosB); + twos=u^twosB; + } + eights=fours&foursA; + fours=fours^foursA; + + tot8 += pop(eights); + i+=4; + } + + if (i<=n-2) { + long b=A[i], c=A[i+1]; + long u=ones ^ b; + long twosA=(ones & b)|( u & c); + ones=u^c; + + long foursA=twos&twosA; + twos=twos^twosA; + + long eights=fours&foursA; + fours=fours^foursA; + + tot8 += pop(eights); + i+=2; + } + + if (i>= 1 + return i + print ','.join([ str(ntz(i)) for i in range(256) ]) + ***/ + /** table of number of trailing zeros in a byte */ + public static final byte[] ntzTable = {8,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}; + + + /** Returns number of trailing zeros in a 64 bit long value. */ + public static int ntz(long val) { + // A full binary search to determine the low byte was slower than + // a linear search for nextSetBit(). This is most likely because + // the implementation of nextSetBit() shifts bits to the right, increasing + // the probability that the first non-zero byte is in the rhs. + // + // This implementation does a single binary search at the top level only + // so that all other bit shifting can be done on ints instead of longs to + // remain friendly to 32 bit architectures. In addition, the case of a + // non-zero first byte is checked for first because it is the most common + // in dense bit arrays. + + int lower = (int)val; + int lowByte = lower & 0xff; + if (lowByte != 0) return ntzTable[lowByte]; + + if (lower!=0) { + lowByte = (lower>>>8) & 0xff; + if (lowByte != 0) return ntzTable[lowByte] + 8; + lowByte = (lower>>>16) & 0xff; + if (lowByte != 0) return ntzTable[lowByte] + 16; + // no need to mask off low byte for the last byte in the 32 bit word + // no need to check for zero on the last byte either. + return ntzTable[lower>>>24] + 24; + } else { + // grab upper 32 bits + int upper=(int)(val>>32); + lowByte = upper & 0xff; + if (lowByte != 0) return ntzTable[lowByte] + 32; + lowByte = (upper>>>8) & 0xff; + if (lowByte != 0) return ntzTable[lowByte] + 40; + lowByte = (upper>>>16) & 0xff; + if (lowByte != 0) return ntzTable[lowByte] + 48; + // no need to mask off low byte for the last byte in the 32 bit word + // no need to check for zero on the last byte either. + return ntzTable[upper>>>24] + 56; + } + } + + /** Returns number of trailing zeros in a 32 bit int value. */ + public static int ntz(int val) { + // This implementation does a single binary search at the top level only. + // In addition, the case of a non-zero first byte is checked for first + // because it is the most common in dense bit arrays. + + int lowByte = val & 0xff; + if (lowByte != 0) return ntzTable[lowByte]; + lowByte = (val>>>8) & 0xff; + if (lowByte != 0) return ntzTable[lowByte] + 8; + lowByte = (val>>>16) & 0xff; + if (lowByte != 0) return ntzTable[lowByte] + 16; + // no need to mask off low byte for the last byte. + // no need to check for zero on the last byte either. + return ntzTable[val>>>24] + 24; + } + + /** returns 0 based index of first set bit + * (only works for x!=0) + *
This is an alternate implementation of ntz() + */ + public static int ntz2(long x) { + int n = 0; + int y = (int)x; + if (y==0) {n+=32; y = (int)(x>>>32); } // the only 64 bit shift necessary + if ((y & 0x0000FFFF) == 0) { n+=16; y>>>=16; } + if ((y & 0x000000FF) == 0) { n+=8; y>>>=8; } + return (ntzTable[ y & 0xff ]) + n; + } + + /** returns 0 based index of first set bit + *
This is an alternate implementation of ntz() + */ + public static int ntz3(long x) { + // another implementation taken from Hackers Delight, extended to 64 bits + // and converted to Java. + // Many 32 bit ntz algorithms are at http://www.hackersdelight.org/HDcode/ntz.cc + int n = 1; + + // do the first step as a long, all others as ints. + int y = (int)x; + if (y==0) {n+=32; y = (int)(x>>>32); } + if ((y & 0x0000FFFF) == 0) { n+=16; y>>>=16; } + if ((y & 0x000000FF) == 0) { n+=8; y>>>=8; } + if ((y & 0x0000000F) == 0) { n+=4; y>>>=4; } + if ((y & 0x00000003) == 0) { n+=2; y>>>=2; } + return n - (y & 1); + } + + + /** returns true if v is a power of two or zero*/ + public static boolean isPowerOfTwo(int v) { + return ((v & (v-1)) == 0); + } + + /** returns true if v is a power of two or zero*/ + public static boolean isPowerOfTwo(long v) { + return ((v & (v-1)) == 0); + } + + /** returns the next highest power of two, or the current value if it's already a power of two or zero*/ + public static int nextHighestPowerOfTwo(int v) { + v--; + v |= v >> 1; + v |= v >> 2; + v |= v >> 4; + v |= v >> 8; + v |= v >> 16; + v++; + return v; + } + + /** returns the next highest power of two, or the current value if it's already a power of two or zero*/ + public static long nextHighestPowerOfTwo(long v) { + v--; + v |= v >> 1; + v |= v >> 2; + v |= v >> 4; + v |= v >> 8; + v |= v >> 16; + v |= v >> 32; + v++; + return v; + } + +}