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 2009/09/03 17:11:06 UTC
svn commit: r810986 -
/incubator/cassandra/branches/cassandra-0.4/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
Author: jbellis
Date: Thu Sep 3 15:11:06 2009
New Revision: 810986
URL: http://svn.apache.org/viewvc?rev=810986&view=rev
Log:
Always generate sstable generation numbers by atomic increment instead of leaving 'holes' that compaction uses. Sometimes, bucketing is not perfect and the holes method will overwrite one of the being-compacted files. Since #223 it has not been important to keep compacted files lower in generation number than newly created ones so this is both simpler and more correct. Patch by Sammy Yu; reviewed by jbellis for CASSANDRA-418
Modified:
incubator/cassandra/branches/cassandra-0.4/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
Modified: incubator/cassandra/branches/cassandra-0.4/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.4/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=810986&r1=810985&r2=810986&view=diff
==============================================================================
--- incubator/cassandra/branches/cassandra-0.4/src/java/org/apache/cassandra/db/ColumnFamilyStore.java (original)
+++ incubator/cassandra/branches/cassandra-0.4/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Thu Sep 3 15:11:06 2009
@@ -340,8 +340,6 @@
String getNextFileName()
{
- // increment twice so that we do not generate consecutive numbers
- fileIndexGenerator_.incrementAndGet();
return String.format("%s-%s-Data.db", columnFamily_, fileIndexGenerator_.incrementAndGet());
}
@@ -365,31 +363,6 @@
columnFamily_, SSTable.TEMPFILE_MARKER, fileIndexGenerator_.incrementAndGet());
}
- /*
- * Return a temporary file name. Based on the list of files input
- * This fn sorts the list and generates a number between he 2 lowest filenames
- * ensuring uniqueness.
- * Since we do not generate consecutive numbers hence the lowest file number
- * can just be incremented to generate the next file.
- */
- String getTempFileName(List<String> files)
- {
- int lowestIndex;
- int index;
- Collections.sort(files, new FileNameComparator(FileNameComparator.Ascending));
-
- if (files.size() <= 1)
- {
- return null;
- }
- lowestIndex = getIndexFromFileName(files.get(0));
-
- index = lowestIndex + 1;
-
- return String.format("%s-%s-%s-Data.db",
- columnFamily_, SSTable.TEMPFILE_MARKER, index);
- }
-
void switchMemtable(Memtable oldMemtable, CommitLog.CommitLogContext ctx)
{
memtableLock_.writeLock().lock();
@@ -1071,7 +1044,7 @@
return 0;
}
- String mergedFileName = getTempFileName(files);
+ String mergedFileName = getTempSSTableFileName();
SSTableWriter writer = null;
SSTableReader ssTable = null;
String lastkey = null;