Skip to content

Commit

Permalink
Merge pull request apache#24 from krummas/pcmanus/8099
Browse files Browse the repository at this point in the history
readd TTLExpiryTest
  • Loading branch information
pcmanus committed Apr 20, 2015
2 parents 60af27c + 5887ddd commit 536a2af
Show file tree
Hide file tree
Showing 5 changed files with 240 additions and 6 deletions.
3 changes: 2 additions & 1 deletion src/java/org/apache/cassandra/db/atoms/AtomIterators.java
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.config.ColumnDefinition;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.partitions.ArrayBackedPartition;
import org.apache.cassandra.db.partitions.PartitionUpdate;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.io.sstable.CorruptSSTableException;
Expand Down Expand Up @@ -719,7 +720,7 @@ public MergedAtom setTo(Atom atom)

public MergedAtom setSecondTo(Atom atom)
{
this.second = second;
this.second = atom;
return this;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ public static enum Flag
public final int nowInSec;
public final int version;

private long maxLiveTimestamp;
private long maxLiveTimestamp = LivenessInfo.NO_TIMESTAMP;
private final ReusableLivenessInfo livenessInfo = new ReusableLivenessInfo();

// The currently read row liveness infos (timestamp, ttl and localDeletionTime).
Expand All @@ -63,7 +63,8 @@ public SerializationHelper(int version, Flag flag, int nowInSec)
public void writePartitionKeyLivenessInfo(Row.Writer writer, long timestamp, int ttl, int localDeletionTime)
{
livenessInfo.setTo(timestamp, ttl, localDeletionTime);
maxLiveTimestamp = timestamp;
if (livenessInfo.isLive(nowInSec))
maxLiveTimestamp = timestamp;
writer.writePartitionKeyLivenessInfo(livenessInfo);

rowTimestamp = timestamp;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -256,7 +256,7 @@ protected boolean include(DeletionTime dt)
@Override
protected boolean include(ColumnDefinition c, DeletionTime dt)
{
return dt.isLive() || !dt.isPurgeable(maxPurgeableTimestamp, controller.gcBefore);
return dt.isLive() || !dt.isPurgeable(getMaxPurgeableTimestamp(), controller.gcBefore);
}

/*
Expand Down
2 changes: 0 additions & 2 deletions src/java/org/apache/cassandra/utils/MergeIterator.java
Original file line number Diff line number Diff line change
Expand Up @@ -60,8 +60,6 @@ public void close()
{
if (iterator instanceof AutoCloseable)
((AutoCloseable)iterator).close();
else if (iterator instanceof Closeable)
((Closeable)iterator).close();
}
catch (Exception e)
{
Expand Down
234 changes: 234 additions & 0 deletions test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,234 @@
package org.apache.cassandra.db.compaction;
/*
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*
*/

import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.cql3.ColumnIdentifier;
import org.apache.cassandra.db.atoms.AtomIterator;
import org.apache.cassandra.db.marshal.AsciiType;
import org.apache.cassandra.db.partitions.ArrayBackedPartition;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.junit.BeforeClass;
import com.google.common.collect.Sets;
import org.junit.Test;
import org.junit.runner.RunWith;

import org.apache.cassandra.OrderedJUnit4ClassRunner;
import org.apache.cassandra.SchemaLoader;
import org.apache.cassandra.Util;
import org.apache.cassandra.config.KSMetaData;
import org.apache.cassandra.db.*;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.io.sstable.ISSTableScanner;
import org.apache.cassandra.locator.SimpleStrategy;
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.FBUtilities;

import java.io.IOException;
import java.util.Collections;
import java.util.Set;
import java.util.concurrent.ExecutionException;

import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;

@RunWith(OrderedJUnit4ClassRunner.class)
public class TTLExpiryTest
{
public static final String KEYSPACE1 = "TTLExpiryTest";
private static final String CF_STANDARD1 = "Standard1";

@BeforeClass
public static void defineSchema() throws ConfigurationException
{
SchemaLoader.prepareServer();
SchemaLoader.createKeyspace(KEYSPACE1,
SimpleStrategy.class,
KSMetaData.optsWithRF(1),
CFMetaData.Builder.create(KEYSPACE1, CF_STANDARD1)
.addPartitionKey("pKey", AsciiType.instance)
.addRegularColumn(new ColumnIdentifier("col1", true), AsciiType.instance)
.addRegularColumn(new ColumnIdentifier("col", true), AsciiType.instance)
.addRegularColumn(new ColumnIdentifier("col311", true), AsciiType.instance)
.addRegularColumn(new ColumnIdentifier("col2", true), AsciiType.instance)
.addRegularColumn(new ColumnIdentifier("col3", true), AsciiType.instance)
.addRegularColumn(new ColumnIdentifier("col7", true), AsciiType.instance)
.addRegularColumn(new ColumnIdentifier("shadow", true), AsciiType.instance)
.build().gcGraceSeconds(0));
}

@Test
public void testAggressiveFullyExpired()
{
ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard1");
cfs.disableAutoCompaction();
cfs.metadata.gcGraceSeconds(0);
String key = "ttl";
new RowUpdateBuilder(cfs.metadata, 1, 1, key)
.add("col1", ByteBufferUtil.EMPTY_BYTE_BUFFER)
.build()
.applyUnsafe();

new RowUpdateBuilder(cfs.metadata, 3, 1, key)
.add("col2", ByteBufferUtil.EMPTY_BYTE_BUFFER)
.build()
.applyUnsafe();
cfs.forceBlockingFlush();
new RowUpdateBuilder(cfs.metadata, 2, 1, key)
.add("col1", ByteBufferUtil.EMPTY_BYTE_BUFFER)
.build()
.applyUnsafe();

new RowUpdateBuilder(cfs.metadata, 5, 1, key)
.add("col2", ByteBufferUtil.EMPTY_BYTE_BUFFER)
.build()
.applyUnsafe();

cfs.forceBlockingFlush();

new RowUpdateBuilder(cfs.metadata, 4, 1, key)
.add("col1", ByteBufferUtil.EMPTY_BYTE_BUFFER)
.build()
.applyUnsafe();

new RowUpdateBuilder(cfs.metadata, 7, 1, key)
.add("shadow", ByteBufferUtil.EMPTY_BYTE_BUFFER)
.build()
.applyUnsafe();

cfs.forceBlockingFlush();


new RowUpdateBuilder(cfs.metadata, 6, 3, key)
.add("shadow", ByteBufferUtil.EMPTY_BYTE_BUFFER)
.build()
.applyUnsafe();

new RowUpdateBuilder(cfs.metadata, 8, 1, key)
.add("col2", ByteBufferUtil.EMPTY_BYTE_BUFFER)
.build()
.applyUnsafe();

cfs.forceBlockingFlush();

Set<SSTableReader> sstables = Sets.newHashSet(cfs.getSSTables());
int now = (int)(System.currentTimeMillis() / 1000);
int gcBefore = now + 2;
Set<SSTableReader> expired = CompactionController.getFullyExpiredSSTables(
cfs,
sstables,
Collections.EMPTY_SET,
gcBefore);
assertEquals(2, expired.size());

cfs.clearUnsafe();
}

@Test
public void testSimpleExpire() throws InterruptedException
{
ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard1");
cfs.disableAutoCompaction();
cfs.metadata.gcGraceSeconds(0);
long timestamp = System.currentTimeMillis();
String key = "ttl";
new RowUpdateBuilder(cfs.metadata, timestamp, 1, key)
.add("col", ByteBufferUtil.EMPTY_BYTE_BUFFER)
.add("col7", ByteBufferUtil.EMPTY_BYTE_BUFFER)
.build()
.applyUnsafe();

cfs.forceBlockingFlush();

new RowUpdateBuilder(cfs.metadata, timestamp, 1, key)
.add("col2", ByteBufferUtil.EMPTY_BYTE_BUFFER)
.build()
.applyUnsafe();


cfs.forceBlockingFlush();
new RowUpdateBuilder(cfs.metadata, timestamp, 1, key)
.add("col3", ByteBufferUtil.EMPTY_BYTE_BUFFER)
.build()
.applyUnsafe();


cfs.forceBlockingFlush();
new RowUpdateBuilder(cfs.metadata, timestamp, 1, key)
.add("col311", ByteBufferUtil.EMPTY_BYTE_BUFFER)
.build()
.applyUnsafe();


cfs.forceBlockingFlush();
Thread.sleep(2000); // wait for ttl to expire
assertEquals(4, cfs.getSSTables().size());
cfs.enableAutoCompaction(true);
assertEquals(0, cfs.getSSTables().size());
}

@Test
public void testNoExpire() throws InterruptedException, IOException
{
ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard1");
cfs.disableAutoCompaction();
cfs.metadata.gcGraceSeconds(0);
long timestamp = System.currentTimeMillis();
String key = "ttl";
new RowUpdateBuilder(cfs.metadata, timestamp, 1, key)
.add("col", ByteBufferUtil.EMPTY_BYTE_BUFFER)
.add("col7", ByteBufferUtil.EMPTY_BYTE_BUFFER)
.build()
.applyUnsafe();

cfs.forceBlockingFlush();
new RowUpdateBuilder(cfs.metadata, timestamp, 1, key)
.add("col2", ByteBufferUtil.EMPTY_BYTE_BUFFER)
.build()
.applyUnsafe();
cfs.forceBlockingFlush();
new RowUpdateBuilder(cfs.metadata, timestamp, 1, key)
.add("col3", ByteBufferUtil.EMPTY_BYTE_BUFFER)
.build()
.applyUnsafe();
cfs.forceBlockingFlush();
String noTTLKey = "nottl";
new RowUpdateBuilder(cfs.metadata, timestamp, noTTLKey)
.add("col311", ByteBufferUtil.EMPTY_BYTE_BUFFER)
.build()
.applyUnsafe();

cfs.forceBlockingFlush();
Thread.sleep(2000); // wait for ttl to expire
assertEquals(4, cfs.getSSTables().size());
cfs.enableAutoCompaction(true);
assertEquals(1, cfs.getSSTables().size());
SSTableReader sstable = cfs.getSSTables().iterator().next();
ISSTableScanner scanner = sstable.getScanner(DataRange.allData(cfs.metadata, sstable.partitioner), FBUtilities.nowInSeconds());
assertTrue(scanner.hasNext());
while(scanner.hasNext())
{
AtomIterator iter = scanner.next();
assertEquals(Util.dk(noTTLKey), iter.partitionKey());
}
scanner.close();
}
}

0 comments on commit 536a2af

Please sign in to comment.