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 java.util.concurrent.ExecutionException; 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.db.ColumnFamilyStore; import org.apache.cassandra.db.DataRange; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.RowMutation; import org.apache.cassandra.db.RowPosition; import org.apache.cassandra.db.columniterator.IdentityQueryFilter; import org.apache.cassandra.db.columniterator.OnDiskAtomIterator; import org.apache.cassandra.db.filter.QueryFilter; import org.apache.cassandra.io.sstable.SSTableReader; import org.apache.cassandra.io.sstable.SSTableScanner; import org.apache.cassandra.utils.ByteBufferUtil; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @RunWith(OrderedJUnit4ClassRunner.class) public class TTLExpiryTest extends SchemaLoader { @Test public void testSimpleExpire() throws ExecutionException, InterruptedException { ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1"); cfs.disableAutoCompaction(); cfs.metadata.gcGraceSeconds(0); long timestamp = System.currentTimeMillis(); RowMutation rm = new RowMutation("Keyspace1", Util.dk("ttl").key); rm.add("Standard1", ByteBufferUtil.bytes("col"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1); rm.add("Standard1", ByteBufferUtil.bytes("col7"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1); rm.apply(); cfs.forceBlockingFlush(); rm = new RowMutation("Keyspace1", Util.dk("ttl").key); rm.add("Standard1", ByteBufferUtil.bytes("col2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1); rm.apply(); cfs.forceBlockingFlush(); rm = new RowMutation("Keyspace1", Util.dk("ttl").key); rm.add("Standard1", ByteBufferUtil.bytes("col3"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1); rm.apply(); cfs.forceBlockingFlush(); rm = new RowMutation("Keyspace1", Util.dk("ttl").key); rm.add("Standard1", ByteBufferUtil.bytes("col311"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1); rm.apply(); 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 ExecutionException, InterruptedException { ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1"); cfs.disableAutoCompaction(); cfs.metadata.gcGraceSeconds(0); long timestamp = System.currentTimeMillis(); RowMutation rm = new RowMutation("Keyspace1", Util.dk("ttl").key); rm.add("Standard1", ByteBufferUtil.bytes("col"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1); rm.add("Standard1", ByteBufferUtil.bytes("col7"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1); rm.apply(); cfs.forceBlockingFlush(); rm = new RowMutation("Keyspace1", Util.dk("ttl").key); rm.add("Standard1", ByteBufferUtil.bytes("col2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1); rm.apply(); cfs.forceBlockingFlush(); rm = new RowMutation("Keyspace1", Util.dk("ttl").key); rm.add("Standard1", ByteBufferUtil.bytes("col3"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1); rm.apply(); cfs.forceBlockingFlush(); DecoratedKey noTTLKey = Util.dk("nottl"); rm = new RowMutation("Keyspace1", noTTLKey.key); rm.add("Standard1", ByteBufferUtil.bytes("col311"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp); rm.apply(); 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(); SSTableScanner scanner = sstable.getScanner(DataRange.allData(sstable.partitioner)); assertTrue(scanner.hasNext()); while(scanner.hasNext()) { OnDiskAtomIterator iter = scanner.next(); assertEquals(noTTLKey, iter.getKey()); } } }