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 2011/09/08 03:58:43 UTC

svn commit: r1166499 - in /cassandra/trunk/src/java/org/apache/cassandra: db/compaction/ db/filter/ db/migration/ io/sstable/ locator/ net/ service/

Author: jbellis
Date: Thu Sep  8 01:58:42 2011
New Revision: 1166499

URL: http://svn.apache.org/viewvc?rev=1166499&view=rev
Log:
add generic wildcards
patch by Norman Maurer and Tupshin Harper for CASSANDRA-2247

Modified:
    cassandra/trunk/src/java/org/apache/cassandra/db/compaction/AbstractCompactedRow.java
    cassandra/trunk/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java
    cassandra/trunk/src/java/org/apache/cassandra/db/filter/IFilter.java
    cassandra/trunk/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java
    cassandra/trunk/src/java/org/apache/cassandra/db/filter/QueryFilter.java
    cassandra/trunk/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
    cassandra/trunk/src/java/org/apache/cassandra/db/migration/Migration.java
    cassandra/trunk/src/java/org/apache/cassandra/io/sstable/IndexSummary.java
    cassandra/trunk/src/java/org/apache/cassandra/io/sstable/KeyIterator.java
    cassandra/trunk/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java
    cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java
    cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java
    cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
    cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractEndpointSnitch.java
    cassandra/trunk/src/java/org/apache/cassandra/net/AsyncResult.java
    cassandra/trunk/src/java/org/apache/cassandra/net/Header.java
    cassandra/trunk/src/java/org/apache/cassandra/net/MessageDeliveryTask.java
    cassandra/trunk/src/java/org/apache/cassandra/service/AbstractRowResolver.java
    cassandra/trunk/src/java/org/apache/cassandra/service/DigestMismatchException.java
    cassandra/trunk/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
    cassandra/trunk/src/java/org/apache/cassandra/service/MigrationManager.java
    cassandra/trunk/src/java/org/apache/cassandra/service/RowRepairResolver.java

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/compaction/AbstractCompactedRow.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/compaction/AbstractCompactedRow.java?rev=1166499&r1=1166498&r2=1166499&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/compaction/AbstractCompactedRow.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/compaction/AbstractCompactedRow.java Thu Sep  8 01:58:42 2011
@@ -21,7 +21,6 @@ package org.apache.cassandra.db.compacti
  */
 
 
-import java.io.Closeable;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.security.MessageDigest;
@@ -35,9 +34,9 @@ import org.apache.cassandra.db.Decorated
  */
 public abstract class AbstractCompactedRow
 {
-    public final DecoratedKey key;
+    public final DecoratedKey<?> key;
 
-    public AbstractCompactedRow(DecoratedKey key)
+    public AbstractCompactedRow(DecoratedKey<?> key)
     {
         this.key = key;
     }

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java?rev=1166499&r1=1166498&r2=1166499&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java Thu Sep  8 01:58:42 2011
@@ -49,7 +49,7 @@ public class PrecompactedRow extends Abs
     private final int gcBefore;
 
     // For testing purposes
-    public PrecompactedRow(DecoratedKey key, ColumnFamily compacted)
+    public PrecompactedRow(DecoratedKey<?> key, ColumnFamily compacted)
     {
         super(key);
         this.compactedCf = compacted;
@@ -57,14 +57,14 @@ public class PrecompactedRow extends Abs
     }
 
     /** it is caller's responsibility to call removeDeleted + removeOldShards from the cf before calling this constructor */
-    public PrecompactedRow(DecoratedKey key, CompactionController controller, ColumnFamily cf)
+    public PrecompactedRow(DecoratedKey<?> key, CompactionController controller, ColumnFamily cf)
     {
         super(key);
         this.gcBefore = controller.gcBefore;
         compactedCf = cf;
     }
 
-    public static ColumnFamily removeDeletedAndOldShards(DecoratedKey key, CompactionController controller, ColumnFamily cf)
+    public static ColumnFamily removeDeletedAndOldShards(DecoratedKey<?> key, CompactionController controller, ColumnFamily cf)
     {
         return removeDeletedAndOldShards(controller.shouldPurge(key), controller, cf);
     }

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/filter/IFilter.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/filter/IFilter.java?rev=1166499&r1=1166498&r2=1166499&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/filter/IFilter.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/filter/IFilter.java Thu Sep  8 01:58:42 2011
@@ -23,7 +23,6 @@ package org.apache.cassandra.db.filter;
 import java.util.Comparator;
 import java.util.Iterator;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.IColumnIterator;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -43,7 +42,7 @@ public interface IFilter
      * returns an iterator that returns columns from the given memtable
      * matching the Filter criteria in sorted order.
      */
-    public abstract IColumnIterator getMemtableColumnIterator(ColumnFamily cf, DecoratedKey key, AbstractType comparator);
+    public abstract IColumnIterator getMemtableColumnIterator(ColumnFamily cf, DecoratedKey<?> key, AbstractType comparator);
 
     /**
      * Get an iterator that returns columns from the given SSTable using the opened file
@@ -52,13 +51,13 @@ public interface IFilter
      * @param file Already opened file data input, saves us opening another one
      * @param key The key of the row we are about to iterate over
      */
-    public abstract IColumnIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput file, DecoratedKey key);
+    public abstract IColumnIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput file, DecoratedKey<?> key);
 
     /**
      * returns an iterator that returns columns from the given SSTable
      * matching the Filter criteria in sorted order.
      */
