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 2010/06/18 06:19:28 UTC

svn commit: r955832 - in /cassandra/trunk: src/java/org/apache/cassandra/db/ src/java/org/apache/cassandra/io/sstable/ src/java/org/apache/cassandra/streaming/ test/unit/org/apache/cassandra/io/sstable/ test/unit/org/apache/cassandra/streaming/

Author: jbellis
Date: Fri Jun 18 04:19:27 2010
New Revision: 955832

URL: http://svn.apache.org/viewvc?rev=955832&view=rev
Log:
extract SSTable.Descriptor to top-level class.  patch by jbellis

Added:
    cassandra/trunk/src/java/org/apache/cassandra/io/sstable/Descriptor.java
Modified:
    cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
    cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTable.java
    cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableTracker.java
    cassandra/trunk/src/java/org/apache/cassandra/streaming/PendingFile.java
    cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamInitiateVerbHandler.java
    cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamOut.java
    cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
    cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
    cassandra/trunk/test/unit/org/apache/cassandra/streaming/BootstrapTest.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=955832&r1=955831&r2=955832&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Fri Jun 18 04:19:27 2010
@@ -33,7 +33,6 @@ import javax.management.MBeanServer;
 import javax.management.ObjectName;
 
 import org.apache.commons.collections.IteratorUtils;
-import org.apache.commons.lang.ArrayUtils;
 
 import com.google.common.collect.Iterables;
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
@@ -41,7 +40,6 @@ import org.apache.cassandra.concurrent.N
 import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.IClock;
 import org.apache.cassandra.db.IClock.ClockRelationship;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.commitlog.CommitLogSegment;
