You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2011/11/09 18:17:54 UTC
svn commit: r1199874 - in /cassandra/branches/cassandra-1.0: ./
src/java/org/apache/cassandra/io/sstable/
test/unit/org/apache/cassandra/io/sstable/
Author: jbellis
Date: Wed Nov 9 17:17:54 2011
New Revision: 1199874
URL: http://svn.apache.org/viewvc?rev=1199874&view=rev
Log:
fix reading metadata/statistics component for version < h
patch by jbellis; reviewed by slebresne for CASSANDRA-3474
Modified:
cassandra/branches/cassandra-1.0/CHANGES.txt
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/Descriptor.java
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java
cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataSerializerTest.java
Modified: cassandra/branches/cassandra-1.0/CHANGES.txt
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/CHANGES.txt?rev=1199874&r1=1199873&r2=1199874&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0/CHANGES.txt (original)
+++ cassandra/branches/cassandra-1.0/CHANGES.txt Wed Nov 9 17:17:54 2011
@@ -5,6 +5,7 @@
* fix querying supercolumns by name returning only a subset of
subcolumns or old subcolumn versions (CASSANDRA-3446)
* automatically compute sha1 sum for uncompressed data files (CASSANDRA-3456)
+ * fix reading metadata/statistics component for version < h (CASSANDRA-3474)
Merged from 0.8:
* Make counter shard merging thread safe (CASSANDRA-3178)
* fix updating CF row_cache_provider (CASSANDRA-3414)
Modified: cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/Descriptor.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/Descriptor.java?rev=1199874&r1=1199873&r2=1199874&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/Descriptor.java (original)
+++ cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/Descriptor.java Wed Nov 9 17:17:54 2011
@@ -63,7 +63,8 @@ public class Descriptor
public final boolean hasEncodedKeys;
public final boolean isLatestVersion;
public final boolean usesOldBloomFilter;
- public final boolean usesHistogramAndReplayPositionStatsFile;
+ public final boolean metadataIncludesReplayPosition;
+ public final boolean tracksMaxTimestamp;
public enum TempState
{
@@ -103,15 +104,11 @@ public class Descriptor
hasIntRowSize = version.compareTo("d") < 0;
hasEncodedKeys = version.compareTo("e") < 0;
usesOldBloomFilter = version.compareTo("f") < 0;
- usesHistogramAndReplayPositionStatsFile = version.compareTo("h") < 0;
+ metadataIncludesReplayPosition = version.compareTo("g") >= 0;
+ tracksMaxTimestamp = version.compareTo("h") >= 0;
isLatestVersion = version.compareTo(CURRENT_VERSION) == 0;
}
- public boolean hasReplayPosition()
- {
- return version.compareTo("g") >= 0;
- }
-
public String filenameFor(Component component)
{
return filenameFor(component.name());
Modified: cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java?rev=1199874&r1=1199873&r2=1199874&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java (original)
+++ cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java Wed Nov 9 17:17:54 2011
@@ -20,7 +20,6 @@
package org.apache.cassandra.io.sstable;
import java.io.BufferedInputStream;
-import java.io.DataInput;
import java.io.DataInputStream;
import java.io.DataOutput;
import java.io.File;
@@ -31,7 +30,6 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.cassandra.db.commitlog.ReplayPosition;
-import org.apache.cassandra.io.ISerializer;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.utils.EstimatedHistogram;
@@ -57,13 +55,7 @@ public class SSTableMetadata
private SSTableMetadata()
{
- this(defaultRowSizeHistogram(), defaultColumnCountHistogram(), ReplayPosition.NONE);
- }
-
- // when there is no max timestamp recorded, default to max long
- private SSTableMetadata(EstimatedHistogram rowSizes, EstimatedHistogram columnCounts, ReplayPosition replayPosition)
- {
- this(rowSizes, columnCounts, replayPosition, Long.MAX_VALUE);
+ this(defaultRowSizeHistogram(), defaultColumnCountHistogram(), ReplayPosition.NONE, Long.MIN_VALUE);
}
private SSTableMetadata(EstimatedHistogram rowSizes, EstimatedHistogram columnCounts, ReplayPosition replayPosition, long maxTimestamp)
@@ -170,7 +162,7 @@ public class SSTableMetadata
}
}
- public static class SSTableMetadataSerializer implements ISerializer<SSTableMetadata>
+ public static class SSTableMetadataSerializer
{
private static final Logger logger = LoggerFactory.getLogger(SSTableMetadataSerializer.class);
@@ -184,6 +176,7 @@ public class SSTableMetadata
public SSTableMetadata deserialize(Descriptor descriptor) throws IOException
{
+ logger.debug("Load metadata for {}", descriptor);
File statsFile = new File(descriptor.filenameFor(SSTable.COMPONENT_STATS));
if (!statsFile.exists())
{
@@ -191,22 +184,10 @@ public class SSTableMetadata
return new SSTableMetadata();
}
- DataInputStream dis = null;
+ DataInputStream dis = new DataInputStream(new BufferedInputStream(new FileInputStream(statsFile)));
try
{
- logger.debug("Load metadata for {}", descriptor);
- dis = new DataInputStream(new BufferedInputStream(new FileInputStream(statsFile)));
-
- if (!descriptor.usesHistogramAndReplayPositionStatsFile)
- return deserialize(dis);
-
- EstimatedHistogram rowSizes = EstimatedHistogram.serializer.deserialize(dis);
- EstimatedHistogram columnCounts = EstimatedHistogram.serializer.deserialize(dis);
- ReplayPosition replayPosition = descriptor.hasReplayPosition()
- ? ReplayPosition.serializer.deserialize(dis)
- : ReplayPosition.NONE;
-
- return new SSTableMetadata(rowSizes, columnCounts, replayPosition);
+ return deserialize(dis, descriptor.metadataIncludesReplayPosition, descriptor.tracksMaxTimestamp);
}
finally
{
@@ -214,18 +195,16 @@ public class SSTableMetadata
}
}
- public SSTableMetadata deserialize(DataInput dis) throws IOException
+ public SSTableMetadata deserialize(DataInputStream dis, boolean includesReplayPosition, boolean tracksMaxTimestamp) throws IOException
{
EstimatedHistogram rowSizes = EstimatedHistogram.serializer.deserialize(dis);
EstimatedHistogram columnCounts = EstimatedHistogram.serializer.deserialize(dis);
- ReplayPosition replayPosition = ReplayPosition.serializer.deserialize(dis);
- long maxTimestamp = dis.readLong();
- return new SSTableMetadata(rowSizes, columnCounts, replayPosition, maxTimestamp);
- }
+ ReplayPosition replayPosition = includesReplayPosition
+ ? ReplayPosition.serializer.deserialize(dis)
+ : ReplayPosition.NONE;
+ long maxTimestamp = tracksMaxTimestamp ? dis.readLong() : Long.MIN_VALUE;
- public long serializedSize(SSTableMetadata object)
- {
- throw new UnsupportedOperationException();
+ return new SSTableMetadata(rowSizes, columnCounts, replayPosition, maxTimestamp);
}
}
}
Modified: cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataSerializerTest.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataSerializerTest.java?rev=1199874&r1=1199873&r2=1199874&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataSerializerTest.java (original)
+++ cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataSerializerTest.java Wed Nov 9 17:17:54 2011
@@ -58,8 +58,7 @@ public class SSTableMetadataSerializerTe
ByteArrayInputStream byteInput = new ByteArrayInputStream(byteOutput.toByteArray());
DataInputStream dis = new DataInputStream(byteInput);
-
- SSTableMetadata stats = SSTableMetadata.serializer.deserialize(dis);
+ SSTableMetadata stats = SSTableMetadata.serializer.deserialize(dis, true, true);
assert stats.getEstimatedRowSize().equals(originalMetadata.getEstimatedRowSize());
assert stats.getEstimatedRowSize().equals(rowSizes);