-    public abstract IColumnIterator getSSTableColumnIterator(SSTableReader sstable, DecoratedKey key);
+    public abstract IColumnIterator getSSTableColumnIterator(SSTableReader sstable, DecoratedKey<?> key);
 
     /**
      * collects columns from reducedColumns into returnCF.  Termination is determined

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java?rev=1166499&r1=1166498&r2=1166499&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java Thu Sep  8 01:58:42 2011
@@ -50,17 +50,17 @@ public class NamesQueryFilter implements
         this(FBUtilities.singleton(column));
     }
 
-    public IColumnIterator getMemtableColumnIterator(ColumnFamily cf, DecoratedKey key, AbstractType comparator)
+    public IColumnIterator getMemtableColumnIterator(ColumnFamily cf, DecoratedKey<?> key, AbstractType comparator)
     {
         return Memtable.getNamesIterator(key, cf, this);
     }
 
-    public IColumnIterator getSSTableColumnIterator(SSTableReader sstable, DecoratedKey key)
+    public IColumnIterator getSSTableColumnIterator(SSTableReader sstable, DecoratedKey<?> key)
     {
         return new SSTableNamesIterator(sstable, key, columns);
     }
     
-    public IColumnIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput file, DecoratedKey key)
+    public IColumnIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput file, DecoratedKey<?> key)
     {
         return new SSTableNamesIterator(sstable, file, key, columns);
     }

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/filter/QueryFilter.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/filter/QueryFilter.java?rev=1166499&r1=1166498&r2=1166499&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/filter/QueryFilter.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/filter/QueryFilter.java Thu Sep  8 01:58:42 2011
@@ -40,14 +40,14 @@ import org.apache.cassandra.utils.MergeI
 
 public class QueryFilter
 {
-    private static Logger logger = LoggerFactory.getLogger(QueryFilter.class);
+    private static final Logger logger = LoggerFactory.getLogger(QueryFilter.class);
 
-    public final DecoratedKey key;
+    public final DecoratedKey<?> key;
     public final QueryPath path;
     public final IFilter filter;
     private final IFilter superFilter;
 
-    public QueryFilter(DecoratedKey key, QueryPath path, IFilter filter)
+    public QueryFilter(DecoratedKey<?> key, QueryPath path, IFilter filter)
     {
         this.key = key;
         this.path = path;
@@ -63,7 +63,7 @@ public class QueryFilter
         return getMemtableColumnIterator(cf, key, comparator);
     }
 
-    public IColumnIterator getMemtableColumnIterator(ColumnFamily cf, DecoratedKey key, AbstractType comparator)
+    public IColumnIterator getMemtableColumnIterator(ColumnFamily cf, DecoratedKey<?> key, AbstractType comparator)
     {
         assert cf != null;
         if (path.superColumnName == null)
@@ -79,7 +79,7 @@ public class QueryFilter
         return superFilter.getSSTableColumnIterator(sstable, key);
     }
 
-    public IColumnIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput file, DecoratedKey key)
+    public IColumnIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput file, DecoratedKey<?> key)
     {
         if (path.superColumnName == null)
             return filter.getSSTableColumnIterator(sstable, file, key);
@@ -166,7 +166,7 @@ public class QueryFilter
      * @param reversed true to start with the largest column (as determined by configured sort order) instead of smallest
      * @param limit maximum number of non-deleted columns to return
      */
