Author: jbellis
Date: Tue Nov  1 03:49:33 2011
New Revision: 1195801

URL: http://svn.apache.org/viewvc?rev=1195801&view=rev
Log:
cleanup and fixes for index debug logging

Modified:
    cassandra/trunk/src/java/org/apache/cassandra/config/CFMetaData.java
    cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
    cassandra/trunk/src/java/org/apache/cassandra/db/Table.java
    cassandra/trunk/src/java/org/apache/cassandra/db/index/keys/KeysIndex.java
    
cassandra/trunk/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java
    cassandra/trunk/test/unit/org/apache/cassandra/SchemaLoader.java
    
cassandra/trunk/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java

Modified: cassandra/trunk/src/java/org/apache/cassandra/config/CFMetaData.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/config/CFMetaData.java?rev=1195801&r1=1195800&r2=1195801&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/config/CFMetaData.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/config/CFMetaData.java Tue 
Nov  1 03:49:33 2011
@@ -232,6 +232,7 @@ public final class CFMetaData
     public static CFMetaData newIndexMetadata(CFMetaData parent, 
ColumnDefinition info, AbstractType columnComparator)
     {
         return new CFMetaData(parent.ksName, 
parent.indexColumnFamilyName(info), ColumnFamilyType.Standard, 
columnComparator, null)
+                             .keyValidator(info.getValidator())
                              .keyCacheSize(0.0)
                              .readRepairChance(0.0)
                              .gcGraceSeconds(parent.gcGraceSeconds)

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=1195801&r1=1195800&r2=1195801&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Tue 
Nov  1 03:49:33 2011
@@ -1533,9 +1533,10 @@ public class ColumnFamilyStore implement
      * For testing.  no effort is made to clear historical memtables, nor for
      * thread safety
      */
-    void clearUnsafe()
+    public void clearUnsafe()
     {
-        data.init();
+        for (ColumnFamilyStore cfs : concatWithIndexes())
+            cfs.data.init();
     }
 
     /**

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/Table.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/Table.java?rev=1195801&r1=1195800&r2=1195801&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/Table.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/Table.java Tue Nov  1 
03:49:33 2011
@@ -512,7 +512,9 @@ public class Table
 
     public static void indexRow(DecoratedKey<?> key, ColumnFamilyStore cfs, 
SortedSet<ByteBuffer> indexedColumns)
     {
-        logger.debug("Indexing row {} ", key);
+        if (logger.isDebugEnabled())
+            logger.debug("Indexing row {} ", 
cfs.metadata.getKeyValidator().getString(key.key));
+
         switchLock.readLock().lock();
         try
         {

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/db/index/keys/KeysIndex.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/index/keys/KeysIndex.java?rev=1195801&r1=1195800&r2=1195801&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/index/keys/KeysIndex.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/index/keys/KeysIndex.java 
Tue Nov  1 03:49:33 2011
@@ -56,19 +56,19 @@ public class KeysIndex extends PerColumn
         assert baseCfs != null && columnDefs != null;
 
         ColumnDefinition columnDef = columnDefs.iterator().next();
-        CFMetaData indexedCfMetadata = 
CFMetaData.newIndexMetadata(baseCfs.metadata, columnDef,indexComparator());
+        CFMetaData indexedCfMetadata = 
CFMetaData.newIndexMetadata(baseCfs.metadata, columnDef, indexComparator());
         indexedCfs = ColumnFamilyStore.createColumnFamilyStore(baseCfs.table,
-                                                               
indexedCfMetadata.cfName,
-                                                               new 
LocalPartitioner(columnDef.getValidator()),
-                                                               
indexedCfMetadata);
+                                                             
indexedCfMetadata.cfName,
+                                                             new 
LocalPartitioner(columnDef.getValidator()),
+                                                             
indexedCfMetadata);
     }
 
     public static AbstractType indexComparator()
     {
         IPartitioner rowPartitioner = StorageService.getPartitioner();
         return (rowPartitioner instanceof OrderPreservingPartitioner || 
rowPartitioner instanceof ByteOrderedPartitioner)
-                                        ? BytesType.instance
-                                        : new 
LocalByPartionerType(StorageService.getPartitioner());
+               ? BytesType.instance
+               : new LocalByPartionerType(StorageService.getPartitioner());
     }
 
     @Override
@@ -99,7 +99,7 @@ public class KeysIndex extends PerColumn
             cfi.addColumn(new Column(rowKey, ByteBufferUtil.EMPTY_BYTE_BUFFER, 
column.timestamp()));
         }
         if (logger.isDebugEnabled())
-            logger.debug("applying index row {}:{}", valueKey, cfi);
+            logger.debug("applying index row {} in {}", 
indexedCfs.metadata.getKeyValidator().getString(valueKey.key), cfi);
         
         indexedCfs.apply(valueKey, cfi);
     }

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java?rev=1195801&r1=1195800&r2=1195801&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java 
(original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java 
Tue Nov  1 03:49:33 2011
@@ -149,7 +149,7 @@ public class KeysSearcher extends Second
              * should be pretty close to `start_key`. */
             if (logger.isDebugEnabled())
                 logger.debug(String.format("Scanning index %s starting with 
%s",
-                                           expressionString(primary), 
index.getUnderlyingCfs().getComparator().getString(startKey)));
+                                           expressionString(primary), 
index.getBaseCFStore().metadata.getKeyValidator().getString(startKey)));
 
             // We shouldn't fetch only 1 row as this provides buggy paging in 
