remove per-row bloom filter of column names patch by jbellis; reviewed by slebresne for CASSANDRA-5492
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/302267e8 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/302267e8 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/302267e8 Branch: refs/heads/cassandra-1.2 Commit: 302267e8a3761053f4acac43dc7dfaa4877384c6 Parents: 573ddbf Author: Jonathan Ellis <[email protected]> Authored: Mon Apr 22 08:40:54 2013 -0500 Committer: Jonathan Ellis <[email protected]> Committed: Tue Apr 23 10:22:13 2013 -0500 ---------------------------------------------------------------------- CHANGES.txt | 1 + src/java/org/apache/cassandra/db/ColumnIndex.java | 31 ++--- .../org/apache/cassandra/db/RowIndexEntry.java | 88 ++++++-------- .../db/columniterator/IndexedSliceReader.java | 2 +- .../db/columniterator/SSTableNamesIterator.java | 46 ++------ .../db/columniterator/SimpleSliceReader.java | 2 +- .../db/compaction/LazilyCompactedRow.java | 2 +- .../cassandra/db/compaction/PrecompactedRow.java | 2 +- .../apache/cassandra/io/sstable/Descriptor.java | 7 +- .../apache/cassandra/io/sstable/IndexHelper.java | 64 +---------- .../io/sstable/SSTableIdentityIterator.java | 7 +- .../apache/cassandra/io/sstable/SSTableWriter.java | 5 +- test/unit/org/apache/cassandra/Util.java | 2 +- .../org/apache/cassandra/cache/ObjectSizeTest.java | 3 +- 14 files changed, 83 insertions(+), 179 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/302267e8/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index 06da9a8..3d43fe6 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 1.2.5 + * remove per-row column name bloom filters (CASSANDRA-5492) * Include fatal errors in trace events (CASSANDRA-5447) * Ensure that PerRowSecondaryIndex is notified of row-level deletes (CASSANDRA-5445) http://git-wip-us.apache.org/repos/asf/cassandra/blob/302267e8/src/java/org/apache/cassandra/db/ColumnIndex.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/ColumnIndex.java b/src/java/org/apache/cassandra/db/ColumnIndex.java index 6eb8833..23e3c00 100644 --- a/src/java/org/apache/cassandra/db/ColumnIndex.java +++ b/src/java/org/apache/cassandra/db/ColumnIndex.java @@ -22,30 +22,28 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.*; +import com.google.common.annotations.VisibleForTesting; + import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.io.sstable.IndexHelper; -import org.apache.cassandra.utils.AlwaysPresentFilter; -import org.apache.cassandra.utils.IFilter; -import org.apache.cassandra.utils.FilterFactory; public class ColumnIndex { public final List<IndexHelper.IndexInfo> columnsIndex; - public final IFilter bloomFilter; - - private static final ColumnIndex EMPTY = new ColumnIndex(Collections.<IndexHelper.IndexInfo>emptyList(), new AlwaysPresentFilter()); - private ColumnIndex(int estimatedColumnCount) - { - this(new ArrayList<IndexHelper.IndexInfo>(), FilterFactory.getFilter(estimatedColumnCount, 4, false)); - } + private static final ColumnIndex EMPTY = new ColumnIndex(Collections.<IndexHelper.IndexInfo>emptyList()); - private ColumnIndex(List<IndexHelper.IndexInfo> columnsIndex, IFilter bloomFilter) + private ColumnIndex(List<IndexHelper.IndexInfo> columnsIndex) { assert columnsIndex != null; this.columnsIndex = columnsIndex; - this.bloomFilter = bloomFilter; + } + + @VisibleForTesting + public static ColumnIndex nothing() + { + return EMPTY; } /** @@ -69,12 +67,11 @@ public class ColumnIndex public Builder(ColumnFamily cf, ByteBuffer key, - int estimatedColumnCount, DataOutput output, boolean fromStream) { this.indexOffset = rowHeaderSize(key, cf.deletionInfo()); - this.result = new ColumnIndex(estimatedColumnCount); + this.result = new ColumnIndex(new ArrayList<IndexHelper.IndexInfo>()); this.output = output; this.atomSerializer = cf.getOnDiskSerializer(); this.tombstoneTracker = fromStream ? null : new RangeTombstone.Tracker(cf.getComparator()); @@ -82,10 +79,9 @@ public class ColumnIndex public Builder(ColumnFamily cf, ByteBuffer key, - int estimatedColumnCount, DataOutput output) { - this(cf, key, estimatedColumnCount, output, false); + this(cf, key, output, false); } /** @@ -157,9 +153,6 @@ public class ColumnIndex { atomCount++; - if (column instanceof IColumn) - result.bloomFilter.add(column.name()); - if (firstColumn == null) { firstColumn = column; http://git-wip-us.apache.org/repos/asf/cassandra/blob/302267e8/src/java/org/apache/cassandra/db/RowIndexEntry.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/RowIndexEntry.java b/src/java/org/apache/cassandra/db/RowIndexEntry.java index fefa45d..c53e52f 100644 --- a/src/java/org/apache/cassandra/db/RowIndexEntry.java +++ b/src/java/org/apache/cassandra/db/RowIndexEntry.java @@ -28,7 +28,7 @@ import org.apache.cassandra.cache.IMeasurableMemory; import org.apache.cassandra.io.sstable.Descriptor; import org.apache.cassandra.io.sstable.IndexHelper; import org.apache.cassandra.io.util.FileUtils; -import org.apache.cassandra.utils.IFilter; +import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.FilterFactory; import org.apache.cassandra.utils.ObjectSizes; @@ -62,7 +62,7 @@ public class RowIndexEntry implements IMeasurableMemory // since if there are insufficient columns to be worth indexing we're going to seek to // the beginning of the row anyway, so we might as well read the tombstone there as well. if (index.columnsIndex.size() > 1) - return new IndexedEntry(position, deletionTime, index.columnsIndex, index.bloomFilter); + return new IndexedEntry(position, deletionTime, index.columnsIndex); else return new RowIndexEntry(position); } @@ -86,11 +86,6 @@ public class RowIndexEntry implements IMeasurableMemory return Collections.emptyList(); } - public IFilter bloomFilter() - { - throw new UnsupportedOperationException(); - } - public long memorySize() { long fields = TypeSizes.NATIVE.sizeof(position) + ObjectSizes.getReferenceSize(); @@ -99,44 +94,46 @@ public class RowIndexEntry implements IMeasurableMemory public static class Serializer { - public void serialize(RowIndexEntry rie, DataOutput dos) throws IOException + public void serialize(RowIndexEntry rie, DataOutput out) throws IOException { - dos.writeLong(rie.position); + out.writeLong(rie.position); + out.writeInt(rie.promotedSize()); + if (rie.isIndexed()) { - dos.writeInt(rie.promotedSize()); - DeletionTime.serializer.serialize(rie.deletionTime(), dos); - dos.writeInt(rie.columnsIndex().size()); + DeletionTime.serializer.serialize(rie.deletionTime(), out); + out.writeInt(rie.columnsIndex().size()); for (IndexHelper.IndexInfo info : rie.columnsIndex()) - info.serialize(dos); - FilterFactory.serialize(rie.bloomFilter(), dos); - } - else - { - dos.writeInt(0); + info.serialize(out); } } - public RowIndexEntry deserialize(DataInput dis, Descriptor.Version version) throws IOException + public RowIndexEntry deserialize(DataInput in, Descriptor.Version version) throws IOException { - long position = dis.readLong(); - if (version.hasPromotedIndexes) + long position = in.readLong(); + + if (!version.hasPromotedIndexes) + return new RowIndexEntry(position); + + int size = in.readInt(); + if (size > 0) { - int size = dis.readInt(); - if (size > 0) - { - DeletionTime deletionTime = DeletionTime.serializer.deserialize(dis); - int entries = dis.readInt(); - List<IndexHelper.IndexInfo> columnsIndex = new ArrayList<IndexHelper.IndexInfo>(entries); - for (int i = 0; i < entries; i++) - columnsIndex.add(IndexHelper.IndexInfo.deserialize(dis)); - IFilter bf = FilterFactory.deserialize(dis, version.filterType, false); - return new IndexedEntry(position, deletionTime, columnsIndex, bf); - } - else + DeletionTime deletionTime = DeletionTime.serializer.deserialize(in); + + int entries = in.readInt(); + List<IndexHelper.IndexInfo> columnsIndex = new ArrayList<IndexHelper.IndexInfo>(entries); + for (int i = 0; i < entries; i++) + columnsIndex.add(IndexHelper.IndexInfo.deserialize(in)); + + if (version.hasRowLevelBF) { - return new RowIndexEntry(position); + // we only ever used murmur3 BF in the promoted index + in.readInt(); // hash count + int words = in.readInt(); // number of Longs in the OpenBitSet + FileUtils.skipBytesFully(in, words * 8); } + + return new IndexedEntry(position, deletionTime, columnsIndex); } else { @@ -144,20 +141,20 @@ public class RowIndexEntry implements IMeasurableMemory } } - public void skip(DataInput dis, Descriptor.Version version) throws IOException + public void skip(DataInput in, Descriptor.Version version) throws IOException { - dis.readLong(); + in.readLong(); if (version.hasPromotedIndexes) - skipPromotedIndex(dis); + skipPromotedIndex(in); } - public void skipPromotedIndex(DataInput dis) throws IOException + public void skipPromotedIndex(DataInput in) throws IOException { - int size = dis.readInt(); + int size = in.readInt(); if (size <= 0) return; - FileUtils.skipBytesFully(dis, size); + FileUtils.skipBytesFully(in, size); } } @@ -168,16 +165,14 @@ public class RowIndexEntry implements IMeasurableMemory { private final DeletionTime deletionTime; private final List<IndexHelper.IndexInfo> columnsIndex; - private final IFilter bloomFilter; - private IndexedEntry(long position, DeletionTime deletionTime, List<IndexHelper.IndexInfo> columnsIndex, IFilter bloomFilter) + private IndexedEntry(long position, DeletionTime deletionTime, List<IndexHelper.IndexInfo> columnsIndex) { super(position); assert deletionTime != null; assert columnsIndex != null && columnsIndex.size() > 1; this.deletionTime = deletionTime; this.columnsIndex = columnsIndex; - this.bloomFilter = bloomFilter; } @Override @@ -193,12 +188,6 @@ public class RowIndexEntry implements IMeasurableMemory } @Override - public IFilter bloomFilter() - { - return bloomFilter; - } - - @Override public int promotedSize() { TypeSizes typeSizes = TypeSizes.NATIVE; @@ -207,7 +196,6 @@ public class RowIndexEntry implements IMeasurableMemory for (IndexHelper.IndexInfo info : columnsIndex) size += info.serializedSize(typeSizes); - size += FilterFactory.serializedSize(bloomFilter); assert size <= Integer.MAX_VALUE; return (int)size; } http://git-wip-us.apache.org/repos/asf/cassandra/blob/302267e8/src/java/org/apache/cassandra/db/columniterator/IndexedSliceReader.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/columniterator/IndexedSliceReader.java b/src/java/org/apache/cassandra/db/columniterator/IndexedSliceReader.java index 1c3ff3e..0493819 100644 --- a/src/java/org/apache/cassandra/db/columniterator/IndexedSliceReader.java +++ b/src/java/org/apache/cassandra/db/columniterator/IndexedSliceReader.java @@ -96,7 +96,7 @@ class IndexedSliceReader extends AbstractIterator<OnDiskAtom> implements OnDiskA else { setToRowStart(sstable, indexEntry, input); - IndexHelper.skipSSTableBloomFilter(file, version); + IndexHelper.skipBloomFilter(file); this.indexes = IndexHelper.deserializeIndex(file); this.emptyColumnFamily = ColumnFamily.create(sstable.metadata); emptyColumnFamily.delete(DeletionInfo.serializer().deserializeFromSSTable(file, version)); http://git-wip-us.apache.org/repos/asf/cassandra/blob/302267e8/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java index 5495458..3750665 100644 --- a/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java +++ b/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java @@ -19,16 +19,13 @@ package org.apache.cassandra.db.columniterator; import java.io.IOException; import java.nio.ByteBuffer; -import java.util.*; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.SortedSet; import org.apache.cassandra.config.CFMetaData; -import org.apache.cassandra.db.ColumnFamily; -import org.apache.cassandra.db.ColumnFamilySerializer; -import org.apache.cassandra.db.DecoratedKey; -import org.apache.cassandra.db.DeletionInfo; -import org.apache.cassandra.db.IColumn; -import org.apache.cassandra.db.RowIndexEntry; -import org.apache.cassandra.db.OnDiskAtom; +import org.apache.cassandra.db.*; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.io.sstable.CorruptSSTableException; import org.apache.cassandra.io.sstable.IndexHelper; @@ -37,7 +34,6 @@ 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.ByteBufferUtil; -import org.apache.cassandra.utils.IFilter; public class SSTableNamesIterator extends SimpleAbstractColumnIterator implements ISSTableColumnIterator { @@ -107,7 +103,6 @@ public class SSTableNamesIterator extends SimpleAbstractColumnIterator implement private void read(SSTableReader sstable, FileDataInput file, RowIndexEntry indexEntry) throws IOException { - IFilter bf; List<IndexHelper.IndexInfo> indexList; // If the entry is not indexed or the index is not promoted, read from the row start @@ -127,20 +122,17 @@ public class SSTableNamesIterator extends SimpleAbstractColumnIterator implement if (sstable.descriptor.version.hasPromotedIndexes) { - bf = indexEntry.isIndexed() ? indexEntry.bloomFilter() : null; indexList = indexEntry.columnsIndex(); } else { assert file != null; - bf = IndexHelper.defreezeBloomFilter(file, sstable.descriptor.version.filterType); + IndexHelper.skipBloomFilter(file); indexList = IndexHelper.deserializeIndex(file); } if (!indexEntry.isIndexed()) { - // 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 ColumnFamilySerializer serializer = ColumnFamily.serializer; try { @@ -159,20 +151,9 @@ public class SSTableNamesIterator extends SimpleAbstractColumnIterator implement } List<OnDiskAtom> result = new ArrayList<OnDiskAtom>(); - List<ByteBuffer> filteredColumnNames = new ArrayList<ByteBuffer>(columns.size()); - for (ByteBuffer name : columns) - { - if (bf == null || bf.isPresent(name)) - { - filteredColumnNames.add(name); - } - } - if (filteredColumnNames.isEmpty()) - return; - if (indexList.isEmpty()) { - readSimpleColumns(file, columns, filteredColumnNames, result); + readSimpleColumns(file, columns, result); } else { @@ -187,19 +168,19 @@ public class SSTableNamesIterator extends SimpleAbstractColumnIterator implement file.readInt(); // column count basePosition = file.getFilePointer(); } - readIndexedColumns(sstable.metadata, file, columns, filteredColumnNames, indexList, basePosition, result); + readIndexedColumns(sstable.metadata, file, columns, indexList, basePosition, result); } // create an iterator view of the columns we read iter = result.iterator(); } - private void readSimpleColumns(FileDataInput file, SortedSet<ByteBuffer> columnNames, List<ByteBuffer> filteredColumnNames, List<OnDiskAtom> result) throws IOException + private void readSimpleColumns(FileDataInput file, SortedSet<ByteBuffer> columnNames, List<OnDiskAtom> result) throws IOException { OnDiskAtom.Serializer atomSerializer = cf.getOnDiskSerializer(); - int columns = file.readInt(); + int count = file.readInt(); int n = 0; - for (int i = 0; i < columns; i++) + for (int i = 0; i < count; i++) { OnDiskAtom column = atomSerializer.deserializeFromSSTable(file, sstable.descriptor.version); if (column instanceof IColumn) @@ -207,7 +188,7 @@ public class SSTableNamesIterator extends SimpleAbstractColumnIterator implement if (columnNames.contains(column.name())) { result.add(column); - if (n++ > filteredColumnNames.size()) + if (++n > columns.size()) break; } } @@ -221,7 +202,6 @@ public class SSTableNamesIterator extends SimpleAbstractColumnIterator implement private void readIndexedColumns(CFMetaData metadata, FileDataInput file, SortedSet<ByteBuffer> columnNames, - List<ByteBuffer> filteredColumnNames, List<IndexHelper.IndexInfo> indexList, long basePosition, List<OnDiskAtom> result) @@ -231,7 +211,7 @@ public class SSTableNamesIterator extends SimpleAbstractColumnIterator implement AbstractType<?> comparator = metadata.comparator; List<IndexHelper.IndexInfo> ranges = new ArrayList<IndexHelper.IndexInfo>(); int lastIndexIdx = -1; - for (ByteBuffer name : filteredColumnNames) + for (ByteBuffer name : columns) { int index = IndexHelper.indexFor(name, indexList, comparator, false, lastIndexIdx); if (index < 0 || index == indexList.size()) http://git-wip-us.apache.org/repos/asf/cassandra/blob/302267e8/src/java/org/apache/cassandra/db/columniterator/SimpleSliceReader.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/columniterator/SimpleSliceReader.java b/src/java/org/apache/cassandra/db/columniterator/SimpleSliceReader.java index b30d360..83cdeaa 100644 --- a/src/java/org/apache/cassandra/db/columniterator/SimpleSliceReader.java +++ b/src/java/org/apache/cassandra/db/columniterator/SimpleSliceReader.java @@ -75,7 +75,7 @@ class SimpleSliceReader extends AbstractIterator<OnDiskAtom> implements OnDiskAt Descriptor.Version version = sstable.descriptor.version; if (!version.hasPromotedIndexes) { - IndexHelper.skipSSTableBloomFilter(file, version); + IndexHelper.skipBloomFilter(file); IndexHelper.skipIndex(file); } http://git-wip-us.apache.org/repos/asf/cassandra/blob/302267e8/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java b/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java index 22f5413..48b5b7d 100644 --- a/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java +++ b/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java @@ -107,7 +107,7 @@ public class LazilyCompactedRow extends AbstractCompactedRow implements Iterable private void indexAndWrite(DataOutput out) throws IOException { - this.indexBuilder = new ColumnIndex.Builder(emptyColumnFamily, key.key, getEstimatedColumnCount(), out); + this.indexBuilder = new ColumnIndex.Builder(emptyColumnFamily, key.key, out); this.columnsIndex = indexBuilder.build(this); } http://git-wip-us.apache.org/repos/asf/cassandra/blob/302267e8/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java b/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java index b5dfed0..34ddb88 100644 --- a/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java +++ b/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java @@ -163,7 +163,7 @@ public class PrecompactedRow extends AbstractCompactedRow { assert compactedCf != null; DataOutputBuffer buffer = new DataOutputBuffer(); - ColumnIndex.Builder builder = new ColumnIndex.Builder(compactedCf, key.key, compactedCf.getColumnCount(), buffer); + ColumnIndex.Builder builder = new ColumnIndex.Builder(compactedCf, key.key, buffer); columnIndex = builder.build(compactedCf); TypeSizes typeSizes = TypeSizes.NATIVE; http://git-wip-us.apache.org/repos/asf/cassandra/blob/302267e8/src/java/org/apache/cassandra/io/sstable/Descriptor.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/io/sstable/Descriptor.java b/src/java/org/apache/cassandra/io/sstable/Descriptor.java index f21a0d5..748bd01 100644 --- a/src/java/org/apache/cassandra/io/sstable/Descriptor.java +++ b/src/java/org/apache/cassandra/io/sstable/Descriptor.java @@ -47,7 +47,7 @@ public class Descriptor public static class Version { // This needs to be at the begining for initialization sake - public static final String current_version = "ib"; + public static final String current_version = "ic"; public static final Version LEGACY = new Version("a"); // "pre-history" // b (0.7.0): added version to sstable filenames @@ -66,6 +66,7 @@ public class Descriptor // records estimated histogram of deletion times in tombstones // bloom filter (keys and columns) upgraded to Murmur3 // ib (1.2.1): tracks min client timestamp in metadata component + // ic (1.2.5): omits per-row bloom filter of column names public static final Version CURRENT = new Version(current_version); @@ -85,7 +86,7 @@ public class Descriptor public final boolean hasPromotedIndexes; public final FilterFactory.Type filterType; public final boolean hasAncestors; - public final boolean hasBloomFilterSizeInHeader; + public final boolean hasRowLevelBF; public Version(String version) { @@ -109,7 +110,7 @@ public class Descriptor filterType = FilterFactory.Type.MURMUR2; else filterType = FilterFactory.Type.MURMUR3; - hasBloomFilterSizeInHeader = version.compareTo("ia") < 0; + hasRowLevelBF = version.compareTo("ic") < 0; } /** http://git-wip-us.apache.org/repos/asf/cassandra/blob/302267e8/src/java/org/apache/cassandra/io/sstable/IndexHelper.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/io/sstable/IndexHelper.java b/src/java/org/apache/cassandra/io/sstable/IndexHelper.java index 444ec0b..b93c019 100644 --- a/src/java/org/apache/cassandra/io/sstable/IndexHelper.java +++ b/src/java/org/apache/cassandra/io/sstable/IndexHelper.java @@ -36,42 +36,10 @@ import org.apache.cassandra.utils.*; */ public class IndexHelper { - public static void skipSSTableBloomFilter(DataInput in, Descriptor.Version version) throws IOException + public static void skipBloomFilter(DataInput in) throws IOException { - if (version.hasBloomFilterSizeInHeader) - { - int size = in.readInt(); - FileUtils.skipBytesFully(in, size); - } - else - { - skipBloomFilter(in, version.filterType); - } - } - - /** - * Skip the bloom filter - * @param in the data input from which the bloom filter should be skipped - * @throws IOException - */ - public static void skipBloomFilter(DataInput in, FilterFactory.Type type) throws IOException - { - /* size of the bloom filter */ int size = in.readInt(); - switch (type) - { - case SHA: - // can skip since bitset = 1 byte - FileUtils.skipBytesFully(in, size); - break; - case MURMUR2: - case MURMUR3: - long bitLength = in.readInt() * 8; - FileUtils.skipBytesFully(in, bitLength); - break; - default: - throw new IllegalStateException("Unknown filterfactory type " + type.toString()); - } + FileUtils.skipBytesFully(in, size); } /** @@ -120,34 +88,6 @@ public class IndexHelper return indexList; } - public static IFilter defreezeBloomFilter(FileDataInput file, FilterFactory.Type type) throws IOException - { - return defreezeBloomFilter(file, Integer.MAX_VALUE, type); - } - - /** - * De-freeze the bloom filter. - * - * @param file - source file - * @param maxSize - sanity check: if filter claimes to be larger than this it is bogus - * @param type - Bloom Filter type. - * - * @return bloom filter summarizing the column information - * @throws java.io.IOException if an I/O error occurs. - * Guarantees that file's current position will be just after the bloom filter, even if - * the filter cannot be deserialized, UNLESS EOFException is thrown. - */ - public static IFilter defreezeBloomFilter(FileDataInput file, long maxSize, FilterFactory.Type type) throws IOException - { - int size = file.readInt(); - if (size > maxSize || size <= 0) - throw new EOFException("bloom filter claims to be " + size + " bytes, longer than entire row size " + maxSize); - ByteBuffer bytes = file.readBytes(size); - - DataInputStream stream = new DataInputStream(ByteBufferUtil.inputStream(bytes)); - return FilterFactory.deserialize(stream, type, false); - } - /** * The index of the IndexInfo in which a scan starting with @name should begin. * http://git-wip-us.apache.org/repos/asf/cassandra/blob/302267e8/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java b/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java index cf6aeb3..07d78e4 100644 --- a/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java +++ b/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java @@ -126,7 +126,7 @@ public class SSTableIdentityIterator implements Comparable<SSTableIdentityIterat { try { - IndexHelper.skipSSTableBloomFilter(file, sstable.descriptor.version); + IndexHelper.skipBloomFilter(file); } catch (Exception e) { @@ -135,10 +135,9 @@ public class SSTableIdentityIterator implements Comparable<SSTableIdentityIterat logger.debug("Invalid bloom filter in {}; will rebuild it", sstable); } - try { - // deFreeze should have left the file position ready to deserialize index + // skipping the old row-level BF should have left the file position ready to deserialize index IndexHelper.deserializeIndex(file); } catch (Exception e) @@ -152,7 +151,7 @@ public class SSTableIdentityIterator implements Comparable<SSTableIdentityIterat if (sstable != null && !dataVersion.hasPromotedIndexes) { - IndexHelper.skipSSTableBloomFilter(inputWithTracker, dataVersion); + IndexHelper.skipBloomFilter(inputWithTracker); IndexHelper.skipIndex(inputWithTracker); } columnFamily = ColumnFamily.create(metadata); http://git-wip-us.apache.org/repos/asf/cassandra/blob/302267e8/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java index 2422f70..d798623 100644 --- a/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java +++ b/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java @@ -143,6 +143,7 @@ public class SSTableWriter extends SSTable if (logger.isTraceEnabled()) logger.trace("wrote " + decoratedKey + " at " + dataPosition); + // range tombstones are part of the Atoms we write as the row contents, so RIE only gets row-level tombstones RowIndexEntry entry = RowIndexEntry.create(dataPosition, delInfo.getTopLevelDeletion(), index); iwriter.append(decoratedKey, entry); dbuilder.addPotentialBoundary(dataPosition); @@ -180,7 +181,7 @@ public class SSTableWriter extends SSTable DataOutputBuffer buffer = new DataOutputBuffer(); // build column index && write columns - ColumnIndex.Builder builder = new ColumnIndex.Builder(cf, decoratedKey.key, cf.getColumnCount(), buffer); + ColumnIndex.Builder builder = new ColumnIndex.Builder(cf, decoratedKey.key, buffer); ColumnIndex index = builder.build(cf); TypeSizes typeSizes = TypeSizes.NATIVE; @@ -240,7 +241,7 @@ public class SSTableWriter extends SSTable ColumnFamily cf = ColumnFamily.create(metadata, ArrayBackedSortedColumns.factory()); cf.delete(deletionInfo); - ColumnIndex.Builder columnIndexer = new ColumnIndex.Builder(cf, key.key, columnCount, dataFile.stream, true); + ColumnIndex.Builder columnIndexer = new ColumnIndex.Builder(cf, key.key, dataFile.stream, true); OnDiskAtom.Serializer atomSerializer = cf.getOnDiskSerializer(); for (int i = 0; i < columnCount; i++) { http://git-wip-us.apache.org/repos/asf/cassandra/blob/302267e8/test/unit/org/apache/cassandra/Util.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java index a6ecbca..b2414ce 100644 --- a/test/unit/org/apache/cassandra/Util.java +++ b/test/unit/org/apache/cassandra/Util.java @@ -294,7 +294,7 @@ public class Util DataOutputStream dos = new DataOutputStream(baos); DeletionInfo.serializer().serializeForSSTable(cf.deletionInfo(), dos); dos.writeInt(cf.getColumnCount()); - new ColumnIndex.Builder(cf, ByteBufferUtil.EMPTY_BYTE_BUFFER, cf.getColumnCount(), dos).build(cf); + new ColumnIndex.Builder(cf, ByteBufferUtil.EMPTY_BYTE_BUFFER, dos).build(cf); return ByteBuffer.wrap(baos.toByteArray()); } catch (IOException e) http://git-wip-us.apache.org/repos/asf/cassandra/blob/302267e8/test/unit/org/apache/cassandra/cache/ObjectSizeTest.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/cache/ObjectSizeTest.java b/test/unit/org/apache/cassandra/cache/ObjectSizeTest.java index d925295..4adf4f9 100644 --- a/test/unit/org/apache/cassandra/cache/ObjectSizeTest.java +++ b/test/unit/org/apache/cassandra/cache/ObjectSizeTest.java @@ -5,6 +5,7 @@ import java.util.UUID; import junit.framework.Assert; +import org.apache.cassandra.db.ColumnIndex; import org.apache.cassandra.db.DeletionInfo; import org.apache.cassandra.db.DeletionTime; import org.apache.cassandra.db.RowIndexEntry; @@ -57,7 +58,7 @@ public class ObjectSizeTest @Test public void testKeyCacheValueWithDelInfo() { - RowIndexEntry entry = RowIndexEntry.create(123, new DeletionTime(123, 123), null); + RowIndexEntry entry = RowIndexEntry.create(123, new DeletionTime(123, 123), ColumnIndex.nothing()); long size = entry.memorySize(); long size2 = meter.measureDeep(entry); Assert.assertEquals(size, size2);
