Author: jbellis
Date: Thu Feb 24 02:20:05 2011
New Revision: 1074021
URL: http://svn.apache.org/viewvc?rev=1074021&view=rev
Log:
revert unreviewed changes to SSTW/Descriptor, take 2
Modified:
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/Descriptor.java
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
Modified:
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/Descriptor.java
URL:
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/Descriptor.java?rev=1074021&r1=1074020&r2=1074021&view=diff
==============================================================================
---
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/Descriptor.java
(original)
+++
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/Descriptor.java
Thu Feb 24 02:20:05 2011
@@ -199,11 +199,7 @@ public class Descriptor
if (!(o instanceof Descriptor))
return false;
Descriptor that = (Descriptor)o;
- return that.directory.equals(this.directory)
- && that.generation == this.generation
- && that.ksname.equals(this.ksname)
- && that.cfname.equals(this.cfname)
- && that.temporary == this.temporary;
+ return that.directory.equals(this.directory) && that.generation ==
this.generation && that.ksname.equals(this.ksname) &&
that.cfname.equals(this.cfname);
}
@Override
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=1074021&r1=1074020&r2=1074021&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
Thu Feb 24 02:20:05 2011
@@ -22,12 +22,9 @@ package org.apache.cassandra.io.sstable;
import java.io.*;
import java.nio.ByteBuffer;
import java.util.Arrays;
-import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
-import com.google.common.collect.Sets;
-
import org.apache.cassandra.utils.ByteBufferUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -207,10 +204,8 @@ public class SSTableWriter extends SSTab
Descriptor newdesc = tmpdesc.asTemporary(false);
try
{
- // do -Data last because -Data present should mean the sstable was
completely renamed before crash
- for (Component component : Sets.difference(components,
Collections.singleton(Component.DATA)))
+ for (Component component : components)
FBUtilities.renameWithConfirm(tmpdesc.filenameFor(component),
newdesc.filenameFor(component));
- FBUtilities.renameWithConfirm(tmpdesc.filenameFor(Component.DATA),
newdesc.filenameFor(Component.DATA));
}
catch (IOException e)
{