@@ -54,7 +52,6 @@ import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.thrift.SliceRange;
 import org.apache.cassandra.utils.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -307,12 +304,12 @@ public class ColumnFamilyStore implement
 
     private static String getColumnFamilyFromFileName(String filename)
     {
-        return SSTable.Descriptor.fromFilename(filename).cfname;
+        return Descriptor.fromFilename(filename).cfname;
     }
 
     public static int getGenerationFromFileName(String filename)
     {
-        return SSTable.Descriptor.fromFilename(filename).generation;
+        return Descriptor.fromFilename(filename).generation;
     }
 
     /*
@@ -331,7 +328,7 @@ public class ColumnFamilyStore implement
 
     public String getTempSSTablePath(String directory)
     {
-        SSTable.Descriptor desc = new SSTable.Descriptor(new File(directory),
+        Descriptor desc = new Descriptor(new File(directory),
                                                          table_,
                                                          columnFamily_,
                                                          fileIndexGenerator_.incrementAndGet(),

Added: cassandra/trunk/src/java/org/apache/cassandra/io/sstable/Descriptor.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/Descriptor.java?rev=955832&view=auto
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/io/sstable/Descriptor.java (added)
+++ cassandra/trunk/src/java/org/apache/cassandra/io/sstable/Descriptor.java Fri Jun 18 04:19:27 2010
@@ -0,0 +1,162 @@
+package org.apache.cassandra.io.sstable;
+
+import java.io.File;
+import java.util.StringTokenizer;
+
+import com.google.common.base.Objects;
+
+/**
+ * A SSTable is described by the keyspace and column family it contains data
+ * for, a generation (where higher generations contain more recent data) and
+ * an alphabetic version string.
+ *
+ * A descriptor can be marked as temporary, which influences generated filenames.
+ */
+public class Descriptor
+{
+    public static final String LEGACY_VERSION = "a";
+    public static final String CURRENT_VERSION = "d";
+
+    public final File directory;
+    public final String version;
+    public final String ksname;
+    public final String cfname;
+    public final int generation;
+    public final boolean temporary;
+    private final int hashCode;
+
+    /**
+     * A descriptor that assumes CURRENT_VERSION.
+     */
+    public Descriptor(File directory, String ksname, String cfname, int generation, boolean temp)
+    {
+        this(CURRENT_VERSION, directory, ksname, cfname, generation, temp);
+    }
+
+    public Descriptor(String version, File directory, String ksname, String cfname, int generation, boolean temp)
+    {
+        assert version != null && directory != null && ksname != null && cfname != null;
+        this.version = version;
+        this.directory = directory;
+        this.ksname = ksname;
+        this.cfname = cfname;
+        this.generation = generation;
+        temporary = temp;
+        hashCode = Objects.hashCode(directory, generation, ksname, cfname);
+    }
+
+    /**
+     * @param suffix A component suffix, such as 'Data.db'/'Index.db'/etc
+     * @return A filename for this descriptor with the given suffix.
+     */
+    public String filenameFor(String suffix)
+    {
+        StringBuilder buff = new StringBuilder();
+        buff.append(directory).append(File.separatorChar);
+        buff.append(cfname).append("-");
+        if (temporary)
+            buff.append(SSTable.TEMPFILE_MARKER).append("-");
+        if (!LEGACY_VERSION.equals(version))
+            buff.append(version).append("-");
+        buff.append(generation).append("-");
+        buff.append(suffix);
+        return buff.toString();
+    }
+
+    /**
+     * Filename of the form "<ksname>/<cfname>-[tmp-][<version>-]<gen>-*"
+     * @param filename A full SSTable filename, including the directory.
+     * @return A SSTable.Descriptor for the filename.
+     */
+    public static Descriptor fromFilename(String filename)
+    {
+        int separatorPos = filename.lastIndexOf(File.separatorChar);
+        assert separatorPos != -1 : "Filename must include parent directory.";
+        File directory = new File(filename.substring(0, separatorPos));
+        String name = filename.substring(separatorPos+1, filename.length());
+
+        // name of parent directory is keyspace name
+        String ksname = directory.getName();
+
+        // tokenize the filename
+        StringTokenizer st = new StringTokenizer(name, "-");
+        String nexttok = null;
+
+        // all filenames must start with a column family
+        String cfname = st.nextToken();
+
+        // optional temporary marker
+        nexttok = st.nextToken();
+        boolean temporary = false;
+        if (nexttok.equals(SSTable.TEMPFILE_MARKER))
+        {
+            temporary = true;
+            nexttok = st.nextToken();
+        }
+
+        // optional version string
+        String version = LEGACY_VERSION;
+        if (versionValidate(nexttok))
+        {
+            version = nexttok;
+            nexttok = st.nextToken();
+        }
+        int generation = Integer.parseInt(nexttok);
+
+        return new Descriptor(version, directory, ksname, cfname, generation, temporary);
+    }
+
+    /**
+     * @return A clone of this descriptor with the given 'temporary' status.
+     */
+    public Descriptor asTemporary(boolean temporary)
+    {
+        return new Descriptor(version, directory, ksname, cfname, generation, temporary);
+    }
+
+    /**
+     * @return True if the given version string is not empty, and
+     * contains all lowercase letters, as defined by java.lang.Character.
+     */
+    private static boolean versionValidate(String ver)
+    {
+        if (ver.length() < 1) return false;
+        for (char ch : ver.toCharArray())
+            if (!Character.isLetter(ch) || !Character.isLowerCase(ch))
+                return false;
+        return true;
+    }
+
+    @Override
+    public String toString()
+    {
+        return this.filenameFor("<>");
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (o == this)
+            return true;
+        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);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return hashCode;
+    }
+
+    public boolean hasStringsInBloomFilter()
+    {
+        return version.compareTo("c") < 0;
+    }
+
+    public boolean hasIntRowSize()
+    {
+        return version.compareTo("d") < 0;
+    }
+}

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=955832&r1=955831&r2=955832&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 Fri Jun 18 04:19:27 2010
@@ -25,7 +25,6 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
 import java.util.Arrays;
-import java.util.StringTokenizer;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -34,8 +33,6 @@ import org.apache.cassandra.dht.IPartiti
 import org.apache.cassandra.utils.BloomFilter;
 import org.apache.cassandra.io.util.FileUtils;
 
