Add getsstables command to nodetool. Patch by Brandon Williams, reviewed by 
yukim for CASSANDRA-4199


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/5cda764a
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/5cda764a
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/5cda764a

Branch: refs/heads/trunk
Commit: 5cda764a9f956c35feba0a4cf3138037fcf39a76
Parents: 3fdc86c
Author: Yuki Morishita <yu...@apache.org>
Authored: Wed May 30 14:39:20 2012 -0500
Committer: Yuki Morishita <mor.y...@gmail.com>
Committed: Wed May 30 14:45:34 2012 -0500

----------------------------------------------------------------------
 CHANGES.txt                                        |    1 +
 .../org/apache/cassandra/db/ColumnFamilyStore.java |   20 ++++++++++++
 .../cassandra/db/ColumnFamilyStoreMBean.java       |    7 ++++
 .../apache/cassandra/io/sstable/SSTableReader.java |   24 ++++++++++----
 src/java/org/apache/cassandra/tools/NodeCmd.java   |   16 ++++++++++
 src/java/org/apache/cassandra/tools/NodeProbe.java |    6 ++++
 6 files changed, 67 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5cda764a/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 6e8aa8f..7ecfa9d 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 1.1.1-dev
+ * add getsstables command to nodetool (CASSANDRA-4199)
  * apply parent CF compaction settings to secondary index CFs (CASSANDRA-4280)
  * preserve commitlog size cap when recycling segments at startup
    (CASSANDRA-4201)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5cda764a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java 
b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 6aef6ab..2ae7e6e 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -1295,6 +1295,26 @@ public class ColumnFamilyStore implements 
ColumnFamilyStoreMBean
         return new ViewFragment(sstables, 
Iterables.concat(Collections.singleton(view.memtable), 
view.memtablesPendingFlush));
     }
 
