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/02/23 20:14:15 UTC
svn commit: r1073889 - in
/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra:
db/BinaryMemtable.java db/ColumnFamilyStore.java db/CompactionManager.java
db/Memtable.java io/sstable/SSTableWriter.java tools/SSTableImport.java
Author: jbellis
Date: Wed Feb 23 19:14:15 2011
New Revision: 1073889
URL: http://svn.apache.org/viewvc?rev=1073889&view=rev
Log:
revert last until tests are fixed
Modified:
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/BinaryMemtable.java
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/CompactionManager.java
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/Memtable.java
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/tools/SSTableImport.java
Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/BinaryMemtable.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/BinaryMemtable.java?rev=1073889&r1=1073888&r2=1073889&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/BinaryMemtable.java (original)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/BinaryMemtable.java Wed Feb 23 19:14:15 2011
@@ -125,7 +125,8 @@ public class BinaryMemtable implements I
private SSTableReader writeSortedContents(List<DecoratedKey> sortedKeys) throws IOException
{
logger.info("Writing " + this);
- SSTableWriter writer = cfs.createFlushWriter(sortedKeys.size());
+ String path = cfs.getFlushPath();
+ SSTableWriter writer = new SSTableWriter(path, sortedKeys.size(), cfs.metadata, cfs.partitioner);
for (DecoratedKey key : sortedKeys)
{
Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=1073889&r1=1073888&r2=1073889&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java (original)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Wed Feb 23 19:14:15 2011
@@ -2063,14 +2063,4 @@ public class ColumnFamilyStore implement
ssTables.getKeyCache().setCapacity(newCapacity);
}
}
-
- public SSTableWriter createFlushWriter(long estimatedRows) throws IOException
- {
- return new SSTableWriter(getFlushPath(), estimatedRows, metadata, partitioner);
- }
-
- public SSTableWriter createCompactionWriter(long estimatedRows, String location) throws IOException
- {
- return new SSTableWriter(getTempSSTablePath(location), estimatedRows, metadata, partitioner);
- }
}
Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/CompactionManager.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/CompactionManager.java?rev=1073889&r1=1073888&r2=1073889&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/CompactionManager.java (original)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/CompactionManager.java Wed Feb 23 19:14:15 2011
@@ -444,7 +444,8 @@ public class CompactionManager implement
return 0;
}
- writer = cfs.createCompactionWriter(expectedBloomFilterSize, compactionFileLocation);
+ String newFilename = new File(cfs.getTempSSTablePath(compactionFileLocation)).getAbsolutePath();
+ writer = new SSTableWriter(newFilename, expectedBloomFilterSize, cfs.metadata, cfs.partitioner);
while (nni.hasNext())
{
AbstractCompactedRow row = nni.next();
@@ -705,7 +706,8 @@ public class CompactionManager implement
if (writer == null)
{
FileUtils.createDirectory(compactionFileLocation);
- writer = cfs.createCompactionWriter(expectedBloomFilterSize, compactionFileLocation);
+ String newFilename = new File(cfs.getTempSSTablePath(compactionFileLocation)).getAbsolutePath();
+ writer = new SSTableWriter(newFilename, expectedBloomFilterSize, cfs.metadata, cfs.partitioner);
}
return writer;
}
Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/Memtable.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/Memtable.java?rev=1073889&r1=1073888&r2=1073889&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/Memtable.java (original)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/Memtable.java Wed Feb 23 19:14:15 2011
@@ -155,7 +155,7 @@ public class Memtable implements Compara
private SSTableReader writeSortedContents() throws IOException
{
logger.info("Writing " + this);
- SSTableWriter writer = cfs.createFlushWriter(columnFamilies.size());
+ SSTableWriter writer = new SSTableWriter(cfs.getFlushPath(), columnFamilies.size(), cfs.metadata, cfs.partitioner);
for (Map.Entry<DecoratedKey, ColumnFamily> entry : columnFamilies.entrySet())
writer.append(entry.getKey(), entry.getValue());
Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java?rev=1073889&r1=1073888&r2=1073889&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java (original)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java Wed Feb 23 19:14:15 2011
@@ -57,6 +57,11 @@ public class SSTableWriter extends SSTab
private DecoratedKey lastWrittenKey;
private FileMark dataMark;
+ public SSTableWriter(String filename, long keyCount) throws IOException
+ {
+ this(filename, keyCount, DatabaseDescriptor.getCFMetaData(Descriptor.fromFilename(filename)), StorageService.getPartitioner());
+ }
+
public SSTableWriter(String filename, long keyCount, CFMetaData metadata, IPartitioner partitioner) throws IOException
{
super(Descriptor.fromFilename(filename),
Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/tools/SSTableImport.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/tools/SSTableImport.java?rev=1073889&r1=1073888&r2=1073889&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/tools/SSTableImport.java (original)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/tools/SSTableImport.java Wed Feb 23 19:14:15 2011
@@ -31,7 +31,6 @@ import org.apache.cassandra.config.Datab
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.filter.QueryPath;
import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.io.sstable.Descriptor;
import org.apache.cassandra.io.sstable.SSTableWriter;
import org.codehaus.jackson.type.TypeReference;
@@ -209,7 +208,7 @@ public class SSTableImport
Map<?, ?> data = parser.readValueAs(new TypeReference<Map<?, ?>>() {});
keyCountToImport = (keyCountToImport == null) ? data.size() : keyCountToImport;
- SSTableWriter writer = new SSTableWriter(ssTablePath, keyCountToImport, DatabaseDescriptor.getCFMetaData(Descriptor.fromFilename(ssTablePath)), partitioner);
+ SSTableWriter writer = new SSTableWriter(ssTablePath, keyCountToImport);
System.out.printf("Importing %s keys...%n", keyCountToImport);
@@ -281,7 +280,7 @@ public class SSTableImport
System.out.printf("Importing %s keys...%n", keyCountToImport);
parser = getParser(jsonFile); // renewing parser
- SSTableWriter writer = new SSTableWriter(ssTablePath, keyCountToImport, DatabaseDescriptor.getCFMetaData(Descriptor.fromFilename(ssTablePath)), partitioner);
+ SSTableWriter writer = new SSTableWriter(ssTablePath, keyCountToImport);
int lineNumber = 1;
DecoratedKey prevStoredKey = null;