-import com.google.common.base.Objects;
-
 /**
  * This class is built on top of the SequenceFile. It stores
  * data on disk in sorted fashion. However the sorting is upto
@@ -167,11 +164,6 @@ public abstract class SSTable
         return desc.ksname;
     }
 
-    public static String parseTableName(String filename)
-    {
-        return Descriptor.fromFilename(filename).ksname;        
-    }
-
     public static long getTotalBytes(Iterable<SSTableReader> sstables)
     {
         long sum = 0;
@@ -199,160 +191,4 @@ public abstract class SSTable
                "path='" + getFilename() + '\'' +
                ')';
     }
-
-    /**
-     * A SSTable is described by the keyspace and column family it contains data
-     * for, a generation (where higher generations contain more recent data) and
-     * an alphabetic version string.
-     *
-     * A descriptor can be marked as temporary, which influences generated filenames.
-     */
-    public static class Descriptor
-    {
-        public static final String LEGACY_VERSION = "a";
-        public static final String CURRENT_VERSION = "d";
-
-        public final File directory;
-        public final String version;
-        public final String ksname;
-        public final String cfname;
-        public final int generation;
-        public final boolean temporary;
-        private final int hashCode;
-
-        /**
-         * A descriptor that assumes CURRENT_VERSION.
-         */
-        public Descriptor(File directory, String ksname, String cfname, int generation, boolean temp)
-        {
-            this(CURRENT_VERSION, directory, ksname, cfname, generation, temp);
-        }
-
-        public Descriptor(String version, File directory, String ksname, String cfname, int generation, boolean temp)
-        {
-            assert version != null && directory != null && ksname != null && cfname != null;
-            this.version = version;
-            this.directory = directory;
-            this.ksname = ksname;
-            this.cfname = cfname;
-            this.generation = generation;
-            temporary = temp;
-            hashCode = Objects.hashCode(directory, generation, ksname, cfname);
-        }
-
-        /**
-         * @param suffix A component suffix, such as 'Data.db'/'Index.db'/etc
-         * @return A filename for this descriptor with the given suffix.
-         */
-        public String filenameFor(String suffix)
-        {
-            StringBuilder buff = new StringBuilder();
-            buff.append(directory).append(File.separatorChar);
-            buff.append(cfname).append("-");
-            if (temporary)
-                buff.append(TEMPFILE_MARKER).append("-");
-            if (!LEGACY_VERSION.equals(version))
-                buff.append(version).append("-");
-            buff.append(generation).append("-");
-            buff.append(suffix);
-            return buff.toString();
-        }
-
-        /**
-         * Filename of the form "<ksname>/<cfname>-[tmp-][<version>-]<gen>-*"
-         * @param filename A full SSTable filename, including the directory.
-         * @return A SSTable.Descriptor for the filename. 
-         */
-        public static Descriptor fromFilename(String filename)
-        {
-            int separatorPos = filename.lastIndexOf(File.separatorChar);
-            assert separatorPos != -1 : "Filename must include parent directory.";
-            File directory = new File(filename.substring(0, separatorPos));
-            String name = filename.substring(separatorPos+1, filename.length());
-
-            // name of parent directory is keyspace name
-            String ksname = directory.getName();
-
-            // tokenize the filename
-            StringTokenizer st = new StringTokenizer(name, "-");
-            String nexttok = null;
-            
-            // all filenames must start with a column family
-            String cfname = st.nextToken();
-
-            // optional temporary marker
-            nexttok = st.nextToken();
-            boolean temporary = false;
-            if (nexttok.equals(TEMPFILE_MARKER))
-            {
-                temporary = true;
-                nexttok = st.nextToken();
-            }
-
-            // optional version string
-            String version = LEGACY_VERSION;
-            if (versionValidate(nexttok))
-            {
-                version = nexttok;
-                nexttok = st.nextToken();
-            }
-            int generation = Integer.parseInt(nexttok);
-
-            return new Descriptor(version, directory, ksname, cfname, generation, temporary);
-        }
-        
-        /**
-         * @return A clone of this descriptor with the given 'temporary' status.
-         */
-        public Descriptor asTemporary(boolean temporary)
-        {
-            return new Descriptor(version, directory, ksname, cfname, generation, temporary);
-        }
-
-        /**
-         * @return True if the given version string is not empty, and
-         * contains all lowercase letters, as defined by java.lang.Character.
-         */
-        private static boolean versionValidate(String ver)
-        {
-            if (ver.length() < 1) return false;
-            for (char ch : ver.toCharArray())
-                if (!Character.isLetter(ch) || !Character.isLowerCase(ch))
-                    return false;
-            return true;
-        }
-
-        @Override
-        public String toString()
-        {
-            return this.filenameFor("<>");
-        }
-
-        @Override
-        public boolean equals(Object o)
-        {
-            if (o == this)
-                return true;
-            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);
-        }
-
-        @Override
-        public int hashCode()
-        {
-            return hashCode;
-        }
-
-        public boolean hasStringsInBloomFilter()
-        {
-            return version.compareTo("c") < 0;
-        }
-
-        public boolean hasIntRowSize()
-        {
-            return version.compareTo("d") < 0;
-        }
-    }
 }