case the first row doesn't satisfy all clauses
             int count = Math.max(clause.count, 2);

Modified: cassandra/trunk/test/unit/org/apache/cassandra/SchemaLoader.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/SchemaLoader.java?rev=1195801&r1=1195800&r2=1195801&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/SchemaLoader.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/SchemaLoader.java Tue Nov  1 
03:49:33 2011
@@ -276,8 +276,9 @@ public class SchemaLoader
     private static CFMetaData indexCFMD(String ksName, String cfName, final 
Boolean withIdxType) throws ConfigurationException
     {
         return standardCFMD(ksName, cfName)
-                .columnMetadata(new HashMap<ByteBuffer, ColumnDefinition>()
-                    {{
+               .keyValidator(AsciiType.instance)
+               .columnMetadata(new HashMap<ByteBuffer, ColumnDefinition>()
+                   {{
                         ByteBuffer cName = 
ByteBuffer.wrap("birthdate".getBytes(Charsets.UTF_8));
                         IndexType keys = withIdxType ? IndexType.KEYS : null;
                         put(cName, new ColumnDefinition(cName, 
LongType.instance, keys, null, ByteBufferUtil.bytesToHex(cName)));

Modified: 
cassandra/trunk/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java?rev=1195801&r1=1195800&r2=1195801&view=diff
==============================================================================
--- 
cassandra/trunk/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
 (original)
+++ 
cassandra/trunk/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
 Tue Nov  1 03:49:33 2011
@@ -47,10 +47,15 @@ import org.apache.cassandra.utils.NodeId
 
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class StreamingTransferTest extends CleanupHelper
 {
+    private static final Logger logger = 
LoggerFactory.getLogger(StreamingTransferTest.class);
+
     public static final InetAddress LOCAL = FBUtilities.getBroadcastAddress();
 
     @BeforeClass
@@ -66,6 +71,7 @@ public class StreamingTransferTest exten
     private List<String> createAndTransfer(Table table, ColumnFamilyStore cfs, 
Mutator mutator) throws Exception
     {
         // write a temporary SSTable, and unregister it
+        logger.debug("Mutating " + cfs.columnFamily);
         long timestamp = 1234;
         for (int i = 1; i <= 3; i++)
             mutator.mutate("key" + i, "col" + i, timestamp);
@@ -78,6 +84,7 @@ public class StreamingTransferTest exten
         cfs.unreferenceSSTables();
 
         // transfer the first and last key
+        logger.debug("Transferring " + cfs.columnFamily);
         int[] offs = new int[]{1, 3};
         IPartitioner p = StorageService.getPartitioner();
         List<Range> ranges = new ArrayList<Range>();
@@ -109,6 +116,8 @@ public class StreamingTransferTest exten
         List<String> keys = new ArrayList<String>();
         for (int off : offs)
             keys.add("key" + off);
+
+        logger.debug("... everything looks good for " + cfs.columnFamily);
         return keys;
     }
 
@@ -128,6 +137,7 @@ public class StreamingTransferTest exten
                 cf.addColumn(column(col, "v", timestamp));
                 cf.addColumn(new Column(ByteBufferUtil.bytes("birthdate"), 
ByteBufferUtil.bytes(val), timestamp));
                 rm.add(cf);
+                logger.debug("Applying row to transfer " + rm);
                 rm.apply();
             }
         });


Reply via email to