Skip to content

Commit

Permalink
add generic wildcards
Browse files Browse the repository at this point in the history
patch by Norman Maurer and Tupshin Harper for CASSANDRA-2247

git-svn-id: https://svn.apache.org/repos/asf/cassandra/trunk@1166499 13f79535-47bb-0310-9956-ffa450edef68
  • Loading branch information
jbellis committed Sep 8, 2011
1 parent 206294b commit a19f7f1
Show file tree
Hide file tree
Showing 22 changed files with 70 additions and 78 deletions.
Expand Up @@ -21,7 +21,6 @@
*/


import java.io.Closeable;
import java.io.DataOutput;
import java.io.IOException;
import java.security.MessageDigest;
Expand All @@ -35,9 +34,9 @@
*/
public abstract class AbstractCompactedRow
{
public final DecoratedKey key;
public final DecoratedKey<?> key;

public AbstractCompactedRow(DecoratedKey key)
public AbstractCompactedRow(DecoratedKey<?> key)
{
this.key = key;
}
Expand Down
Expand Up @@ -49,22 +49,22 @@ public class PrecompactedRow extends AbstractCompactedRow
private final int gcBefore;

// For testing purposes
public PrecompactedRow(DecoratedKey key, ColumnFamily compacted)
public PrecompactedRow(DecoratedKey<?> key, ColumnFamily compacted)
{
super(key);
this.compactedCf = compacted;
this.gcBefore = Integer.MAX_VALUE;
}

/** 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);
}
Expand Down
7 changes: 3 additions & 4 deletions src/java/org/apache/cassandra/db/filter/IFilter.java
Expand Up @@ -23,7 +23,6 @@
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;
Expand All @@ -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
Expand All @@ -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
Expand Down
6 changes: 3 additions & 3 deletions src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java
Expand Up @@ -50,17 +50,17 @@ public NamesQueryFilter(ByteBuffer column)
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);
}
Expand Down
18 changes: 9 additions & 9 deletions src/java/org/apache/cassandra/db/filter/QueryFilter.java
Expand Up @@ -40,14 +40,14 @@

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;
Expand All @@ -63,7 +63,7 @@ public IColumnIterator getMemtableColumnIterator(Memtable memtable, AbstractType
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)
Expand All @@ -79,7 +79,7 @@ public IColumnIterator getSSTableColumnIterator(SSTableReader sstable)
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);
Expand Down Expand Up @@ -166,7 +166,7 @@ public static boolean isRelevant(IColumn column, IColumnContainer container, int
* @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));
}
Expand All @@ -175,7 +175,7 @@ public static QueryFilter getSliceFilter(DecoratedKey key, QueryPath path, ByteB
* 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());
}
Expand All @@ -186,7 +186,7 @@ public static QueryFilter getIdentityFilter(DecoratedKey key, QueryPath path)
* @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));
}
Expand All @@ -207,7 +207,7 @@ public static IFilter getFilter(SlicePredicate predicate, AbstractType comparato
/**
* 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));
}
Expand Down
6 changes: 3 additions & 3 deletions src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
Expand Up @@ -56,17 +56,17 @@ public SliceQueryFilter(ByteBuffer start, ByteBuffer finish, boolean reversed, i
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);
}
Expand Down
12 changes: 6 additions & 6 deletions src/java/org/apache/cassandra/db/migration/Migration.java
Expand Up @@ -130,10 +130,10 @@ public final void apply() throws IOException, ConfigurationException
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
Expand Down Expand Up @@ -172,7 +172,7 @@ public final void passiveAnnounce()

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);
Expand Down Expand Up @@ -267,8 +267,8 @@ public static Migration deserialize(ByteBuffer bytes, int version) throws IOExce
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();
}
Expand Down Expand Up @@ -296,7 +296,7 @@ public static Migration deserialize(ByteBuffer bytes, int version) throws IOExce
/** 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,
Expand Down
8 changes: 4 additions & 4 deletions src/java/org/apache/cassandra/io/sstable/IndexSummary.java
Expand Up @@ -56,12 +56,12 @@ public boolean shouldAddEntry()
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);
Expand All @@ -86,10 +86,10 @@ public void complete()
*/
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;
Expand Down
4 changes: 2 additions & 2 deletions src/java/org/apache/cassandra/io/sstable/KeyIterator.java
Expand Up @@ -51,13 +51,13 @@ public KeyIterator(Descriptor desc)
}
}

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;
}
Expand Down
Expand Up @@ -42,14 +42,14 @@ public ReducingKeyIterator(Collection<SSTableReader> sstables)
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;
}
Expand Down Expand Up @@ -94,7 +94,7 @@ public boolean hasNext()
return mi.hasNext();
}

public DecoratedKey next()
public DecoratedKey<?> next()
{
return mi.next();
}
Expand Down
Expand Up @@ -30,7 +30,6 @@
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;
Expand All @@ -40,7 +39,7 @@ public class SSTableIdentityIterator implements Comparable<SSTableIdentityIterat
{
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;
Expand All @@ -66,7 +65,7 @@ public class SSTableIdentityIterator implements Comparable<SSTableIdentityIterat
* @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);
Expand All @@ -82,20 +81,20 @@ public SSTableIdentityIterator(SSTableReader sstable, RandomAccessReader file, D
* @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;
Expand Down Expand Up @@ -157,7 +156,7 @@ private SSTableIdentityIterator(CFMetaData metadata, DataInput input, DecoratedK
}
}

public DecoratedKey getKey()
public DecoratedKey<?> getKey()
{
return key;
}
Expand Down
4 changes: 2 additions & 2 deletions src/java/org/apache/cassandra/io/sstable/SSTableScanner.java
Expand Up @@ -84,7 +84,7 @@ public void close() throws IOException
file.close();
}

public void seekTo(DecoratedKey seekKey)
public void seekTo(DecoratedKey<?> seekKey)
{
try
{
Expand Down Expand Up @@ -165,7 +165,7 @@ public IColumnIterator next()
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);
Expand Down

0 comments on commit a19f7f1

Please sign in to comment.