Modified: cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableTracker.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableTracker.java?rev=955832&r1=955831&r2=955832&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableTracker.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableTracker.java Fri Jun 18 04:19:27 2010
@@ -42,7 +42,7 @@ public class SSTableTracker implements I
     private final String ksname;
     private final String cfname;
 
-    private final JMXInstrumentedCache<Pair<SSTable.Descriptor,DecoratedKey>,Long> keyCache;
+    private final JMXInstrumentedCache<Pair<Descriptor,DecoratedKey>,Long> keyCache;
     private final JMXInstrumentedCache<DecoratedKey, ColumnFamily> rowCache;
 
     public SSTableTracker(String ksname, String cfname)
@@ -50,7 +50,7 @@ public class SSTableTracker implements I
         this.ksname = ksname;
         this.cfname = cfname;
         sstables = Collections.emptySet();
-        keyCache = new JMXInstrumentedCache<Pair<SSTable.Descriptor,DecoratedKey>,Long>(ksname, cfname + "KeyCache", 0);
+        keyCache = new JMXInstrumentedCache<Pair<Descriptor,DecoratedKey>,Long>(ksname, cfname + "KeyCache", 0);
         rowCache = new JMXInstrumentedCache<DecoratedKey, ColumnFamily>(ksname, cfname + "RowCache", 0);
     }
 
@@ -176,7 +176,7 @@ public class SSTableTracker implements I
         totalSize.addAndGet(-size);
     }
 
-    public JMXInstrumentedCache<Pair<SSTable.Descriptor, DecoratedKey>, Long> getKeyCache()
+    public JMXInstrumentedCache<Pair<Descriptor, DecoratedKey>, Long> getKeyCache()
     {
         return keyCache;
     }

Modified: cassandra/trunk/src/java/org/apache/cassandra/streaming/PendingFile.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/streaming/PendingFile.java?rev=955832&r1=955831&r2=955832&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/streaming/PendingFile.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/streaming/PendingFile.java Fri Jun 18 04:19:27 2010
@@ -26,7 +26,7 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 
 import org.apache.cassandra.io.ICompactSerializer;
-import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.Descriptor;
 
 class PendingFile
 {
@@ -42,12 +42,12 @@ class PendingFile
         return serializer_;
     }
 
-    private SSTable.Descriptor desc;        
+    private Descriptor desc;
     private String component;
     private long expectedBytes;                     
     private long ptr;
 
-    public PendingFile(SSTable.Descriptor desc, String component, long expectedBytes)
+    public PendingFile(Descriptor desc, String component, long expectedBytes)
     {
         this.desc = desc;
         this.component = component;
@@ -70,7 +70,7 @@ class PendingFile
         return component;
     }
 
-    public SSTable.Descriptor getDescriptor()
+    public Descriptor getDescriptor()
     {
         return desc;
     }
