Author: xedin
Date: Tue Aug 16 21:31:38 2011
New Revision: 1158452

URL: http://svn.apache.org/viewvc?rev=1158452&view=rev
Log:
Refactor and optimize ColumnFamilyStore.files(...) and 
Descriptor.fromFilename(String) and few other places responsible for work with 
SSTable files
patch by Pavel Yaskevich; reviewed by Jonathan Ellis for CASSANDRA-3040

Modified:
    cassandra/trunk/CHANGES.txt
    cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
    cassandra/trunk/src/java/org/apache/cassandra/io/sstable/Descriptor.java
    cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTable.java

Modified: cassandra/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/CHANGES.txt?rev=1158452&r1=1158451&r2=1158452&view=diff
==============================================================================
--- cassandra/trunk/CHANGES.txt (original)
+++ cassandra/trunk/CHANGES.txt Tue Aug 16 21:31:38 2011
@@ -33,6 +33,8 @@
  * make CL > ONE reads wait for digest reconciliation before returning
    (CASSANDRA-2494)
  * fix missing logging for some exceptions (CASSANDRA-2061)
+ * refactor and optimize ColumnFamilyStore.files(...) and 
Descriptor.fromFilename(String)
+   and few other places responsible for work with SSTable files 
(CASSANDRA-3040)
 
 0.8.5
  * fix NPE when encryption_options is unspecified (CASSANDRA-3007)

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=1158452&r1=1158451&r2=1158452&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Tue 
Aug 16 21:31:38 2011
@@ -32,7 +32,6 @@ import javax.management.MBeanServer;
 import javax.management.ObjectName;
 
 import com.google.common.collect.Iterables;
-import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -54,19 +53,13 @@ import org.apache.cassandra.db.commitlog
 import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.filter.*;
-import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.db.index.SecondaryIndexManager;
 import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.BytesType;
-import org.apache.cassandra.db.marshal.LocalByPartionerType;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.thrift.IndexClause;
-import org.apache.cassandra.thrift.IndexExpression;
-import org.apache.cassandra.thrift.IndexOperator;
-import org.apache.cassandra.thrift.IndexType;
 import org.apache.cassandra.utils.*;
 import org.cliffc.high_scale_lib.NonBlockingHashMap;
 
