Author: slebresne Date: Tue Nov 22 09:19:57 2011 New Revision: 1204869 URL: http://svn.apache.org/viewvc?rev=1204869&view=rev Log: Fix incorrect query results due to invalid SSTable.maxTimestamp patch by slebresne; reviewed by jbellis and amorton for CASSANDRA-3510
Modified: cassandra/branches/cassandra-1.0/CHANGES.txt cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/CollationController.java cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/EchoedRow.java cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/AbstractCompactedRow.java cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java Modified: cassandra/branches/cassandra-1.0/CHANGES.txt URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/CHANGES.txt?rev=1204869&r1=1204868&r2=1204869&view=diff ============================================================================== --- cassandra/branches/cassandra-1.0/CHANGES.txt (original) +++ cassandra/branches/cassandra-1.0/CHANGES.txt Tue Nov 22 09:19:57 2011 @@ -6,6 +6,7 @@ * record partitioner in sstable metadata component (CASSANDRA-3407) * add new upgradesstables nodetool command (CASSANDRA-3406) * skip --debug requirement to see common exceptions in CLI (CASSANDRA-3508) + * fix incorrect query results due to invalid max timestamp (CASSANDRA-3510) 1.0.3 * revert name-based query defragmentation aka CASSANDRA-2503 (CASSANDRA-3491) Modified: cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/CollationController.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/CollationController.java?rev=1204869&r1=1204868&r2=1204869&view=diff ============================================================================== --- cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/CollationController.java (original) +++ cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/CollationController.java Tue Nov 22 09:19:57 2011 @@ -170,7 +170,8 @@ public class CollationController AbstractColumnContainer container = filter.path.superColumnName == null ? returnCF : (SuperColumn) returnCF.getColumn(filter.path.superColumnName); - if (container == null) + // MIN_VALUE means we don't know any information + if (container == null || sstableTimestamp == Long.MIN_VALUE) return; for (Iterator<ByteBuffer> iterator = ((NamesQueryFilter) filter.filter).columns.iterator(); iterator.hasNext(); ) Modified: cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/EchoedRow.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/EchoedRow.java?rev=1204869&r1=1204868&r2=1204869&view=diff ============================================================================== --- cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/EchoedRow.java (original) +++ cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/EchoedRow.java Tue Nov 22 09:19:57 2011 @@ -73,6 +73,6 @@ public class EchoedRow extends AbstractC public long maxTimestamp() { - throw new UnsupportedOperationException(); + return Long.MIN_VALUE; } } Modified: cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/AbstractCompactedRow.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/AbstractCompactedRow.java?rev=1204869&r1=1204868&r2=1204869&view=diff ============================================================================== --- cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/AbstractCompactedRow.java (original) +++ cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/AbstractCompactedRow.java Tue Nov 22 09:19:57 2011 @@ -69,7 +69,9 @@ public abstract class AbstractCompactedR public abstract int columnCount(); /** - * @return the max column timestamp in the row + * @return the max column timestamp in the row or Long.MIN_VALUE if + * computing this value would require extra effort we're not willing to + * make. */ public abstract long maxTimestamp(); } Modified: cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java?rev=1204869&r1=1204868&r2=1204869&view=diff ============================================================================== --- cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java (original) +++ cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java Tue Nov 22 09:19:57 2011 @@ -160,8 +160,14 @@ public class SSTableWriter extends SSTab long dataSize = row.write(dataFile.stream); assert dataSize == dataFile.getFilePointer() - (dataStart + 8) : "incorrect row data size " + dataSize + " written to " + dataFile.getPath() + "; correct is " + (dataFile.getFilePointer() - (dataStart + 8)); - // max timestamp is not collected here, because we want to avoid deserializing an EchoedRow - // instead, it is collected when calling ColumnFamilyStore.createCompactionWriter + /* + * The max timestamp is not always collected here (more precisely, row.maxTimestamp() may return Long.MIN_VALUE), + * to avoid deserializing an EchoedRow. + * This is the reason why it is collected first when calling ColumnFamilyStore.createCompactionWriter + * However, for old sstables without timestamp, we still want to update the timestamp (and we know + * that in this case we will not use EchoedRow, since CompactionControler.needsDeserialize() will be true). + */ + sstableMetadataCollector.updateMaxTimestamp(row.maxTimestamp()); sstableMetadataCollector.addRowSize(dataFile.getFilePointer() - currentPosition); sstableMetadataCollector.addColumnCount(row.columnCount()); afterAppend(row.key, currentPosition); Modified: cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java?rev=1204869&r1=1204868&r2=1204869&view=diff ============================================================================== --- cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java (original) +++ cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java Tue Nov 22 09:19:57 2011 @@ -41,6 +41,7 @@ import org.apache.cassandra.dht.Range; import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.Descriptor; import org.apache.cassandra.io.sstable.SSTableReader; +import org.apache.cassandra.io.sstable.SSTable; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.thrift.*; import org.apache.cassandra.utils.ByteBufferUtil; @@ -737,4 +738,38 @@ public class ColumnFamilyStoreTest exten assertEquals(ByteBufferUtil.bytes("A"), columnSliced.getSubColumn(ByteBufferUtil.bytes("a")).value()); assertEquals(ByteBufferUtil.bytes("B"), columnSliced.getSubColumn(ByteBufferUtil.bytes("b")).value()); } + + @Test + public void testSliceByNamesCommandOldMetatada() throws Throwable + { + String tableName = "Keyspace1"; + String cfName= "Standard1"; + DecoratedKey key = Util.dk("slice-name-old-metadata"); + ByteBuffer cname = ByteBufferUtil.bytes("c1"); + Table table = Table.open(tableName); + ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName); + cfs.clearUnsafe(); + + // Create a column a 'high timestamp' + putColsStandard(cfs, key, new Column(cname, ByteBufferUtil.bytes("a"), 2)); + cfs.forceBlockingFlush(); + + // Nuke the metadata and reload that sstable + Collection<SSTableReader> ssTables = cfs.getSSTables(); + assertEquals(1, ssTables.size()); + cfs.clearUnsafe(); + assertEquals(0, cfs.getSSTables().size()); + + new File(ssTables.iterator().next().descriptor.filenameFor(SSTable.COMPONENT_STATS)).delete(); + cfs.loadNewSSTables(); + + // Add another column with a lower timestamp + putColsStandard(cfs, key, new Column(cname, ByteBufferUtil.bytes("b"), 1)); + + // Test fetching the column by name returns the first column + SliceByNamesReadCommand cmd = new SliceByNamesReadCommand(tableName, key.key, new QueryPath(cfName), Collections.singletonList(cname)); + ColumnFamily cf = cmd.getRow(table).cf; + Column column = (Column) cf.getColumn(cname); + assert column.value().equals(ByteBufferUtil.bytes("a")) : "expecting a, got " + ByteBufferUtil.string(column.value()); + } }