Author: jbellis
Date: Tue Oct 5 16:00:40 2010
New Revision: 1004693
URL: http://svn.apache.org/viewvc?rev=1004693&view=rev
Log:
r/m getters for public final sstablereader fields. patch by jbellis
Modified:
cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTable.java
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableDeletingReference.java
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamOut.java
cassandra/trunk/test/unit/org/apache/cassandra/db/TableTest.java
cassandra/trunk/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java
cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableTest.java
cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java
Modified:
cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=1004693&r1=1004692&r2=1004693&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Tue
Oct 5 16:00:40 2010
@@ -59,10 +59,7 @@ import org.apache.cassandra.service.Stor
import org.apache.cassandra.thrift.IndexClause;
import org.apache.cassandra.thrift.IndexExpression;
import org.apache.cassandra.thrift.IndexOperator;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.LatencyTracker;
-import org.apache.cassandra.utils.Pair;
-import org.apache.cassandra.utils.WrappedRunnable;
+import org.apache.cassandra.utils.*;
public class ColumnFamilyStore implements ColumnFamilyStoreMBean
{
@@ -278,8 +275,8 @@ public class ColumnFamilyStore implement
long min = 0;
for (SSTableReader sstable : ssTables)
{
- if (min == 0 || sstable.getEstimatedRowSize().min() < min)
- min = sstable.getEstimatedRowSize().min();
+ if (min == 0 || sstable.estimatedRowSize.min() < min)
+ min = sstable.estimatedRowSize.min();
}
return min;
}
@@ -289,8 +286,8 @@ public class ColumnFamilyStore implement
long max = 0;
for (SSTableReader sstable : ssTables)
{
- if (sstable.getEstimatedRowSize().max() > max)
- max = sstable.getEstimatedRowSize().max();
+ if (sstable.estimatedRowSize.max() > max)
+ max = sstable.estimatedRowSize.max();
}
return max;
}
@@ -301,7 +298,7 @@ public class ColumnFamilyStore implement
long count = 0;
for (SSTableReader sstable : ssTables)
{
- sum += sstable.getEstimatedRowSize().median();
+ sum += sstable.estimatedRowSize.median();
count++;
}
return count > 0 ? sum / count : 0;
@@ -313,7 +310,7 @@ public class ColumnFamilyStore implement
int count = 0;
for (SSTableReader sstable : ssTables)
{
- sum += sstable.getEstimatedColumnCount().median();
+ sum += sstable.estimatedColumnCount.median();
count++;
}
return count > 0 ? (int) (sum / count) : 0;
@@ -1353,14 +1350,14 @@ public class ColumnFamilyStore implement
try
{
// mkdir
- File dataDirectory =
ssTable.getDescriptor().directory.getParentFile();
+ File dataDirectory =
ssTable.descriptor.directory.getParentFile();
String snapshotDirectoryPath =
Table.getSnapshotPath(dataDirectory.getAbsolutePath(), table.name,
snapshotName);
FileUtils.createDirectory(snapshotDirectoryPath);
// hard links
- for (Component component : ssTable.getComponents())
+ for (Component component : ssTable.components)
{
- File sourceFile = new
File(ssTable.getDescriptor().filenameFor(component));
+ File sourceFile = new
File(ssTable.descriptor.filenameFor(component));
File targetLink = new File(snapshotDirectoryPath,
sourceFile.getName());
FileUtils.createHardLink(sourceFile, targetLink);
}
Modified:
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java?rev=1004693&r1=1004692&r2=1004693&view=diff
==============================================================================
---
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java
(original)
+++
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java
Tue Oct 5 16:00:40 2010
@@ -61,11 +61,11 @@ public class SSTableNamesIterator extend
try
{
- DecoratedKey keyInDisk =
SSTableReader.decodeKey(sstable.getPartitioner(),
-
sstable.getDescriptor(),
+ DecoratedKey keyInDisk =
SSTableReader.decodeKey(sstable.partitioner,
+
sstable.descriptor,
FBUtilities.readShortByteArray(file));
assert keyInDisk.equals(key) : String.format("%s != %s in %s",
keyInDisk, key, file.getPath());
- SSTableReader.readRowSize(file, sstable.getDescriptor());
+ SSTableReader.readRowSize(file, sstable.descriptor);
read(sstable.metadata, file);
}
catch (IOException e)
Modified:
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java?rev=1004693&r1=1004692&r2=1004693&view=diff
==============================================================================
---
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java
(original)
+++
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java
Tue Oct 5 16:00:40 2010
@@ -52,12 +52,12 @@ public class SSTableSliceIterator implem
try
{
- DecoratedKey keyInDisk =
SSTableReader.decodeKey(sstable.getPartitioner(),
-
sstable.getDescriptor(),
+ DecoratedKey keyInDisk =
SSTableReader.decodeKey(sstable.partitioner,
+
sstable.descriptor,
FBUtilities.readShortByteArray(fileToClose));
assert keyInDisk.equals(key)
: String.format("%s != %s in %s", keyInDisk, key,
fileToClose.getPath());
- SSTableReader.readRowSize(fileToClose, sstable.getDescriptor());
+ SSTableReader.readRowSize(fileToClose, sstable.descriptor);
}
catch (IOException e)
{
Modified:
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java?rev=1004693&r1=1004692&r2=1004693&view=diff
==============================================================================
---
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java
(original)
+++
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java
Tue Oct 5 16:00:40 2010
@@ -42,7 +42,7 @@ public class ReducingKeyIterator impleme
ci = FBUtilities.getCollatingIterator();
for (SSTableReader sstable : sstables)
{
- ci.addIterator(new KeyIterator(sstable.desc));
+ ci.addIterator(new KeyIterator(sstable.descriptor));
}
iter = new ReducingIterator<DecoratedKey, DecoratedKey>(ci)
Modified: cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTable.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTable.java?rev=1004693&r1=1004692&r2=1004693&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTable.java
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTable.java Tue
Oct 5 16:00:40 2010
@@ -59,54 +59,29 @@ public abstract class SSTable
public static final String COMPONENT_COMPACTED =
Component.Type.COMPACTED_MARKER.repr;
- protected final Descriptor desc;
- protected final Set<Component> components;
- public final CFMetaData metadata;
- protected final IPartitioner partitioner;
-
public static final String TEMPFILE_MARKER = "tmp";
- protected EstimatedHistogram estimatedRowSize = new
EstimatedHistogram(150);
- protected EstimatedHistogram estimatedColumnCount = new
EstimatedHistogram(114);
+ public final Descriptor descriptor;
+ public final Set<Component> components;
+ public final CFMetaData metadata;
+ public final IPartitioner partitioner;
- protected SSTable(Descriptor desc, CFMetaData metadata, IPartitioner
partitioner)
+ public final EstimatedHistogram estimatedRowSize = new
EstimatedHistogram(150);
+ public final EstimatedHistogram estimatedColumnCount = new
EstimatedHistogram(114);
+
+ protected SSTable(Descriptor descriptor, CFMetaData metadata, IPartitioner
partitioner)
{
- this(desc, new HashSet<Component>(), metadata, partitioner);
+ this(descriptor, new HashSet<Component>(), metadata, partitioner);
}
- protected SSTable(Descriptor desc, Set<Component> components, CFMetaData
metadata, IPartitioner partitioner)
+ protected SSTable(Descriptor descriptor, Set<Component> components,
CFMetaData metadata, IPartitioner partitioner)
{
- this.desc = desc;
+ this.descriptor = descriptor;
this.components = components;
this.metadata = metadata;
this.partitioner = partitioner;
}
- public EstimatedHistogram getEstimatedRowSize()
- {
- return estimatedRowSize;
- }
-
- public EstimatedHistogram getEstimatedColumnCount()
- {
- return estimatedColumnCount;
- }
-
- public IPartitioner getPartitioner()
- {
- return partitioner;
- }
-
- public Descriptor getDescriptor()
- {
- return desc;
- }
-
- public Set<Component> getComponents()
- {
- return components;
- }
-
/**
* We use a ReferenceQueue to manage deleting files that have been
compacted
* and for which no more SSTable references exist. But this is not
guaranteed
@@ -148,17 +123,17 @@ public abstract class SSTable
public String getFilename()
{
- return desc.filenameFor(COMPONENT_DATA);
+ return descriptor.filenameFor(COMPONENT_DATA);
}
public String getColumnFamilyName()
{
- return desc.cfname;
+ return descriptor.cfname;
}
public String getTableName()
{
- return desc.ksname;
+ return descriptor.ksname;
}
/**
@@ -212,7 +187,7 @@ public abstract class SSTable
long bytes = 0;
for (Component component : components)
{
- bytes += new File(desc.filenameFor(component)).length();
+ bytes += new File(descriptor.filenameFor(component)).length();
}
return bytes;
}
Modified:
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableDeletingReference.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableDeletingReference.java?rev=1004693&r1=1004692&r2=1004693&view=diff
==============================================================================
---
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableDeletingReference.java
(original)
+++
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableDeletingReference.java
Tue Oct 5 16:00:40 2010
@@ -48,8 +48,8 @@ public class SSTableDeletingReference ex
{
super(referent, q);
this.tracker = tracker;
- this.desc = referent.getDescriptor();
- this.components = referent.getComponents();
+ this.desc = referent.descriptor;
+ this.components = referent.components;
this.size = referent.bytesOnDisk();
}
Modified:
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableReader.java?rev=1004693&r1=1004692&r2=1004693&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
Tue Oct 5 16:00:40 2010
@@ -224,7 +224,7 @@ public class SSTableReader extends SSTab
void loadBloomFilter() throws IOException
{
- DataInputStream stream = new DataInputStream(new
FileInputStream(desc.filenameFor(Component.FILTER)));
+ DataInputStream stream = new DataInputStream(new
FileInputStream(descriptor.filenameFor(Component.FILTER)));
try
{
bf = BloomFilter.serializer().deserialize(stream);
@@ -245,7 +245,7 @@ public class SSTableReader extends SSTab
// we read the positions in a BRAF so we don't have to worry about an
entry spanning a mmap boundary.
indexSummary = new IndexSummary();
- BufferedRandomAccessFile input = new
BufferedRandomAccessFile(desc.filenameFor(Component.PRIMARY_INDEX), "r");
+ BufferedRandomAccessFile input = new
BufferedRandomAccessFile(descriptor.filenameFor(Component.PRIMARY_INDEX), "r");
try
{
long indexSize = input.length();
@@ -258,7 +258,7 @@ public class SSTableReader extends SSTab
if (indexPosition == indexSize)
break;
- DecoratedKey decoratedKey = decodeKey(partitioner, desc,
FBUtilities.readShortByteArray(input));
+ DecoratedKey decoratedKey = decodeKey(partitioner, descriptor,
FBUtilities.readShortByteArray(input));
if (recreatebloom)
bf.add(decoratedKey.key);
long dataPosition = input.readLong();
@@ -276,8 +276,8 @@ public class SSTableReader extends SSTab
// finalize the state of the reader
indexSummary.complete();
- ifile = ibuilder.complete(desc.filenameFor(Component.PRIMARY_INDEX));
- dfile = dbuilder.complete(desc.filenameFor(Component.DATA));
+ ifile =
ibuilder.complete(descriptor.filenameFor(Component.PRIMARY_INDEX));
+ dfile = dbuilder.complete(descriptor.filenameFor(Component.DATA));
}
/** get the position in the index file to start scanning to find the given
key (at most indexInterval keys away) */
@@ -381,7 +381,7 @@ public class SSTableReader extends SSTab
return -1;
// next, the key cache
- Pair<Descriptor, DecoratedKey> unifiedKey = new Pair<Descriptor,
DecoratedKey>(desc, decoratedKey);
+ Pair<Descriptor, DecoratedKey> unifiedKey = new Pair<Descriptor,
DecoratedKey>(descriptor, decoratedKey);
if (keyCache != null && keyCache.getCapacity() > 0)
{
Long cachedPosition = keyCache.get(unifiedKey);
@@ -411,7 +411,7 @@ public class SSTableReader extends SSTab
while (!input.isEOF())
{
// read key & data position from index entry
- DecoratedKey indexDecoratedKey = decodeKey(partitioner,
desc, FBUtilities.readShortByteArray(input));
+ DecoratedKey indexDecoratedKey = decodeKey(partitioner,
descriptor, FBUtilities.readShortByteArray(input));
long dataPosition = input.readLong();
int comparison = indexDecoratedKey.compareTo(decoratedKey);
@@ -471,7 +471,7 @@ public class SSTableReader extends SSTab
logger.debug("Marking " + getFilename() + " compacted");
try
{
- if (!new
File(desc.filenameFor(Component.COMPACTED_MARKER)).createNewFile())
+ if (!new
File(descriptor.filenameFor(Component.COMPACTED_MARKER)).createNewFile())
throw new IOException("Unable to create compaction marker");
}
catch (IOException e)
@@ -513,7 +513,7 @@ public class SSTableReader extends SSTab
public int compareTo(SSTableReader o)
{
- return desc.generation - o.desc.generation;
+ return descriptor.generation - o.descriptor.generation;
}
public AbstractType getColumnComparator()
Modified:
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java?rev=1004693&r1=1004692&r2=1004693&view=diff
==============================================================================
---
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java
(original)
+++
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java
Tue Oct 5 16:00:40 2010
@@ -167,10 +167,10 @@ public class SSTableScanner implements I
file.seek(finishedAt);
assert !file.isEOF();
- DecoratedKey key =
SSTableReader.decodeKey(sstable.getPartitioner(),
-
sstable.getDescriptor(),
+ DecoratedKey key = SSTableReader.decodeKey(sstable.partitioner,
+ sstable.descriptor,
FBUtilities.readShortByteArray(file));
- long dataSize = SSTableReader.readRowSize(file,
sstable.getDescriptor());
+ long dataSize = SSTableReader.readRowSize(file,
sstable.descriptor);
dataStart = file.getFilePointer();
finishedAt = dataStart + dataSize;
Modified:
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java?rev=1004693&r1=1004692&r2=1004693&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
Tue Oct 5 16:00:40 2010
@@ -21,8 +21,6 @@ package org.apache.cassandra.io.sstable;
import java.io.*;
import java.util.Set;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -57,7 +55,7 @@ public class SSTableWriter extends SSTab
public SSTableWriter(String filename, long keyCount, CFMetaData metadata,
IPartitioner partitioner) throws IOException
{
super(Descriptor.fromFilename(filename), metadata, partitioner);
- iwriter = new IndexWriter(desc, partitioner, keyCount);
+ iwriter = new IndexWriter(descriptor, partitioner, keyCount);
dbuilder =
SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode());
dataFile = new BufferedRandomAccessFile(getFilename(), "rw",
DatabaseDescriptor.getInMemoryCompactionLimit());
@@ -148,10 +146,10 @@ public class SSTableWriter extends SSTab
dataFile.close(); // calls force
// write sstable statistics
- writeStatistics(desc);
+ writeStatistics(descriptor);
// remove the 'tmp' marker from all components
- final Descriptor newdesc = rename(desc, components);
+ final Descriptor newdesc = rename(descriptor, components);
// finalize in-memory state for the reader
Modified: cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamOut.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamOut.java?rev=1004693&r1=1004692&r2=1004693&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamOut.java
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamOut.java Tue
Oct 5 16:00:40 2010
@@ -162,7 +162,7 @@ public class StreamOut
List<PendingFile> pending = new ArrayList<PendingFile>();
for (SSTableReader sstable : sstables)
{
- Descriptor desc = sstable.getDescriptor();
+ Descriptor desc = sstable.descriptor;
List<Pair<Long,Long>> sections =
sstable.getPositionsForRanges(ranges);
if (sections.isEmpty())
continue;
Modified: cassandra/trunk/test/unit/org/apache/cassandra/db/TableTest.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/db/TableTest.java?rev=1004693&r1=1004692&r2=1004693&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/db/TableTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/db/TableTest.java Tue Oct 5
16:00:40 2010
@@ -399,7 +399,7 @@ public class TableTest extends CleanupHe
BufferedRandomAccessFile file = new
BufferedRandomAccessFile(sstable.getFilename(), "r");
file.seek(position);
assert Arrays.equals(FBUtilities.readShortByteArray(file), key.key);
- SSTableReader.readRowSize(file, sstable.getDescriptor());
+ SSTableReader.readRowSize(file, sstable.descriptor);
IndexHelper.skipBloomFilter(file);
ArrayList<IndexHelper.IndexInfo> indexes =
IndexHelper.deserializeIndex(file);
assert indexes.size() > 2;
Modified:
cassandra/trunk/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java?rev=1004693&r1=1004692&r2=1004693&view=diff
==============================================================================
---
cassandra/trunk/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java
(original)
+++
cassandra/trunk/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java
Tue Oct 5 16:00:40 2010
@@ -24,28 +24,19 @@ package org.apache.cassandra.io;
import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
import java.io.IOException;
-import java.net.InetAddress;
-import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
-import java.util.List;
import java.util.concurrent.ExecutionException;
import org.junit.Test;
import org.apache.cassandra.CleanupHelper;
-import org.apache.cassandra.Util;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.filter.QueryPath;
-import org.apache.cassandra.dht.CollatingOrderPreservingPartitioner;
-import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.dht.Range;
import org.apache.cassandra.io.sstable.IndexHelper;
import org.apache.cassandra.io.sstable.SSTableReader;
import org.apache.cassandra.io.util.DataOutputBuffer;
-import org.apache.cassandra.utils.BloomFilter;
-import org.apache.cassandra.utils.FBUtilities;
import static junit.framework.Assert.assertEquals;
@@ -78,8 +69,8 @@ public class LazilyCompactedRowTest exte
// key isn't part of what CompactedRow writes, that's done by
SSTW.append
// row size can differ b/c of bloom filter counts being different
- long rowSize1 = SSTableReader.readRowSize(in1,
sstables.iterator().next().getDescriptor());
- long rowSize2 = SSTableReader.readRowSize(in2,
sstables.iterator().next().getDescriptor());
+ long rowSize1 = SSTableReader.readRowSize(in1,
sstables.iterator().next().descriptor);
+ long rowSize2 = SSTableReader.readRowSize(in2,
sstables.iterator().next().descriptor);
assertEquals(out1.getLength(), rowSize1 + 8);
assertEquals(out2.getLength(), rowSize2 + 8);
// bloom filter
Modified:
cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java?rev=1004693&r1=1004692&r2=1004693&view=diff
==============================================================================
---
cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
(original)
+++
cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
Tue Oct 5 16:00:40 2010
@@ -116,8 +116,8 @@ public class SSTableReaderTest extends C
{
DecoratedKey dk = Util.dk(String.valueOf(j));
FileDataInput file = sstable.getFileDataInput(dk,
DatabaseDescriptor.getIndexedReadBufferSizeInKB() * 1024);
- DecoratedKey keyInDisk =
SSTableReader.decodeKey(sstable.getPartitioner(),
-
sstable.getDescriptor(),
+ DecoratedKey keyInDisk =
SSTableReader.decodeKey(sstable.partitioner,
+
sstable.descriptor,
FBUtilities.readShortByteArray(file));
assert keyInDisk.equals(dk) : String.format("%s != %s in %s",
keyInDisk, dk, file.getPath());
}
Modified:
cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableTest.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableTest.java?rev=1004693&r1=1004692&r2=1004693&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableTest.java
(original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableTest.java
Tue Oct 5 16:00:40 2010
@@ -43,7 +43,7 @@ public class SSTableTest extends Cleanup
// verify
verifySingle(ssTable, bytes, key);
- ssTable = SSTableReader.open(ssTable.getDescriptor()); // read the
index from disk
+ ssTable = SSTableReader.open(ssTable.descriptor); // read the index
from disk
verifySingle(ssTable, bytes, key);
}
@@ -52,7 +52,7 @@ public class SSTableTest extends Cleanup
BufferedRandomAccessFile file = new
BufferedRandomAccessFile(sstable.getFilename(), "r");
file.seek(sstable.getPosition(sstable.partitioner.decorateKey(key),
SSTableReader.Operator.EQ));
assert Arrays.equals(key, FBUtilities.readShortByteArray(file));
- int size = (int)SSTableReader.readRowSize(file,
sstable.getDescriptor());
+ int size = (int)SSTableReader.readRowSize(file, sstable.descriptor);
byte[] bytes2 = new byte[size];
file.readFully(bytes2);
assert Arrays.equals(bytes2, bytes);
@@ -71,7 +71,7 @@ public class SSTableTest extends Cleanup
// verify
verifyMany(ssTable, map);
- ssTable = SSTableReader.open(ssTable.getDescriptor()); // read the
index from disk
+ ssTable = SSTableReader.open(ssTable.descriptor); // read the index
from disk
verifyMany(ssTable, map);
}
@@ -84,7 +84,7 @@ public class SSTableTest extends Cleanup
{
file.seek(sstable.getPosition(sstable.partitioner.decorateKey(key),
SSTableReader.Operator.EQ));
assert Arrays.equals(key, FBUtilities.readShortByteArray(file));
- int size = (int)SSTableReader.readRowSize(file,
sstable.getDescriptor());
+ int size = (int)SSTableReader.readRowSize(file,
sstable.descriptor);
byte[] bytes2 = new byte[size];
file.readFully(bytes2);
assert Arrays.equals(bytes2, map.get(key));
Modified:
cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java?rev=1004693&r1=1004692&r2=1004693&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
(original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
Tue Oct 5 16:00:40 2010
@@ -25,7 +25,6 @@ import java.util.*;
import org.apache.cassandra.db.*;
import org.apache.cassandra.io.util.DataOutputBuffer;
-import org.apache.cassandra.service.StorageService;
public class SSTableUtils
{
@@ -91,8 +90,8 @@ public class SSTableUtils
sortedEntries.put(writer.partitioner.decorateKey(entry.getKey()),
entry.getValue());
for (Map.Entry<DecoratedKey, byte[]> entry : sortedEntries.entrySet())
writer.append(entry.getKey(), entry.getValue());
- new
File(writer.desc.filenameFor(Component.PRIMARY_INDEX)).deleteOnExit();
- new File(writer.desc.filenameFor(Component.FILTER)).deleteOnExit();
+ new
File(writer.descriptor.filenameFor(Component.PRIMARY_INDEX)).deleteOnExit();
+ new
File(writer.descriptor.filenameFor(Component.FILTER)).deleteOnExit();
return writer.closeAndOpenReader();
}
Modified:
cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java?rev=1004693&r1=1004692&r2=1004693&view=diff
==============================================================================
---
cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java
(original)
+++
cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java
Tue Oct 5 16:00:40 2010
@@ -23,7 +23,6 @@ package org.apache.cassandra.io.sstable;
import static org.junit.Assert.*;
-import java.io.File;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
@@ -76,10 +75,10 @@ public class SSTableWriterTest extends C
SSTableReader orig = SSTableUtils.writeRawSSTable("Keyspace1",
"Indexed1", entries);
// whack the index to trigger the recover
-
FileUtils.deleteWithConfirm(orig.desc.filenameFor(Component.PRIMARY_INDEX));
- FileUtils.deleteWithConfirm(orig.desc.filenameFor(Component.FILTER));
+
FileUtils.deleteWithConfirm(orig.descriptor.filenameFor(Component.PRIMARY_INDEX));
+
FileUtils.deleteWithConfirm(orig.descriptor.filenameFor(Component.FILTER));
- SSTableReader sstr =
CompactionManager.instance.submitSSTableBuild(orig.desc).get();
+ SSTableReader sstr =
CompactionManager.instance.submitSSTableBuild(orig.descriptor).get();
ColumnFamilyStore cfs =
Table.open("Keyspace1").getColumnFamilyStore("Indexed1");
cfs.addSSTable(sstr);
cfs.buildSecondaryIndexes(cfs.getSSTables(), cfs.getIndexedColumns());