@@ -335,10 +328,10 @@ public class ColumnFamilyStore implement
         List<Integer> generations = new ArrayList<Integer>();
         for (String path : 
DatabaseDescriptor.getAllDataFileLocationsForTable(table.name))
         {
-            Iterable<Pair<Descriptor, Component>> pairs = files(new 
File(path));
+            Iterable<Pair<Descriptor, Component>> pairs = files(new 
File(path), columnFamily);
             File incrementalsPath = new File(path, "backups");
             if (incrementalsPath.exists())
-                pairs = Iterables.concat(pairs, files(incrementalsPath));
+                pairs = Iterables.concat(pairs, files(incrementalsPath, 
columnFamily));
 
             for (Pair<Descriptor, Component> pair : pairs)
             {
@@ -471,9 +464,9 @@ public class ColumnFamilyStore implement
         final Map<Descriptor,Set<Component>> sstables = new 
HashMap<Descriptor,Set<Component>>();
         for (String directory : 
DatabaseDescriptor.getAllDataFileLocationsForTable(keyspace))
         {
-            for (Pair<Descriptor, Component> component : files(new 
File(directory)))
+            for (Pair<Descriptor, Component> component : files(new 
File(directory), columnFamily))
             {
-                if (component != null && 
component.left.cfname.equals(columnFamily))
+                if (component != null)
                 {
                     if ((includeCompacted || !new 
File(component.left.filenameFor(Component.COMPACTED_MARKER)).exists())
                      && (includeTemporary || !component.left.temporary))
@@ -494,15 +487,19 @@ public class ColumnFamilyStore implement
         return sstables;
     }
 
-    private static List<Pair<Descriptor, Component>> files(File path)
+    private static List<Pair<Descriptor, Component>> files(File path, final 
String columnFamilyName)
     {
         final List<Pair<Descriptor, Component>> sstables = new 
ArrayList<Pair<Descriptor, Component>>();
         // NB: we never "accept" a file in the FilenameFilter sense: they are 
added to the sstable map
-        path.list(new FilenameFilter()
+        path.listFiles(new FileFilter()
         {
-            public boolean accept(File dir, String name)
+            public boolean accept(File file)
             {
-                Pair<Descriptor, Component> pair = 
SSTable.tryComponentFromFilename(dir, name);
+                // we are only interested in the SSTable files that belong to 
the specific ColumnFamily
+                if (file.isDirectory() || 
!file.getName().startsWith(columnFamilyName))
+                    return false;
+
+                Pair<Descriptor, Component> pair = 
SSTable.tryComponentFromFilename(file.getParentFile(), file.getName());
 
                 if (pair != null)
                     sstables.add(pair);
@@ -510,6 +507,7 @@ public class ColumnFamilyStore implement
                 return false;
             }
         });
+
         return sstables;
     }
 

Modified: 
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=1158452&r1=1158451&r2=1158452&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/io/sstable/Descriptor.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/io/sstable/Descriptor.java 
Tue Aug 16 21:31:38 2011
@@ -139,18 +139,22 @@ public class Descriptor
 
     /**
      * @see #fromFilename(File directory, String name)
+     * @param filename The SSTable filename
+     * @return Descriptor of the SSTable initialized from 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());
-        return fromFilename(directory, name).left;
+        File file = new File(filename);
+        assert file.getParentFile() != null : "Filename must include parent 
directory.";
+        return fromFilename(file.getParentFile(), file.getName()).left;
     }
 
     /**
      * Filename of the form 
"<ksname>/<cfname>-[tmp-][<version>-]<gen>-<component>"
+     *
+     * @param directory The directory of the SSTable files
+     * @param name The name of the SSTable file
+     *
      * @return A Descriptor for the SSTable, and the Component remainder.
      */
     public static Pair<Descriptor,String> fromFilename(File directory, String 
name)
@@ -160,7 +164,7 @@ public class Descriptor
 
         // tokenize the filename
         StringTokenizer st = new StringTokenizer(name, "-");
-        String nexttok = null;
+        String nexttok;
 
         // all filenames must start with a column family
         String cfname = st.nextToken();
@@ -199,8 +203,8 @@ public class Descriptor
      * @param directory a directory containing SSTables
      * @return the keyspace name
      */
-    public static String extractKeyspaceName(File directory) {
-
+    public static String extractKeyspaceName(File directory)
+    {
         if (isSnapshotInPath(directory))
         {
             // We need to move backwards. If this is a snapshot, first parent 
takes us to:
@@ -211,9 +215,11 @@ public class Descriptor
     }
 
     /**
+     * @param directory The directory to check
      * @return <code>TRUE</code> if this directory represents a snapshot 
directory. <code>FALSE</code> otherwise.
      */
-    private static boolean isSnapshotInPath(File directory) {
+    private static boolean isSnapshotInPath(File directory)
+    {
         File curDirectory = directory;
         while (curDirectory != null)
         {
@@ -227,6 +233,7 @@ public class Descriptor
     }
 
     /**
+     * @param temporary temporary flag
      * @return A clone of this descriptor with the given 'temporary' status.
      */
     public Descriptor asTemporary(boolean temporary)
@@ -235,6 +242,7 @@ public class Descriptor
     }
 
     /**
+     * @param ver SSTable version
      * @return True if the given version string is not empty, and
      * contains all lowercase letters, as defined by java.lang.Character.
      */

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=1158452&r1=1158451&r2=1158452&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 Tue 
Aug 16 21:31:38 2011
@@ -20,7 +20,7 @@
 package org.apache.cassandra.io.sstable;
 
 import java.io.File;
-import java.io.FilenameFilter;
+import java.io.FileFilter;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.HashSet;
@@ -158,16 +158,23 @@ public abstract class SSTable
     static Set<Component> componentsFor(final Descriptor desc, final 
Descriptor.TempState matchState)
     {
         final Set<Component> components = new HashSet<Component>();
-        desc.directory.list(new FilenameFilter()
+
+        desc.directory.listFiles(new FileFilter()
         {
-            public boolean accept(File dir, String name)
+            public boolean accept(File file)
             {
-                Pair<Descriptor,Component> component = 
tryComponentFromFilename(dir, name);
+                if (file.isDirectory() || 
!file.getName().startsWith(desc.cfname))
+                    return false;
+
+                Pair<Descriptor, Component> component = 
tryComponentFromFilename(file.getParentFile(), file.getName());
+
                 if (component != null && component.left.equals(desc) && 
(matchState.isMatch(component.left)))
                     components.add(component.right);
+
                 return false;
             }
         });
+
         return components;
     }
 


Reply via email to