Return-Path: Delivered-To: apmail-incubator-cassandra-commits-archive@minotaur.apache.org Received: (qmail 7054 invoked from network); 5 Dec 2009 18:47:16 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 5 Dec 2009 18:47:16 -0000 Received: (qmail 19598 invoked by uid 500); 5 Dec 2009 18:47:16 -0000 Delivered-To: apmail-incubator-cassandra-commits-archive@incubator.apache.org Received: (qmail 19543 invoked by uid 500); 5 Dec 2009 18:47:16 -0000 Mailing-List: contact cassandra-commits-help@incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: cassandra-dev@incubator.apache.org Delivered-To: mailing list cassandra-commits@incubator.apache.org Received: (qmail 19533 invoked by uid 99); 5 Dec 2009 18:47:16 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 05 Dec 2009 18:47:16 +0000 X-ASF-Spam-Status: No, hits=-2.6 required=5.0 tests=BAYES_00 X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 05 Dec 2009 18:47:13 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id 1B9F623888E9; Sat, 5 Dec 2009 18:46:53 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r887574 - in /incubator/cassandra/trunk/test/unit/org/apache/cassandra: db/ColumnFamilyStoreTest.java db/ColumnFamilyStoreUtils.java io/SSTableTest.java io/SSTableUtils.java Date: Sat, 05 Dec 2009 18:46:52 -0000 To: cassandra-commits@incubator.apache.org From: junrao@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20091205184653.1B9F623888E9@eris.apache.org> Author: junrao Date: Sat Dec 5 18:46:52 2009 New Revision: 887574 URL: http://svn.apache.org/viewvc?rev=887574&view=rev Log: preparation for AntiEntropyService; patched by Stu Hood, reviewed by junrao for CASSANDRA-193 Added: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyStoreUtils.java incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/SSTableUtils.java Modified: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/SSTableTest.java Modified: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java?rev=887574&r1=887573&r2=887574&view=diff ============================================================================== --- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java (original) +++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java Sat Dec 5 18:46:52 2009 @@ -57,17 +57,15 @@ @Test public void testGetColumnWithWrongBF() throws IOException, ExecutionException, InterruptedException { - Table table = Table.open("Keyspace1"); - ColumnFamilyStore store = table.getColumnFamilyStore("Standard1"); + List rms = new LinkedList(); RowMutation rm; - - // add data rm = new RowMutation("Keyspace1", "key1"); rm.add(new QueryPath("Standard1", null, "Column1".getBytes()), "asdf".getBytes(), 0); rm.add(new QueryPath("Standard1", null, "Column2".getBytes()), "asdf".getBytes(), 0); - rm.apply(); - store.forceBlockingFlush(); + rms.add(rm); + ColumnFamilyStore store = ColumnFamilyStoreUtils.writeColumnFamily(rms); + Table table = Table.open("Keyspace1"); List ssTables = table.getAllSSTablesOnDisk(); assertEquals(1, ssTables.size()); ssTables.get(0).forceBloomFilterFailures(); @@ -106,18 +104,16 @@ */ private void testAntiCompaction(String columnFamilyName, int insertsPerTable) throws IOException, ExecutionException, InterruptedException { - Table table = Table.open("Keyspace1"); - ColumnFamilyStore store = table.getColumnFamilyStore(columnFamilyName); - + List rms = new ArrayList(); for (int j = 0; j < insertsPerTable; j++) { String key = String.valueOf(j); RowMutation rm = new RowMutation("Keyspace1", key); rm.add(new QueryPath(columnFamilyName, null, "0".getBytes()), new byte[0], j); - rm.apply(); + rms.add(rm); } + ColumnFamilyStore store = ColumnFamilyStoreUtils.writeColumnFamily(rms); - store.forceBlockingFlush(); List ranges = new ArrayList(); IPartitioner partitioner = new CollatingOrderPreservingPartitioner(); Range r = new Range(partitioner.getToken("0"), partitioner.getToken("zzzzzzz")); Added: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyStoreUtils.java URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyStoreUtils.java?rev=887574&view=auto ============================================================================== --- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyStoreUtils.java (added) +++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyStoreUtils.java Sat Dec 5 18:46:52 2009 @@ -0,0 +1,48 @@ +/* +* 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. +*/ +package org.apache.cassandra.db; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.ExecutionException; + +public class ColumnFamilyStoreUtils +{ + /** + * Writes out a bunch of rows for a single column family. + * + * @param rows A group of RowMutations for the same table and column family. + * @return The ColumnFamilyStore that was used. + */ + public static ColumnFamilyStore writeColumnFamily(List rms) throws IOException, ExecutionException, InterruptedException + { + RowMutation first = rms.get(0); + String tablename = first.getTable(); + String cfname = first.columnFamilyNames().iterator().next(); + + Table table = Table.open(tablename); + ColumnFamilyStore store = table.getColumnFamilyStore(cfname); + + for (RowMutation rm : rms) + rm.apply(); + + store.forceBlockingFlush(); + return store; + } +} Modified: incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/SSTableTest.java URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/SSTableTest.java?rev=887574&r1=887573&r2=887574&view=diff ============================================================================== --- incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/SSTableTest.java (original) +++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/SSTableTest.java Sat Dec 5 18:46:52 2009 @@ -23,26 +23,28 @@ import java.util.*; import org.junit.Test; +import static org.junit.Assert.*; import org.apache.cassandra.CleanupHelper; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.dht.OrderPreservingPartitioner; +import com.google.common.base.Predicate; +import com.google.common.base.Predicates; + public class SSTableTest extends CleanupHelper { @Test public void testSingleWrite() throws IOException { - File f = tempSSTableFileName(); - // write test data - SSTableWriter writer = new SSTableWriter(f.getAbsolutePath(), 1, new OrderPreservingPartitioner()); - Random random = new Random(); + String key = Integer.toString(1); byte[] bytes = new byte[1024]; - random.nextBytes(bytes); + new Random().nextBytes(bytes); - String key = Integer.toString(1); - writer.append(writer.partitioner.decorateKey(key), bytes); - SSTableReader ssTable = writer.closeAndOpenReader(0.01); + TreeMap map = new TreeMap(); + map.put(key, bytes); + SSTableReader ssTable = SSTableUtils.writeSSTable("singlewrite", map, 1, + new OrderPreservingPartitioner(), 0.01); // verify verifySingle(ssTable, bytes, key); @@ -50,11 +52,6 @@ verifySingle(ssTable, bytes, key); } - private File tempSSTableFileName() throws IOException - { - return File.createTempFile("sstable", "-" + SSTable.TEMPFILE_MARKER + "-Data.db"); - } - private void verifySingle(SSTableReader sstable, byte[] bytes, String key) throws IOException { BufferedRandomAccessFile file = new BufferedRandomAccessFile(sstable.path, "r"); @@ -68,8 +65,6 @@ @Test public void testManyWrites() throws IOException { - File f = tempSSTableFileName(); - TreeMap map = new TreeMap(); for ( int i = 100; i < 1000; ++i ) { @@ -77,12 +72,8 @@ } // write - SSTableWriter writer = new SSTableWriter(f.getAbsolutePath(), 1000, new OrderPreservingPartitioner()); - for (String key: map.navigableKeySet()) - { - writer.append(writer.partitioner.decorateKey(key), map.get(key)); - } - SSTableReader ssTable = writer.closeAndOpenReader(0.01); + SSTableReader ssTable = SSTableUtils.writeSSTable("manywrites", map, 1000, + new OrderPreservingPartitioner(), 0.01); // verify verifyMany(ssTable, map); @@ -105,4 +96,36 @@ assert Arrays.equals(bytes2, map.get(key)); } } + + @Test + public void testGetIndexedDecoratedKeysFor() throws IOException { + final String ssname = "indexedkeys"; + + int numkeys = 1000; + TreeMap map = new TreeMap(); + for ( int i = 0; i < numkeys; i++ ) + { + map.put(Integer.toString(i), "blah".getBytes()); + } + + // write + SSTableReader ssTable = SSTableUtils.writeSSTable(ssname, map, 1000, + new OrderPreservingPartitioner(), 0.01); + + + // verify + Predicate cfpred; + Predicate dkpred; + + cfpred = new Predicate() { + public boolean apply(SSTable ss) + { + return ss.getColumnFamilyName().equals(ssname); + } + }; + dkpred = Predicates.alwaysTrue(); + int actual = SSTableReader.getIndexedDecoratedKeysFor(cfpred, dkpred).size(); + assert 0 < actual; + assert actual <= Math.ceil((double)numkeys/SSTableReader.indexInterval()); + } } Added: incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/SSTableUtils.java URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/SSTableUtils.java?rev=887574&view=auto ============================================================================== --- incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/SSTableUtils.java (added) +++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/SSTableUtils.java Sat Dec 5 18:46:52 2009 @@ -0,0 +1,46 @@ +/* +* 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. +*/ +package org.apache.cassandra.io; + +import java.io.File; +import java.io.IOException; + +import java.util.Map; +import java.util.SortedMap; + +import org.apache.cassandra.dht.IPartitioner; + +public class SSTableUtils +{ + public static File tempSSTableFileName(String cfname) throws IOException + { + return File.createTempFile(cfname + "-", "-" + SSTable.TEMPFILE_MARKER + "-Data.db"); + } + + public static SSTableReader writeSSTable(String cfname, SortedMap entries, int expectedKeys, IPartitioner partitioner, double cacheFraction) throws IOException + { + File f = tempSSTableFileName(cfname); + SSTableWriter writer = new SSTableWriter(f.getAbsolutePath(), expectedKeys, partitioner); + for (Map.Entry entry : entries.entrySet()) + { + writer.append(writer.partitioner.decorateKey(entry.getKey()), entry.getValue()); + } + return writer.closeAndOpenReader(cacheFraction); + } +}