Updated Branches:
refs/heads/trunk f650d3e82 -> 111f6c939
Merge branch 'cassandra-1.1' into trunk
Conflicts:
src/java/org/apache/cassandra/db/ColumnFamilyStore.java
src/java/org/apache/cassandra/io/sstable/Descriptor.java
src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java
src/java/org/apache/cassandra/io/sstable/SSTableReader.java
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/111f6c93
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/111f6c93
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/111f6c93
Branch: refs/heads/trunk
Commit: 111f6c9392766205823e326422f3dffae826cf97
Parents: f650d3e 6a6b7ec
Author: Sylvain Lebresne <[email protected]>
Authored: Thu Jul 26 18:04:14 2012 +0200
Committer: Sylvain Lebresne <[email protected]>
Committed: Thu Jul 26 18:04:14 2012 +0200
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../org/apache/cassandra/db/ColumnFamilyStore.java | 34 +++++++++++--
src/java/org/apache/cassandra/db/Directories.java | 11 ----
.../org/apache/cassandra/io/sstable/Component.java | 2 +
.../apache/cassandra/io/sstable/Descriptor.java | 8 +++
.../org/apache/cassandra/io/sstable/SSTable.java | 2 +
.../cassandra/io/sstable/SSTableMetadata.java | 33 +++++++++---
.../apache/cassandra/io/sstable/SSTableReader.java | 40 ++++-----------
.../apache/cassandra/tools/StandaloneScrubber.java | 2 +-
.../org/apache/cassandra/db/DirectoriesTest.java | 21 ++------
10 files changed, 84 insertions(+), 70 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/111f6c93/CHANGES.txt
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/111f6c93/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 7692e59,a39530a..695e480
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -229,13 -231,28 +229,28 @@@ public class ColumnFamilyStore implemen
if (loadSSTables)
{
- Directories.SSTableLister sstables =
directories.sstableLister().skipCompacted(true).skipTemporary(true);
-
data.addInitialSSTables(SSTableReader.batchOpen(sstables.list().entrySet(),
data, metadata, this.partitioner));
- Directories.SSTableLister sstableFiles =
directories.sstableLister().skipCompacted(true).skipTemporary(true);
- Collection<SSTableReader> sstables =
SSTableReader.batchOpen(sstableFiles.list().entrySet(), savedKeys, data,
metadata, this.partitioner);
++ Directories.SSTableLister sstableFiles =
directories.sstableLister().skipTemporary(true);
++ Collection<SSTableReader> sstables =
SSTableReader.batchOpen(sstableFiles.list().entrySet(), data, metadata,
this.partitioner);
+
+ // Filter non-compacted sstables, remove compacted ones
+ Set<Integer> compactedSSTables = new HashSet<Integer>();
+ for (SSTableReader sstable : sstables)
+ compactedSSTables.addAll(sstable.getAncestors());
+
+ Set<SSTableReader> liveSSTables = new HashSet<SSTableReader>();
+ for (SSTableReader sstable : sstables)
+ {
+ if (compactedSSTables.contains(sstable.descriptor.generation))
+ sstable.releaseReference(); // this amount to deleting
the sstable
+ else
+ liveSSTables.add(sstable);
+ }
+ data.addInitialSSTables(liveSSTables);
}
+ if (caching == Caching.ALL || caching == Caching.KEYS_ONLY)
+ CacheService.instance.keyCache.loadSaved(this);
+
// compaction strategy should be created after the CFS has been
prepared
this.compactionStrategy =
metadata.createCompactionStrategyInstance(this);
@@@ -451,7 -480,7 +466,7 @@@
currentDescriptors.add(sstable.descriptor);
Set<SSTableReader> newSSTables = new HashSet<SSTableReader>();
-- Directories.SSTableLister lister =
directories.sstableLister().skipCompacted(true).skipTemporary(true);
++ Directories.SSTableLister lister =
directories.sstableLister().skipTemporary(true);
for (Map.Entry<Descriptor, Set<Component>> entry :
lister.list().entrySet())
{
Descriptor descriptor = entry.getKey();
@@@ -478,7 -507,7 +493,7 @@@
SSTableReader reader;
try
{
- reader = SSTableReader.open(newDescriptor, entry.getValue(),
data, metadata, partitioner);
- reader = SSTableReader.open(newDescriptor, entry.getValue(),
Collections.<DecoratedKey>emptySet(), metadata, partitioner);
++ reader = SSTableReader.open(newDescriptor, entry.getValue(),
metadata, partitioner);
}
catch (IOException e)
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/111f6c93/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/Directories.java
index 02b1f25,f023c8c..20a8c20
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@@ -185,7 -185,7 +185,6 @@@ public class Directorie
public class SSTableLister
{
-- private boolean skipCompacted;
private boolean skipTemporary;
private boolean includeBackups;
private boolean onlyBackups;
@@@ -194,14 -194,14 +193,6 @@@
private boolean filtered;
private String snapshotName;
-- public SSTableLister skipCompacted(boolean b)
-- {
-- if (filtered)
-- throw new IllegalStateException("list() has already been
called");
-- skipCompacted = b;
-- return this;
-- }
--
public SSTableLister skipTemporary(boolean b)
{
if (filtered)
@@@ -294,8 -294,8 +285,6 @@@
if (pair == null)
return false;
-- if (skipCompacted && new
File(pair.left.filenameFor(Component.COMPACTED_MARKER)).exists())
-- return false;
if (skipTemporary && pair.left.temporary)
return false;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/111f6c93/src/java/org/apache/cassandra/io/sstable/Component.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/111f6c93/src/java/org/apache/cassandra/io/sstable/Descriptor.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/Descriptor.java
index 31ac072,07bd6e8..4f776d6
--- a/src/java/org/apache/cassandra/io/sstable/Descriptor.java
+++ b/src/java/org/apache/cassandra/io/sstable/Descriptor.java
@@@ -44,118 -47,19 +44,121 @@@ public class Descripto
// we always incremented the major version. In particular, versions g
and h are
// forwards-compatible with version f, so if the above convention had
been followed,
// we would have labeled them fb and fc.
- public static final String LEGACY_VERSION = "a"; // "pre-history"
- // b (0.7.0): added version to sstable filenames
- // c (0.7.0): bloom filter component computes hashes over raw key bytes
instead of strings
- // d (0.7.0): row size in data component becomes a long instead of int
- // e (0.7.0): stores undecorated keys in data and index components
- // f (0.7.0): switched bloom filter implementations in data component
- // g (0.8): tracks flushed-at context in metadata component
- // h (1.0): tracks max client timestamp in metadata component
- // hb (1.0.3): records compression ration in metadata component
- // hc (1.0.4): records partitioner in metadata component
- // hd (1.0.10): includes row tombstones in maxtimestamp
- // he (1.0.11): includes row tombstones in maxtimestamp
- public static final String CURRENT_VERSION = "he";
+ public static class Version
+ {
+ // This needs to be at the begining for initialization sake
+ private static final String current_version = "ia";
+
+ public static final Version LEGACY = new Version("a"); //
"pre-history"
+ // b (0.7.0): added version to sstable filenames
+ // c (0.7.0): bloom filter component computes hashes over raw key
bytes instead of strings
+ // d (0.7.0): row size in data component becomes a long instead of int
+ // e (0.7.0): stores undecorated keys in data and index components
+ // f (0.7.0): switched bloom filter implementations in data component
+ // g (0.8): tracks flushed-at context in metadata component
+ // h (1.0): tracks max client timestamp in metadata component
+ // hb (1.0.3): records compression ration in metadata component
+ // hc (1.0.4): records partitioner in metadata component
+ // hd (1.0.10): includes row tombstones in maxtimestamp
++ // he (1.1.3): includes ancestors generation in metadata component
+ // ia (1.2.0): column indexes are promoted to the index file
+ // records estimated histogram of deletion times in
tombstones
+ // bloom filter (keys and columns) upgraded to Murmur3
+
+ public static final Version CURRENT = new Version(current_version);
+
+ private final String version;
+
+ public final boolean hasStringsInBloomFilter;
+ public final boolean hasIntRowSize;
+ public final boolean hasEncodedKeys;
+ public final boolean isLatestVersion;
+ public final boolean metadataIncludesReplayPosition;
+ public final boolean tracksMaxTimestamp;
+ public final boolean hasCompressionRatio;
+ public final boolean hasPartitioner;
+ public final boolean tracksTombstones;
+ public final boolean hasPromotedIndexes;
+ public final FilterFactory.Type filterType;
++ public final boolean hasAncestors;
+
+ public Version(String version)
+ {
+ this.version = version;
+ hasStringsInBloomFilter = version.compareTo("c") < 0;
+ hasIntRowSize = version.compareTo("d") < 0;
+ hasEncodedKeys = version.compareTo("e") < 0;
+ metadataIncludesReplayPosition = version.compareTo("g") >= 0;
+ tracksMaxTimestamp = version.compareTo("hd") >= 0;
+ hasCompressionRatio = version.compareTo("hb") >= 0;
+ hasPartitioner = version.compareTo("hc") >= 0;
+ tracksTombstones = version.compareTo("ia") >= 0;
+ hasPromotedIndexes = version.compareTo("ia") >= 0;
+ isLatestVersion = version.compareTo(current_version) == 0;
+ if (version.compareTo("f") < 0)
+ filterType = FilterFactory.Type.SHA;
+ else if (version.compareTo("ia") < 0)
+ filterType = FilterFactory.Type.MURMUR2;
+ else
+ filterType = FilterFactory.Type.MURMUR3;
++ hasAncestors = version.compareTo("he") >= 0;
+ }
+
+ /**
+ * @param ver SSTable version
+ * @return True if the given version string matches the format.
+ * @see #version
+ */
+ static boolean validate(String ver)
+ {
+ return ver != null && ver.matches("[a-z]+");
+ }
+
+ public boolean isCompatible()
+ {
+ return version.charAt(0) <= CURRENT.version.charAt(0);
+ }
+
+ public boolean isStreamCompatible()
+ {
+ // we could add compatibility for earlier versions with the new
single-pass streaming
+ // (see SSTableWriter.appendFromStream) but versions earlier than
0.7.1 don't have the
+ // MessagingService version awareness anyway so there's no point.
+ return isCompatible() && version.charAt(0) >= 'i';
+ }
+
+ /**
+ * Versions [h..hc] contained a timestamp value that was computed
incorrectly, ignoring row tombstones.
+ * containsTimestamp returns true if there is a timestamp value in
the metadata file; to know if it
+ * actually contains a *correct* timestamp, see tracksMaxTimestamp.
+ */
+ public boolean containsTimestamp()
+ {
+ return version.compareTo("h") >= 0;
+ }
+
+ @Override
+ public String toString()
+ {
+ return version;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (o == this)
+ return true;
+ if (!(o instanceof Version))
+ return false;
+ return version.equals(((Version)o).version);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return version.hashCode();
+ }
+ }
public final File directory;
/** version has the following format: <code>[a-z]+</code> */
@@@ -189,8 -99,24 +192,13 @@@
this.generation = generation;
temporary = temp;
hashCode = Objects.hashCode(directory, generation, ksname, cfname);
-
- hasStringsInBloomFilter = version.compareTo("c") < 0;
- hasIntRowSize = version.compareTo("d") < 0;
- hasEncodedKeys = version.compareTo("e") < 0;
- usesOldBloomFilter = version.compareTo("f") < 0;
- metadataIncludesReplayPosition = version.compareTo("g") >= 0;
- tracksMaxTimestamp = version.compareTo("hd") >= 0;
- hasCompressionRatio = version.compareTo("hb") >= 0;
- hasPartitioner = version.compareTo("hc") >= 0;
- hasAncestors = version.compareTo("he") >= 0;
- isLatestVersion = version.compareTo(CURRENT_VERSION) == 0;
}
+ public Descriptor withGeneration(int newGeneration)
+ {
+ return new Descriptor(version, directory, ksname, cfname,
newGeneration, temporary);
+ }
+
public String filenameFor(Component component)
{
return filenameFor(component.name());
http://git-wip-us.apache.org/repos/asf/cassandra/blob/111f6c93/src/java/org/apache/cassandra/io/sstable/SSTable.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/SSTable.java
index ad1a104,9a29066..db7a9f8
--- a/src/java/org/apache/cassandra/io/sstable/SSTable.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTable.java
@@@ -142,8 -140,9 +142,10 @@@ public abstract class SSTabl
FileUtils.deleteWithConfirm(desc.filenameFor(component));
}
// remove the COMPACTED_MARKER component last if it exists
+ // Note: newly created sstable should not have a marker, but we keep
this for now to make sure
+ // we don't leave older marker around
FileUtils.delete(desc.filenameFor(Component.COMPACTED_MARKER));
+ FileUtils.delete(desc.filenameFor(Component.SUMMARY));
logger.debug("Deleted {}", desc);
return true;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/111f6c93/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java
index a0bc2cc,147f2b2..0413871
--- a/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java
@@@ -1,30 -1,27 +1,26 @@@
/*
-* 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.
-*/
-
+ * 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.io.sstable;
- import java.io.BufferedInputStream;
- import java.io.DataInputStream;
- import java.io.DataOutput;
- import java.io.File;
- import java.io.FileInputStream;
- import java.io.IOException;
+ import java.io.*;
+ import java.util.*;
+import org.apache.cassandra.utils.StreamingHistogram;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@@ -41,7 -38,7 +37,8 @@@ import org.apache.cassandra.utils.Estim
* - max column timestamp
* - compression ratio
* - partitioner
+ * - generations of sstables from which this sstable was compacted, if any
+ * - tombstone drop time histogram
*
* An SSTableMetadata should be instantiated via the Collector,
openFromDescriptor()
* or createDefaultInstance()
@@@ -58,7 -55,7 +55,8 @@@ public class SSTableMetadat
public final long maxTimestamp;
public final double compressionRatio;
public final String partitioner;
+ public final Set<Integer> ancestors;
+ public final StreamingHistogram estimatedTombstoneDropTime;
private SSTableMetadata()
{
@@@ -68,11 -65,10 +66,12 @@@
Long.MIN_VALUE,
Double.MIN_VALUE,
null,
- Collections.<Integer>emptySet());
++ Collections.<Integer>emptySet(),
+ defaultTombstoneDropTimeHistogram());
}
- private SSTableMetadata(EstimatedHistogram rowSizes, EstimatedHistogram
columnCounts, ReplayPosition replayPosition, long maxTimestamp, double cr,
String partitioner, Set<Integer> ancestors)
+ private SSTableMetadata(EstimatedHistogram rowSizes, EstimatedHistogram
columnCounts, ReplayPosition replayPosition, long maxTimestamp,
- double cr, String partitioner, StreamingHistogram
estimatedTombstoneDropTime)
++ double cr, String partitioner, Set<Integer>
ancestors, StreamingHistogram estimatedTombstoneDropTime)
{
this.estimatedRowSize = rowSizes;
this.estimatedColumnCount = columnCounts;
@@@ -80,7 -76,7 +79,8 @@@
this.maxTimestamp = maxTimestamp;
this.compressionRatio = cr;
this.partitioner = partitioner;
+ this.ancestors = ancestors;
+ this.estimatedTombstoneDropTime = estimatedTombstoneDropTime;
}
public static SSTableMetadata createDefaultInstance()
@@@ -132,7 -108,7 +132,8 @@@
protected ReplayPosition replayPosition = ReplayPosition.NONE;
protected long maxTimestamp = Long.MIN_VALUE;
protected double compressionRatio = Double.MIN_VALUE;
+ protected Set<Integer> ancestors = new HashSet<Integer>();
+ protected StreamingHistogram estimatedTombstoneDropTime =
defaultTombstoneDropTimeHistogram();
public void addRowSize(long rowSize)
{
@@@ -171,7 -142,7 +172,8 @@@
maxTimestamp,
compressionRatio,
partitioner,
- ancestors);
++ ancestors,
+ estimatedTombstoneDropTime);
}
public Collector estimatedRowSize(EstimatedHistogram estimatedRowSize)
@@@ -192,20 -163,11 +194,26 @@@
return this;
}
+ public Collector addAncestor(int generation)
+ {
+ this.ancestors.add(generation);
+ return this;
+ }
++
+ void update(long size, ColumnStats stats)
+ {
+ /*
+ * The max timestamp is not always collected here (more
precisely, row.maxTimestamp() may return Long.MIN_VALUE),
+ * to avoid deserializing an EchoedRow.
+ * This is the reason why it is collected first when calling
ColumnFamilyStore.createCompactionWriter
+ * However, for old sstables without timestamp, we still want to
update the timestamp (and we know
+ * that in this case we will not use EchoedRow, since
CompactionControler.needsDeserialize() will be true).
+ */
+ updateMaxTimestamp(stats.maxTimestamp);
+ addRowSize(size);
+ addColumnCount(stats.columnCount);
+ mergeTombstoneHistogram(stats.tombstoneHistogram);
+ }
}
public static class SSTableMetadataSerializer
@@@ -222,7 -184,9 +230,10 @@@
dos.writeLong(sstableStats.maxTimestamp);
dos.writeDouble(sstableStats.compressionRatio);
dos.writeUTF(sstableStats.partitioner);
+ dos.writeInt(sstableStats.ancestors.size());
+ for (Integer g : sstableStats.ancestors)
+ dos.writeInt(g);
+
StreamingHistogram.serializer.serialize(sstableStats.estimatedTombstoneDropTime,
dos);
}
public SSTableMetadata deserialize(Descriptor descriptor) throws
IOException
@@@ -250,20 -214,21 +261,24 @@@
{
EstimatedHistogram rowSizes =
EstimatedHistogram.serializer.deserialize(dis);
EstimatedHistogram columnCounts =
EstimatedHistogram.serializer.deserialize(dis);
- ReplayPosition replayPosition =
desc.metadataIncludesReplayPosition
+ ReplayPosition replayPosition =
desc.version.metadataIncludesReplayPosition
?
ReplayPosition.serializer.deserialize(dis)
: ReplayPosition.NONE;
- long maxTimestamp = desc.containsTimestamp() ? dis.readLong() :
Long.MIN_VALUE;
- if (!desc.tracksMaxTimestamp) // see javadoc to
Descriptor.containsTimestamp
+ long maxTimestamp = desc.version.containsTimestamp() ?
dis.readLong() : Long.MIN_VALUE;
+ if (!desc.version.tracksMaxTimestamp) // see javadoc to
Descriptor.containsTimestamp
maxTimestamp = Long.MIN_VALUE;
- double compressionRatio = desc.hasCompressionRatio
+ double compressionRatio = desc.version.hasCompressionRatio
? dis.readDouble()
- : Double.MIN_VALUE;
+ : Double.MIN_VALUE;
- String partitioner = desc.hasPartitioner ? dis.readUTF() : null;
- int nbAncestors = desc.hasAncestors ? dis.readInt() : 0;
+ String partitioner = desc.version.hasPartitioner ? dis.readUTF()
: null;
++ int nbAncestors = desc.version.hasAncestors ? dis.readInt() : 0;
+ Set<Integer> ancestors = new HashSet<Integer>(nbAncestors);
+ for (int i = 0; i < nbAncestors; i++)
+ ancestors.add(dis.readInt());
- return new SSTableMetadata(rowSizes, columnCounts,
replayPosition, maxTimestamp, compressionRatio, partitioner, ancestors);
+ StreamingHistogram tombstoneHistogram =
desc.version.tracksTombstones
+ ?
StreamingHistogram.serializer.deserialize(dis)
+ :
defaultTombstoneDropTimeHistogram();
- return new SSTableMetadata(rowSizes, columnCounts,
replayPosition, maxTimestamp, compressionRatio, partitioner,
tombstoneHistogram);
++ return new SSTableMetadata(rowSizes, columnCounts,
replayPosition, maxTimestamp, compressionRatio, partitioner, ancestors,
tombstoneHistogram);
}
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/111f6c93/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/SSTableReader.java
index 680fd8e,21dc71d..e188636
--- a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
@@@ -140,22 -140,22 +140,16 @@@ public class SSTableReader extends SSTa
public static SSTableReader openNoValidation(Descriptor descriptor,
Set<Component> components, CFMetaData metadata) throws IOException
{
- return open(descriptor, components, null, metadata,
StorageService.getPartitioner(), false);
- return open(descriptor, components,
Collections.<DecoratedKey>emptySet(), metadata,
StorageService.getPartitioner(), false);
++ return open(descriptor, components, metadata,
StorageService.getPartitioner(), false);
}
public static SSTableReader open(Descriptor descriptor, Set<Component>
components, CFMetaData metadata, IPartitioner partitioner) throws IOException
{
- return open(descriptor, components, null, metadata, partitioner);
- }
-
- public static SSTableReader open(Descriptor descriptor, Set<Component>
components, DataTracker tracker, CFMetaData metadata, IPartitioner partitioner)
throws IOException
- {
- return open(descriptor, components, tracker, metadata, partitioner,
true);
- return open(descriptor, components,
Collections.<DecoratedKey>emptySet(), metadata, partitioner);
- }
-
- public static SSTableReader open(Descriptor descriptor, Set<Component>
components, Set<DecoratedKey> savedKeys, CFMetaData metadata, IPartitioner
partitioner) throws IOException
- {
- return open(descriptor, components, savedKeys, metadata, partitioner,
true);
++ return open(descriptor, components, metadata, partitioner, true);
}
private static SSTableReader open(Descriptor descriptor,
Set<Component> components,
- DataTracker tracker,
- Set<DecoratedKey> savedKeys,
CFMetaData metadata,
IPartitioner partitioner,
boolean validate) throws IOException
@@@ -190,12 -190,10 +184,10 @@@
null,
System.currentTimeMillis(),
sstableMetadata);
- sstable.setTrackedBy(tracker);
-
// versions before 'c' encoded keys as utf-16 before hashing to the
filter
- if (descriptor.hasStringsInBloomFilter)
+ if (descriptor.version.hasStringsInBloomFilter)
{
- sstable.load(true, savedKeys);
+ sstable.load(true);
}
else
{
@@@ -240,7 -239,7 +232,7 @@@
SSTableReader sstable;
try
{
- sstable = open(entry.getKey(), entry.getValue(),
tracker, metadata, partitioner);
- sstable = open(entry.getKey(), entry.getValue(),
savedKeys, metadata, partitioner);
++ sstable = open(entry.getKey(), entry.getValue(),
metadata, partitioner);
}
catch (IOException ex)
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/111f6c93/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index 557f296,0ac6a80..7e42d75
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@@ -66,7 -67,7 +66,7 @@@ public class StandaloneScrubbe
String snapshotName = "pre-scrub-" + System.currentTimeMillis();
OutputHandler handler = new
OutputHandler.SystemOutput(options.verbose, options.debug);
-- Directories.SSTableLister lister =
cfs.directories.sstableLister().skipCompacted(true).skipTemporary(true);
++ Directories.SSTableLister lister =
cfs.directories.sstableLister().skipTemporary(true);
List<SSTableReader> sstables = new ArrayList<SSTableReader>();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/111f6c93/test/unit/org/apache/cassandra/db/DirectoriesTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/DirectoriesTest.java
index 3668735,3668735..d3683d2
--- a/test/unit/org/apache/cassandra/db/DirectoriesTest.java
+++ b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
@@@ -66,23 -66,23 +66,22 @@@ public class DirectoriesTes
File dir = cfDir(cf);
dir.mkdirs();
-- createFakeSSTable(dir, cf, 1, false, false, fs);
-- createFakeSSTable(dir, cf, 2, true, false, fs);
-- createFakeSSTable(dir, cf, 3, false, true, fs);
++ createFakeSSTable(dir, cf, 1, false, fs);
++ createFakeSSTable(dir, cf, 2, true, fs);
// leveled manifest
new File(dir, cf + LeveledManifest.EXTENSION).createNewFile();
File backupDir = new File(dir, Directories.BACKUPS_SUBDIR);
backupDir.mkdir();
-- createFakeSSTable(backupDir, cf, 1, false, false, fs);
++ createFakeSSTable(backupDir, cf, 1, false, fs);
File snapshotDir = new File(dir, Directories.SNAPSHOT_SUBDIR +
File.separator + "42");
snapshotDir.mkdirs();
-- createFakeSSTable(snapshotDir, cf, 1, false, false, fs);
++ createFakeSSTable(snapshotDir, cf, 1, false, fs);
}
}
-- private static void createFakeSSTable(File dir, String cf, int gen,
boolean temp, boolean compacted, List<File> addTo) throws IOException
++ private static void createFakeSSTable(File dir, String cf, int gen,
boolean temp, List<File> addTo) throws IOException
{
Descriptor desc = new Descriptor(dir, KS, cf, gen, temp);
for (Component c : new Component[]{ Component.DATA,
Component.PRIMARY_INDEX, Component.FILTER })
@@@ -91,12 -91,12 +90,6 @@@
f.createNewFile();
addTo.add(f);
}
-- if (compacted)
-- {
-- File f = new File(desc.filenameFor(Component.COMPACTED_MARKER));
-- f.createNewFile();
-- addTo.add(f);
-- }
}
private static File cfDir(String cf)
@@@ -153,7 -153,7 +146,7 @@@
}
// Skip temporary and compacted
-- lister =
directories.sstableLister().skipTemporary(true).skipCompacted(true);
++ lister = directories.sstableLister().skipTemporary(true);
listed = new HashSet<File>(lister.listFiles());
for (File f : files.get(cf))
{
@@@ -161,8 -161,8 +154,6 @@@
assert !listed.contains(f) : f + " should not be listed";
else if (f.getName().contains("-tmp-"))
assert !listed.contains(f) : f + " should not be listed";
-- else if (f.getName().endsWith("Compacted") || new
File(f.getPath().replaceFirst("-[a-zA-Z]+.db", "-Compacted")).exists())
-- assert !listed.contains(f) : f + " should not be listed";
else
assert listed.contains(f) : f + " is missing";
}