-    public static QueryFilter getSliceFilter(DecoratedKey key, QueryPath path, ByteBuffer start, ByteBuffer finish, boolean reversed, int limit)
+    public static QueryFilter getSliceFilter(DecoratedKey<?> key, QueryPath path, ByteBuffer start, ByteBuffer finish, boolean reversed, int limit)
     {
         return new QueryFilter(key, path, new SliceQueryFilter(start, finish, reversed, limit));
     }
@@ -175,7 +175,7 @@ public class QueryFilter
      * return a QueryFilter object that includes every column in the row.
      * This is dangerous on large rows; avoid except for test code.
      */
-    public static QueryFilter getIdentityFilter(DecoratedKey key, QueryPath path)
+    public static QueryFilter getIdentityFilter(DecoratedKey<?> key, QueryPath path)
     {
         return new QueryFilter(key, path, new IdentityQueryFilter());
     }
@@ -186,7 +186,7 @@ public class QueryFilter
      * @param path path to the level to slice at (CF or SuperColumn)
      * @param columns the column names to restrict the results to
      */
-    public static QueryFilter getNamesFilter(DecoratedKey key, QueryPath path, SortedSet<ByteBuffer> columns)
+    public static QueryFilter getNamesFilter(DecoratedKey<?> key, QueryPath path, SortedSet<ByteBuffer> columns)
     {
         return new QueryFilter(key, path, new NamesQueryFilter(columns));
     }
@@ -207,7 +207,7 @@ public class QueryFilter
     /**
      * convenience method for creating a name filter matching a single column
      */
-    public static QueryFilter getNamesFilter(DecoratedKey key, QueryPath path, ByteBuffer column)
+    public static QueryFilter getNamesFilter(DecoratedKey<?> key, QueryPath path, ByteBuffer column)
     {
         return new QueryFilter(key, path, new NamesQueryFilter(column));
     }

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java?rev=1166499&r1=1166498&r2=1166499&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java Thu Sep  8 01:58:42 2011
@@ -56,17 +56,17 @@ public class SliceQueryFilter implements
         this.count = count;
     }
 
-    public IColumnIterator getMemtableColumnIterator(ColumnFamily cf, DecoratedKey key, AbstractType comparator)
+    public IColumnIterator getMemtableColumnIterator(ColumnFamily cf, DecoratedKey<?> key, AbstractType comparator)
     {
         return Memtable.getSliceIterator(key, cf, this, comparator);
     }
 
-    public IColumnIterator getSSTableColumnIterator(SSTableReader sstable, DecoratedKey key)
+    public IColumnIterator getSSTableColumnIterator(SSTableReader sstable, DecoratedKey<?> key)
     {
         return new SSTableSliceIterator(sstable, key, start, finish, reversed);
     }
     
-    public IColumnIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput file, DecoratedKey key)
+    public IColumnIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput file, DecoratedKey<?> key)
     {
         return new SSTableSliceIterator(sstable, file, key, start, finish, reversed);
     }

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/migration/Migration.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/migration/Migration.java?rev=1166499&r1=1166498&r2=1166499&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/migration/Migration.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/migration/Migration.java Thu Sep  8 01:58:42 2011
@@ -130,10 +130,10 @@ public abstract class Migration
                 Table.open(Table.SYSTEM_TABLE).getColumnFamilyStore(Migration.MIGRATIONS_CF),
                 Table.open(Table.SYSTEM_TABLE).getColumnFamilyStore(Migration.SCHEMA_CF)
             };
-            List<Future> flushes = new ArrayList<Future>();
+            List<Future<?>> flushes = new ArrayList<Future<?>>();
             for (ColumnFamilyStore cfs : schemaStores)
                 flushes.add(cfs.forceFlush());
