Author: jbellis
Date: Wed Sep 2 20:30:43 2009
New Revision: 810695
URL: http://svn.apache.org/viewvc?rev=810695&view=rev
Log:
clarify open/get code and r/m unnecessary synchronization; it's confusing.
(openedFiles is backed by a NonBlockingHashMap and all our uses of it are
threadsafe w/o extra locking)
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=810695&r1=810694&r2=810695&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
(original)
+++
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
Wed Sep 2 20:30:43 2009
@@ -852,6 +852,7 @@
if (logger_.isDebugEnabled())
logger_.debug("New file : " + newfile + " of size " + new
File(newfile).length());
assert newfile != null;
+ // TODO convert this to SSTableWriter.renameAndOpen
ssTables_.put(newfile, SSTableReader.open(newfile));
}
SSTableReader.get(file).delete();
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java?rev=810695&r1=810694&r2=810695&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java
(original)
+++
incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java
Wed Sep 2 20:30:43 2009
@@ -32,6 +32,10 @@
import org.apache.cassandra.db.marshal.AbstractType;
import org.cliffc.high_scale_lib.NonBlockingHashMap;
+/**
+ * SSTableReaders are open()ed by Table.onStart; after that they are created
by SSTableWriter.renameAndOpen.
+ * Do not use open() on existing SSTable files; use the references kept by
ColumnFamilyStore post-start instead.
+ */
public class SSTableReader extends SSTable
{
private static final Logger logger = Logger.getLogger(SSTableReader.class);
@@ -80,31 +84,28 @@
return indexedKeys;
}
- public static synchronized SSTableReader open(String dataFileName) throws
IOException
+ public static SSTableReader open(String dataFileName) throws IOException
{
return open(dataFileName, StorageService.getPartitioner());
}
-
+
public static synchronized SSTableReader open(String dataFileName,
IPartitioner partitioner) throws IOException
{
- SSTableReader sstable = openedFiles.get(dataFileName);
- if (sstable == null)
- {
- assert partitioner != null;
- sstable = new SSTableReader(dataFileName, partitioner);
+ assert partitioner != null;
+ assert openedFiles.get(dataFileName) == null;
- long start = System.currentTimeMillis();
- sstable.loadIndexFile();
- sstable.loadBloomFilter();
- if (logger.isDebugEnabled())
- logger.debug("INDEX LOAD TIME for " + dataFileName + ": " +
(System.currentTimeMillis() - start) + " ms.");
+ long start = System.currentTimeMillis();
+ SSTableReader sstable = new SSTableReader(dataFileName, partitioner);
+ sstable.loadIndexFile();
+ sstable.loadBloomFilter();
+ if (logger.isDebugEnabled())
+ logger.debug("INDEX LOAD TIME for " + dataFileName + ": " +
(System.currentTimeMillis() - start) + " ms.");
- openedFiles.put(dataFileName, sstable);
- }
return sstable;
}
- public static synchronized SSTableReader get(String dataFileName)
+ @Deprecated // move away from get() towards using the SSTR objects CFS
knows about
+ public static SSTableReader get(String dataFileName)
{
SSTableReader sstable = openedFiles.get(dataFileName);
assert sstable != null;
@@ -116,10 +117,7 @@
super(filename, partitioner);
this.indexPositions = indexPositions;
this.bf = bloomFilter;
- synchronized (SSTableReader.this)
- {
- openedFiles.put(filename, this);
- }
+ openedFiles.put(filename, this);
}
private SSTableReader(String filename, IPartitioner partitioner)