@@ -115,7 +115,7 @@ class PendingFile
 
         public PendingFile deserialize(DataInputStream dis) throws IOException
         {
-            SSTable.Descriptor desc = SSTable.Descriptor.fromFilename(dis.readUTF());
+            Descriptor desc = Descriptor.fromFilename(dis.readUTF());
             String component = dis.readUTF();
             long expectedBytes = dis.readLong();           
             return new PendingFile(desc, component, expectedBytes);

Modified: cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamInitiateVerbHandler.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamInitiateVerbHandler.java?rev=955832&r1=955831&r2=955832&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamInitiateVerbHandler.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamInitiateVerbHandler.java Fri Jun 18 04:19:27 2010
@@ -26,15 +26,12 @@ import java.util.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Table;
-import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.net.IVerbHandler;
 import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.streaming.StreamInitiateMessage;
-import org.apache.cassandra.streaming.StreamInManager;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -106,18 +103,18 @@ public class StreamInitiateVerbHandler i
     {
         /* Create a local sstable for each remote sstable */
         LinkedHashMap<PendingFile, PendingFile> mapping = new LinkedHashMap<PendingFile, PendingFile>();
-        Map<SSTable.Descriptor, SSTable.Descriptor> sstables = new HashMap<SSTable.Descriptor, SSTable.Descriptor>();
+        Map<Descriptor, Descriptor> sstables = new HashMap<Descriptor, Descriptor>();
         for (PendingFile remote : remoteFiles)
         {
-            SSTable.Descriptor remotedesc = remote.getDescriptor();
-            SSTable.Descriptor localdesc = sstables.get(remotedesc);
+            Descriptor remotedesc = remote.getDescriptor();
+            Descriptor localdesc = sstables.get(remotedesc);
             if (localdesc == null)
             {
                 // new local sstable
                 Table table = Table.open(remotedesc.ksname);
                 ColumnFamilyStore cfStore = table.getColumnFamilyStore(remotedesc.cfname);
 
-                localdesc = SSTable.Descriptor.fromFilename(cfStore.getFlushPath());
+                localdesc = Descriptor.fromFilename(cfStore.getFlushPath());
                 sstables.put(remotedesc, localdesc);
             }
 

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=955832&r1=955831&r2=955832&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamOut.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamOut.java Fri Jun 18 04:19:27 2010
@@ -33,6 +33,7 @@ import org.apache.commons.lang.StringUti
 
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.db.Table;
+import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.SSTable;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.net.Message;
@@ -122,7 +123,7 @@ public class StreamOut
         {
             for (String component : SSTable.components)
             {
-                SSTable.Descriptor desc = sstable.getDescriptor();
+                Descriptor desc = sstable.getDescriptor();
                 long filelen = new File(desc.filenameFor(component)).length();
                 pendingFiles[i++] = new PendingFile(desc, component, filelen);
             }

Modified: cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java?rev=955832&r1=955831&r2=955832&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java Fri Jun 18 04:19:27 2010
@@ -25,16 +25,11 @@ import java.util.*;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
-import static org.junit.Assert.*;
 
 import org.apache.cassandra.CleanupHelper;
 import org.apache.cassandra.io.util.BufferedRandomAccessFile;
-import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.utils.FBUtilities;
 
-import com.google.common.base.Predicate;
-import com.google.common.base.Predicates;
-
 /**
  * Tests backwards compatibility for SSTables. Requires that older SSTables match up with the existing config file,
  * and currently only tests specific cases for specific upgrades.
@@ -66,10 +61,10 @@ public class LegacySSTableTest extends C
     /**
      * Get a descriptor for the legacy sstable at the given version.
      */
-    protected SSTable.Descriptor getDescriptor(String ver) throws IOException
+    protected Descriptor getDescriptor(String ver) throws IOException
     {
         File directory = new File(LEGACY_SSTABLE_ROOT + File.separator + ver + File.separator + KSNAME);
-        return new SSTable.Descriptor(ver, directory, KSNAME, CFNAME, 0, false);
+        return new Descriptor(ver, directory, KSNAME, CFNAME, 0, false);
     }
 
     /**

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=955832&r1=955831&r2=955832&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 Fri Jun 18 04:19:27 2010
@@ -52,7 +52,7 @@ public class SSTableUtils
         File tabledir = new File(tempdir, tablename);
         tabledir.mkdir();
         tabledir.deleteOnExit();
-        File datafile = new File(new SSTable.Descriptor(tabledir, tablename, cfname, 0,
+        File datafile = new File(new Descriptor(tabledir, tablename, cfname, 0,
                                                         false).filenameFor("Data.db"));
         assert datafile.createNewFile();
         datafile.deleteOnExit();

Modified: cassandra/trunk/test/unit/org/apache/cassandra/streaming/BootstrapTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/streaming/BootstrapTest.java?rev=955832&r1=955831&r2=955832&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/streaming/BootstrapTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/streaming/BootstrapTest.java Fri Jun 18 04:19:27 2010
@@ -19,16 +19,13 @@
 package org.apache.cassandra.streaming;
 
 import static junit.framework.Assert.assertEquals;
-import static org.junit.Assert.*;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.Descriptor;
 
 import org.junit.Test;
 
@@ -37,7 +34,7 @@ public class BootstrapTest extends Schem
     @Test
     public void testGetNewNames() throws IOException
     {
-        SSTable.Descriptor desc = SSTable.Descriptor.fromFilename(new File("Keyspace1", "Standard1-500-Data.db").toString());
+        Descriptor desc = Descriptor.fromFilename(new File("Keyspace1", "Standard1-500-Data.db").toString());
         PendingFile[] pendingFiles = new PendingFile[]{ new PendingFile(desc, "Data.db", 100),
                                                         new PendingFile(desc, "Index.db", 100),
                                                         new PendingFile(desc, "Filter.db", 100) };