-            for (Future f : flushes)
+            for (Future<?> f : flushes)
             {
                 if (f == null)
                     // applying the migration triggered a flush independently
@@ -172,7 +172,7 @@ public abstract class Migration
 
     public static UUID getLastMigrationId()
     {
-        DecoratedKey dkey = StorageService.getPartitioner().decorateKey(LAST_MIGRATION_KEY);
+        DecoratedKey<?> dkey = StorageService.getPartitioner().decorateKey(LAST_MIGRATION_KEY);
         Table defs = Table.open(Table.SYSTEM_TABLE);
         ColumnFamilyStore cfStore = defs.getColumnFamilyStore(SCHEMA_CF);
         QueryFilter filter = QueryFilter.getNamesFilter(dkey, new QueryPath(SCHEMA_CF), LAST_MIGRATION_KEY);
@@ -267,8 +267,8 @@ public abstract class Migration
         Migration migration;
         try
         {
-            Class migrationClass = Class.forName(mi.classname.toString());
-            Constructor migrationConstructor = migrationClass.getDeclaredConstructor();
+            Class<?> migrationClass = Class.forName(mi.classname.toString());
+            Constructor<?> migrationConstructor = migrationClass.getDeclaredConstructor();
             migrationConstructor.setAccessible(true);
             migration = (Migration)migrationConstructor.newInstance();
         }
@@ -296,7 +296,7 @@ public abstract class Migration
     /** load serialized migrations. */
     public static Collection<IColumn> getLocalMigrations(UUID start, UUID end)
     {
-        DecoratedKey dkey = StorageService.getPartitioner().decorateKey(MIGRATIONS_KEY);
+        DecoratedKey<?> dkey = StorageService.getPartitioner().decorateKey(MIGRATIONS_KEY);
         Table defs = Table.open(Table.SYSTEM_TABLE);
         ColumnFamilyStore cfStore = defs.getColumnFamilyStore(Migration.MIGRATIONS_CF);
         QueryFilter filter = QueryFilter.getSliceFilter(dkey,

Modified: cassandra/trunk/src/java/org/apache/cassandra/io/sstable/IndexSummary.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/IndexSummary.java?rev=1166499&r1=1166498&r2=1166499&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/io/sstable/IndexSummary.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/io/sstable/IndexSummary.java Thu Sep  8 01:58:42 2011
@@ -56,12 +56,12 @@ public class IndexSummary
         return keysWritten % DatabaseDescriptor.getIndexInterval() == 0;
     }
 
-    public void addEntry(DecoratedKey decoratedKey, long indexPosition)
+    public void addEntry(DecoratedKey<?> decoratedKey, long indexPosition)
     {
         indexPositions.add(new KeyPosition(decoratedKey, indexPosition));
     }
 
-    public void maybeAddEntry(DecoratedKey decoratedKey, long indexPosition)
+    public void maybeAddEntry(DecoratedKey<?> decoratedKey, long indexPosition)
     {
         if (shouldAddEntry())
             addEntry(decoratedKey, indexPosition);
@@ -86,10 +86,10 @@ public class IndexSummary
      */
     public static final class KeyPosition implements Comparable<KeyPosition>
     {
-        public final DecoratedKey key;
+        public final DecoratedKey<?> key;
         public final long indexPosition;
 
-        public KeyPosition(DecoratedKey key, long indexPosition)
+        public KeyPosition(DecoratedKey<?> key, long indexPosition)
         {
             this.key = key;
             this.indexPosition = indexPosition;

Modified: cassandra/trunk/src/java/org/apache/cassandra/io/sstable/KeyIterator.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/KeyIterator.java?rev=1166499&r1=1166498&r2=1166499&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/io/sstable/KeyIterator.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/io/sstable/KeyIterator.java Thu Sep  8 01:58:42 2011
@@ -51,13 +51,13 @@ public class KeyIterator extends Abstrac
         }
     }
 
-    protected DecoratedKey computeNext()
+    protected DecoratedKey<?> computeNext()
     {
         try
         {
             if (in.isEOF())
                 return endOfData();
-            DecoratedKey key = SSTableReader.decodeKey(StorageService.getPartitioner(), desc, ByteBufferUtil.readWithShortLength(in));
+            DecoratedKey<?> key = SSTableReader.decodeKey(StorageService.getPartitioner(), desc, ByteBufferUtil.readWithShortLength(in));
             in.readLong(); // skip data position
             return key;
         }

Modified: cassandra/trunk/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java?rev=1166499&r1=1166498&r2=1166499&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java Thu Sep  8 01:58:42 2011
@@ -42,14 +42,14 @@ public class ReducingKeyIterator impleme
             iters.add(new KeyIterator(sstable.descriptor));
         mi = MergeIterator.get(iters, DecoratedKey.comparator, new MergeIterator.Reducer<DecoratedKey,DecoratedKey>()
         {
-            DecoratedKey reduced = null;
+            DecoratedKey<?> reduced = null;
 
             public void reduce(DecoratedKey current)
             {
                 reduced = current;
             }
 
-            protected DecoratedKey getReduced()
+            protected DecoratedKey<?> getReduced()
             {
                 return reduced;
             }
@@ -94,7 +94,7 @@ public class ReducingKeyIterator impleme
         return mi.hasNext();
     }
 
-    public DecoratedKey next()
+    public DecoratedKey<?> next()
     {
         return mi.next();
     }

Modified: cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java?rev=1166499&r1=1166498&r2=1166499&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java Thu Sep  8 01:58:42 2011
@@ -30,7 +30,6 @@ import org.apache.cassandra.config.CFMet
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.IColumn;
-import org.apache.cassandra.db.columniterator.IColumnIterator;
 import org.apache.cassandra.db.columniterator.ICountableColumnIterator;
 import org.apache.cassandra.db.marshal.MarshalException;
 import org.apache.cassandra.io.util.RandomAccessReader;
@@ -40,7 +39,7 @@ public class SSTableIdentityIterator imp
 {
     private static final Logger logger = LoggerFactory.getLogger(SSTableIdentityIterator.class);
 
-    private final DecoratedKey key;
+    private final DecoratedKey<?> key;
     private final DataInput input;
     private final long dataStart;
     public final long dataSize;
@@ -66,7 +65,7 @@ public class SSTableIdentityIterator imp
      * @param dataSize length of row data
      * @throws IOException
      */
-    public SSTableIdentityIterator(SSTableReader sstable, RandomAccessReader file, DecoratedKey key, long dataStart, long dataSize)
+    public SSTableIdentityIterator(SSTableReader sstable, RandomAccessReader file, DecoratedKey<?> key, long dataStart, long dataSize)
     throws IOException
     {
         this(sstable, file, key, dataStart, dataSize, false);
@@ -82,20 +81,20 @@ public class SSTableIdentityIterator imp
      * @param checkData if true, do its best to deserialize and check the coherence of row data
      * @throws IOException
      */
-    public SSTableIdentityIterator(SSTableReader sstable, RandomAccessReader file, DecoratedKey key, long dataStart, long dataSize, boolean checkData)
+    public SSTableIdentityIterator(SSTableReader sstable, RandomAccessReader file, DecoratedKey<?> key, long dataStart, long dataSize, boolean checkData)
     throws IOException
     {
         this(sstable.metadata, file, key, dataStart, dataSize, checkData, sstable, false);
     }
 
-    public SSTableIdentityIterator(CFMetaData metadata, DataInput file, DecoratedKey key, long dataStart, long dataSize, boolean fromRemote)
+    public SSTableIdentityIterator(CFMetaData metadata, DataInput file, DecoratedKey<?> key, long dataStart, long dataSize, boolean fromRemote)
     throws IOException
     {
         this(metadata, file, key, dataStart, dataSize, false, null, fromRemote);
     }
 
     // sstable may be null *if* deserializeRowHeader is false
-    private SSTableIdentityIterator(CFMetaData metadata, DataInput input, DecoratedKey key, long dataStart, long dataSize, boolean checkData, SSTableReader sstable, boolean fromRemote)
+    private SSTableIdentityIterator(CFMetaData metadata, DataInput input, DecoratedKey<?> key, long dataStart, long dataSize, boolean checkData, SSTableReader sstable, boolean fromRemote)
     throws IOException
     {
         this.input = input;
@@ -157,7 +156,7 @@ public class SSTableIdentityIterator imp
         }
     }
 
-    public DecoratedKey getKey()
+    public DecoratedKey<?> getKey()
     {
         return key;
     }

Modified: cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java?rev=1166499&r1=1166498&r2=1166499&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java Thu Sep  8 01:58:42 2011
@@ -84,7 +84,7 @@ public class SSTableScanner implements C
         file.close();
     }
 
-    public void seekTo(DecoratedKey seekKey)
+    public void seekTo(DecoratedKey<?> seekKey)
     {
         try
         {
@@ -165,7 +165,7 @@ public class SSTableScanner implements C
                     file.seek(finishedAt);
                 assert !file.isEOF();
 
-                DecoratedKey key = SSTableReader.decodeKey(sstable.partitioner,
+                DecoratedKey<?> key = SSTableReader.decodeKey(sstable.partitioner,
                                                            sstable.descriptor,
                                                            ByteBufferUtil.readWithShortLength(file));
                 long dataSize = SSTableReader.readRowSize(file, sstable.descriptor);

Modified: cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java?rev=1166499&r1=1166498&r2=1166499&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java Thu Sep  8 01:58:42 2011
@@ -48,7 +48,7 @@ public class SSTableWriter extends SSTab
     private IndexWriter iwriter;
     private SegmentedFile.Builder dbuilder;
     private final SequentialWriter dataFile;
-    private DecoratedKey lastWrittenKey;
+    private DecoratedKey<?> lastWrittenKey;
     private FileMark dataMark;
     private SSTableMetadata.Collector sstableMetadataCollector;
 
@@ -72,7 +72,7 @@ public class SSTableWriter extends SSTab
     public SSTableWriter(String filename,
                          long keyCount,
                          CFMetaData metadata,
-                         IPartitioner partitioner,
+                         IPartitioner<?> partitioner,
                          SSTableMetadata.Collector sstableMetadataCollector) throws IOException
     {
         super(Descriptor.fromFilename(filename),
@@ -117,7 +117,7 @@ public class SSTableWriter extends SSTab
         }
     }
 
-    private long beforeAppend(DecoratedKey decoratedKey) throws IOException
+    private long beforeAppend(DecoratedKey<?> decoratedKey) throws IOException
     {
         if (decoratedKey == null)
         {
@@ -133,7 +133,7 @@ public class SSTableWriter extends SSTab
         return (lastWrittenKey == null) ? 0 : dataFile.getFilePointer();
     }
 
-    private void afterAppend(DecoratedKey decoratedKey, long dataPosition) throws IOException
+    private void afterAppend(DecoratedKey<?> decoratedKey, long dataPosition) throws IOException
     {
         lastWrittenKey = decoratedKey;
         this.last = lastWrittenKey;
@@ -162,7 +162,7 @@ public class SSTableWriter extends SSTab
         return currentPosition;
     }
 
-    public void append(DecoratedKey decoratedKey, ColumnFamily cf) throws IOException
+    public void append(DecoratedKey<?> decoratedKey, ColumnFamily cf) throws IOException
     {
         long startPosition = beforeAppend(decoratedKey);
         ByteBufferUtil.writeWithShortLength(decoratedKey.key, dataFile.stream);
@@ -183,7 +183,7 @@ public class SSTableWriter extends SSTab
         sstableMetadataCollector.addColumnCount(columnCount);
     }
 
-    public void append(DecoratedKey decoratedKey, ByteBuffer value) throws IOException
+    public void append(DecoratedKey<?> decoratedKey, ByteBuffer value) throws IOException
     {
         long currentPosition = beforeAppend(decoratedKey);
         ByteBufferUtil.writeWithShortLength(decoratedKey.key, dataFile.stream);
@@ -193,7 +193,7 @@ public class SSTableWriter extends SSTab
         afterAppend(decoratedKey, currentPosition);
     }
 
-    public long appendFromStream(DecoratedKey key, CFMetaData metadata, long dataSize, DataInput in) throws IOException
+    public long appendFromStream(DecoratedKey<?> key, CFMetaData metadata, long dataSize, DataInput in) throws IOException
     {
         long currentPosition = beforeAppend(key);
         ByteBufferUtil.writeWithShortLength(key.key, dataFile.stream);
@@ -359,13 +359,13 @@ public class SSTableWriter extends SSTab
     {
         private final SequentialWriter indexFile;
         public final Descriptor desc;
-        public final IPartitioner partitioner;
+        public final IPartitioner<?> partitioner;
         public final SegmentedFile.Builder builder;
         public final IndexSummary summary;
         public final BloomFilter bf;
         private FileMark mark;
 
-        IndexWriter(Descriptor desc, IPartitioner part, long keyCount) throws IOException
+        IndexWriter(Descriptor desc, IPartitioner<?> part, long keyCount) throws IOException
         {
             this.desc = desc;
             this.partitioner = part;
@@ -375,7 +375,7 @@ public class SSTableWriter extends SSTab
             bf = BloomFilter.getFilter(keyCount, 15);
         }
 
-        public void afterAppend(DecoratedKey key, long dataPosition) throws IOException
+        public void afterAppend(DecoratedKey<?> key, long dataPosition) throws IOException
         {
             bf.add(key.key);
             long indexPosition = indexFile.getFilePointer();

Modified: cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractEndpointSnitch.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractEndpointSnitch.java?rev=1166499&r1=1166498&r2=1166499&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractEndpointSnitch.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractEndpointSnitch.java Thu Sep  8 01:58:42 2011
@@ -23,13 +23,8 @@ import java.net.InetAddress;
 import java.util.Collection;
 import java.util.List;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 public abstract class AbstractEndpointSnitch implements IEndpointSnitch
 {
-    private static final Logger logger = LoggerFactory.getLogger(AbstractEndpointSnitch.class);
-    
     public abstract List<InetAddress> getSortedListByProximity(InetAddress address, Collection<InetAddress> unsortedAddress);
     public abstract void sortByProximity(InetAddress address, List<InetAddress> addresses);
 
@@ -40,6 +35,6 @@ public abstract class AbstractEndpointSn
 
     public void gossiperStarting()
     {
-        //noop by default
+        // noop by default
     }
 }

Modified: cassandra/trunk/src/java/org/apache/cassandra/net/AsyncResult.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/net/AsyncResult.java?rev=1166499&r1=1166498&r2=1166499&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/net/AsyncResult.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/net/AsyncResult.java Thu Sep  8 01:58:42 2011
@@ -31,7 +31,8 @@ import org.slf4j.LoggerFactory;
 
 class AsyncResult implements IAsyncResult
 {
-    private static Logger logger = LoggerFactory.getLogger(AsyncResult.class);
+    private static final Logger logger = LoggerFactory.getLogger(AsyncResult.class);
+
     private byte[] result;
     private AtomicBoolean done = new AtomicBoolean(false);
     private Lock lock = new ReentrantLock();

Modified: cassandra/trunk/src/java/org/apache/cassandra/net/Header.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/net/Header.java?rev=1166499&r1=1166498&r2=1166499&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/net/Header.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/net/Header.java Thu Sep  8 01:58:42 2011
@@ -25,7 +25,6 @@ import java.net.InetAddress;
 import java.util.Hashtable;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.cassandra.io.ICompactSerializer;
 import org.apache.cassandra.service.StorageService;

Modified: cassandra/trunk/src/java/org/apache/cassandra/net/MessageDeliveryTask.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/net/MessageDeliveryTask.java?rev=1166499&r1=1166498&r2=1166499&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/net/MessageDeliveryTask.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/net/MessageDeliveryTask.java Thu Sep  8 01:58:42 2011
@@ -26,7 +26,7 @@ import org.apache.cassandra.service.Stor
 
 public class MessageDeliveryTask implements Runnable
 {
-    private static final Logger logger_ = LoggerFactory.getLogger(MessageDeliveryTask.class);    
+    private static final Logger logger = LoggerFactory.getLogger(MessageDeliveryTask.class);
 
     private Message message;
     private final long constructionTime = System.currentTimeMillis();
@@ -38,9 +38,9 @@ public class MessageDeliveryTask impleme
         this.message = message;
         this.id = id;
     }
-    
+
     public void run()
-    { 
+    {
         StorageService.Verb verb = message.getVerb();
         if (MessagingService.DROPPABLE_VERBS.contains(verb)
             && System.currentTimeMillis() > constructionTime + DatabaseDescriptor.getRpcTimeout())
@@ -52,7 +52,7 @@ public class MessageDeliveryTask impleme
         IVerbHandler verbHandler = MessagingService.instance().getVerbHandler(verb);
         if (verbHandler == null)
         {
-            logger_.debug("Unknown verb {}", verb);
+            logger.debug("Unknown verb {}", verb);
             return;
         }
 

Modified: cassandra/trunk/src/java/org/apache/cassandra/service/AbstractRowResolver.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/service/AbstractRowResolver.java?rev=1166499&r1=1166498&r2=1166499&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/service/AbstractRowResolver.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/service/AbstractRowResolver.java Thu Sep  8 01:58:42 2011
@@ -47,7 +47,7 @@ public abstract class AbstractRowResolve
 
     protected final String table;
     protected final ConcurrentMap<Message, ReadResponse> replies = new NonBlockingHashMap<Message, ReadResponse>();
-    protected final DecoratedKey key;
+    protected final DecoratedKey<?> key;
 
     public AbstractRowResolver(ByteBuffer key, String table)
     {

Modified: cassandra/trunk/src/java/org/apache/cassandra/service/DigestMismatchException.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/service/DigestMismatchException.java?rev=1166499&r1=1166498&r2=1166499&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/service/DigestMismatchException.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/service/DigestMismatchException.java Thu Sep  8 01:58:42 2011
@@ -23,9 +23,10 @@ import java.nio.ByteBuffer;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+@SuppressWarnings("serial")
 public class DigestMismatchException extends Exception
 {
-    public DigestMismatchException(DecoratedKey key, ByteBuffer digest1, ByteBuffer digest2)
+    public DigestMismatchException(DecoratedKey<?> key, ByteBuffer digest1, ByteBuffer digest2)
     {
         super(String.format("Mismatch for key %s (%s vs %s)",
                             key.toString(),

Modified: cassandra/trunk/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java?rev=1166499&r1=1166498&r2=1166499&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java Thu Sep  8 01:58:42 2011
@@ -24,7 +24,6 @@ package org.apache.cassandra.service;
 import java.io.IOException;
 
 import org.apache.cassandra.thrift.CassandraDaemon;
-import org.apache.thrift.transport.TTransportException;
 
 /**
  * An embedded, in-memory cassandra storage service that listens

Modified: cassandra/trunk/src/java/org/apache/cassandra/service/MigrationManager.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/service/MigrationManager.java?rev=1166499&r1=1166498&r2=1166499&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/service/MigrationManager.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/service/MigrationManager.java Thu Sep  8 01:58:42 2011
@@ -152,14 +152,14 @@ public class MigrationManager implements
      */
     public static void applyMigrations(final UUID from, final UUID to) throws IOException
     {
-        List<Future> updates = new ArrayList<Future>();
+        List<Future<?>> updates = new ArrayList<Future<?>>();
         Collection<IColumn> migrations = Migration.getLocalMigrations(from, to);
         for (IColumn col : migrations)
         {
             // assuming MessagingService.version_ is a bit of a risk, but you're playing with fire if you purposefully
             // take down a node to upgrade it during the middle of a schema update.
             final Migration migration = Migration.deserialize(col.value(), MessagingService.version_);
-            Future update = StageManager.getStage(Stage.MIGRATION).submit(new Runnable()
+            Future<?> update = StageManager.getStage(Stage.MIGRATION).submit(new Runnable()
             {
                 public void run()
                 {
@@ -182,7 +182,7 @@ public class MigrationManager implements
         }
         
         // wait on all the updates before proceeding.
-        for (Future f : updates)
+        for (Future<?> f : updates)
         {
             try
             {

Modified: cassandra/trunk/src/java/org/apache/cassandra/service/RowRepairResolver.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/service/RowRepairResolver.java?rev=1166499&r1=1166498&r2=1166499&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/service/RowRepairResolver.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/service/RowRepairResolver.java Thu Sep  8 01:58:42 2011
@@ -110,7 +110,7 @@ public class RowRepairResolver extends A
      * For each row version, compare with resolved (the superset of all row versions);
      * if it is missing anything, send a mutation to the endpoint it come from.
      */
-    public static List<IAsyncResult> scheduleRepairs(ColumnFamily resolved, String table, DecoratedKey key, List<ColumnFamily> versions, List<InetAddress> endpoints)
+    public static List<IAsyncResult> scheduleRepairs(ColumnFamily resolved, String table, DecoratedKey<?> key, List<ColumnFamily> versions, List<InetAddress> endpoints)
     {
         List<IAsyncResult> results = new ArrayList<IAsyncResult>(versions.size());