+    public List<String> getSSTablesForKey(String key)
+    {
+        DecoratedKey dk = new 
DecoratedKey(partitioner.getToken(ByteBuffer.wrap(key.getBytes())), 
ByteBuffer.wrap(key.getBytes()));
+        ViewFragment view = markReferenced(dk);
+        try
+        {
+            List<String> files = new ArrayList<String>();
+            for (SSTableReader sstr : view.sstables)
+            {
+                // check if the key actually exists in this sstable, without 
updating cache and stats
+                if (sstr.getPosition(dk, SSTableReader.Operator.EQ, false) > 
-1)
+                    files.add(sstr.getFilename());
+            }
+            return files;
+        }
+        finally {
+            SSTableReader.releaseReferences(view.sstables);
+        }
+    }
+
     private ColumnFamily getTopLevelColumns(QueryFilter filter, int gcBefore, 
boolean forCache)
     {
         CollationController controller = new CollationController(this, 
forCache, filter, gcBefore);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5cda764a/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java 
b/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java
index d8243c5..68448c9 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java
@@ -227,6 +227,13 @@ public interface ColumnFamilyStoreMBean
     public List<String> getBuiltIndexes();
 
     /**
+     * Returns a list of filenames that contain the given key on this node
+     * @param key
+     * @return list of filenames containing the key
+     */
+    public List<String> getSSTablesForKey(String key);
+
+    /**
      * Scan through Keyspace/ColumnFamily's data directory
      * determine which SSTables should be loaded and load them
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5cda764a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java 
b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
index 98058e8..7d4d304 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
@@ -648,12 +648,22 @@ public class SSTableReader extends SSTable
     }
 
     /**
+     * Get position updating key cache and stats.
+     * @see #getPosition(org.apache.cassandra.db.RowPosition, 
org.apache.cassandra.io.sstable.SSTableReader.Operator, boolean)
+     */
+    public long getPosition(RowPosition key, Operator op)
+    {
+        return getPosition(key, op, true);
+    }
+
+    /**
      * @param key The key to apply as the rhs to the given Operator. A 'fake' 
key is allowed to
      * allow key selection by token bounds but only if op != * EQ
      * @param op The Operator defining matching keys: the nearest key to the 
target matching the operator wins.
+     * @param updateCacheAndStats true if updating stats and cache
      * @return The position in the data file to find the key, or -1 if the key 
is not present
      */
-    public long getPosition(RowPosition key, Operator op)
+    public long getPosition(RowPosition key, Operator op, boolean 
updateCacheAndStats)
     {
         // first, check bloom filter
         if (op == Operator.EQ)
@@ -667,7 +677,7 @@ public class SSTableReader extends SSTable
         if ((op == Operator.EQ || op == Operator.GE) && (key instanceof 
DecoratedKey))
         {
             DecoratedKey decoratedKey = (DecoratedKey)key;
-            Long cachedPosition = getCachedPosition(new 
KeyCacheKey(descriptor, decoratedKey.key), true);
+            Long cachedPosition = getCachedPosition(new 
KeyCacheKey(descriptor, decoratedKey.key), updateCacheAndStats);
             if (cachedPosition != null)
                 return cachedPosition;
         }
@@ -676,7 +686,7 @@ public class SSTableReader extends SSTable
         long sampledPosition = getIndexScanPosition(key);
         if (sampledPosition == -1)
         {
-            if (op == Operator.EQ)
+            if (op == Operator.EQ && updateCacheAndStats)
                 bloomFilterTracker.addFalsePositive();
             // we matched the -1th position: if the operator might match 
forward, return the 0th position
             return op.apply(1) >= 0 ? 0 : -1;
@@ -699,20 +709,20 @@ public class SSTableReader extends SSTable
                     int v = op.apply(comparison);
                     if (v == 0)
                     {
-                        if (comparison == 0 && keyCache != null && 
keyCache.getCapacity() > 0)
+                        if (comparison == 0 && keyCache != null && 
keyCache.getCapacity() > 0 && updateCacheAndStats)
                         {
                             assert key instanceof DecoratedKey; // key can be 
== to the index key only if it's a true row key
                             DecoratedKey decoratedKey = (DecoratedKey)key;
                             // store exact match for the key
                             cacheKey(decoratedKey, dataPosition);
                         }
-                        if (op == Operator.EQ)
+                        if (op == Operator.EQ && updateCacheAndStats)
                             bloomFilterTracker.addTruePositive();
                         return dataPosition;
                     }
                     if (v < 0)
                     {
-                        if (op == Operator.EQ)
+                        if (op == Operator.EQ && updateCacheAndStats)
                             bloomFilterTracker.addFalsePositive();
                         return -1;
                     }
@@ -729,7 +739,7 @@ public class SSTableReader extends SSTable
             }
         }
 
-        if (op == Operator.EQ)
+        if (op == Operator.EQ && updateCacheAndStats)
             bloomFilterTracker.addFalsePositive();
         return -1;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5cda764a/src/java/org/apache/cassandra/tools/NodeCmd.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeCmd.java 
b/src/java/org/apache/cassandra/tools/NodeCmd.java
index be813ed..a8d3f55 100644
--- a/src/java/org/apache/cassandra/tools/NodeCmd.java
+++ b/src/java/org/apache/cassandra/tools/NodeCmd.java
@@ -99,6 +99,7 @@ public class NodeCmd
         FLUSH,
         GETCOMPACTIONTHRESHOLD,
         GETENDPOINTS,
+        GETSSTABLES,
         GOSSIPINFO,
         INFO,
         INVALIDATEKEYCACHE,
@@ -184,6 +185,7 @@ public class NodeCmd
 
         // Three args
         addCmdHelp(header, "getendpoints <keyspace> <cf> <key>", "Print the 
end points that owns the key");
+        addCmdHelp(header, "getsstables <keyspace> <cf> <key>", "Print the 
sstable filenames that own the key");
 
         // Four args
         addCmdHelp(header, "setcachecapacity <keyspace> <cfname> 
<keycachecapacity> <rowcachecapacity>", "Set the key and row cache capacities 
of a given column family");
@@ -616,6 +618,15 @@ public class NodeCmd
         }
     }
 
+    private void printSSTables(String keyspace, String cf, String key, 
PrintStream output)
+    {
+        List<String> sstables = this.probe.getSSTables(keyspace, cf, key);
+        for (String sstable : sstables)
+        {
+            output.println(sstable);
+        }
+    }
+
     private void printIsThriftServerRunning(PrintStream outs)
     {
         outs.println(probe.isThriftServerRunning() ? "running" : "not 
running");
@@ -829,6 +840,11 @@ public class NodeCmd
                     nodeCmd.printEndPoints(arguments[0], arguments[1], 
arguments[2], System.out);
                     break;
 
+                case GETSSTABLES:
+                    if (arguments.length != 3) { badUse("getsstables requires 
ks, cf and key args"); }
+                    nodeCmd.printSSTables(arguments[0], arguments[1], 
arguments[2], System.out);
+                    break;
+
                 case REFRESH:
                     if (arguments.length != 2) { badUse("load_new_sstables 
requires ks and cf args"); }
                     probe.loadNewSSTables(arguments[0], arguments[1]);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5cda764a/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java 
b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 1d74a6b..d1a615d 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -464,6 +464,12 @@ public class NodeProbe
         return ssProxy.getNaturalEndpoints(keyspace, cf, key);
     }
 
+    public List<String> getSSTables(String keyspace, String cf, String key)
+    {
+        ColumnFamilyStoreMBean cfsProxy = getCfsProxy(keyspace, cf);
+        return cfsProxy.getSSTablesForKey(key);
+    }
+
     public Set<InetAddress> getStreamDestinations()
     {
         return streamProxy.getStreamDestinations();

Reply via email to