You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by gd...@apache.org on 2010/11/19 17:55:38 UTC

svn commit: r1036937 [2/2] - in /cassandra/branches/cassandra-0.7: ./ src/avro/ src/java/org/apache/cassandra/avro/ src/java/org/apache/cassandra/config/ src/java/org/apache/cassandra/db/ src/java/org/apache/cassandra/db/commitlog/ src/java/org/apache/...

Added: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/DefaultDouble.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/DefaultDouble.java?rev=1036937&view=auto
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/DefaultDouble.java (added)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/DefaultDouble.java Fri Nov 19 16:55:37 2010
@@ -0,0 +1,47 @@
+package org.apache.cassandra.utils;
+
+/**
+ * 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.
+ */
+
+
+public class DefaultDouble
+{
+    private final double originalValue;
+    private double currentValue;
+    
+    public DefaultDouble(double value)
+    {
+        originalValue = value;
+        currentValue = value;
+    }
+    
+    public double value() 
+    {
+        return currentValue;
+    }
+    
+    public void set(double d)
+    {
+        currentValue = d;
+    }
+    
+    public boolean isModified()
+    {
+        return originalValue != currentValue;
+    }
+}

Added: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/DefaultInteger.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/DefaultInteger.java?rev=1036937&view=auto
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/DefaultInteger.java (added)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/DefaultInteger.java Fri Nov 19 16:55:37 2010
@@ -0,0 +1,47 @@
+package org.apache.cassandra.utils;
+
+/**
+ * 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.
+ */
+
+
+public class DefaultInteger 
+{
+    private final int originalValue;
+    private int currentValue;
+    
+    public DefaultInteger(int value)
+    {
+        originalValue = value;
+        currentValue = value;
+    }
+    
+    public int value() 
+    {
+        return currentValue;
+    }
+    
+    public void set(int i)
+    {
+        currentValue = i;
+    }
+    
+    public boolean isModified()
+    {
+        return originalValue != currentValue;
+    }
+}

Modified: cassandra/branches/cassandra-0.7/test/long/org/apache/cassandra/db/LongCompactionSpeedTest.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/test/long/org/apache/cassandra/db/LongCompactionSpeedTest.java?rev=1036937&r1=1036936&r2=1036937&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/test/long/org/apache/cassandra/db/LongCompactionSpeedTest.java (original)
+++ cassandra/branches/cassandra-0.7/test/long/org/apache/cassandra/db/LongCompactionSpeedTest.java Fri Nov 19 16:55:37 2010
@@ -95,7 +95,7 @@ public class LongCompactionSpeedTest ext
         Thread.sleep(1000);
 
         long start = System.currentTimeMillis();
-        CompactionManager.instance.doCompaction(store, sstables, (int) (System.currentTimeMillis() / 1000) - DatabaseDescriptor.getCFMetaData(TABLE1, "Standard1").gcGraceSeconds);
+        CompactionManager.instance.doCompaction(store, sstables, (int) (System.currentTimeMillis() / 1000) - DatabaseDescriptor.getCFMetaData(TABLE1, "Standard1").getGcGraceSeconds());
         System.out.println(String.format("%s: sstables=%d rowsper=%d colsper=%d: %d ms",
                                          this.getClass().getName(),
                                          sstableCount,

Modified: cassandra/branches/cassandra-0.7/test/system/test_thrift_server.py
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/test/system/test_thrift_server.py?rev=1036937&r1=1036936&r2=1036937&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/test/system/test_thrift_server.py (original)
+++ cassandra/branches/cassandra-0.7/test/system/test_thrift_server.py Fri Nov 19 16:55:37 2010
@@ -1329,12 +1329,16 @@ class TestMutations(ThriftTester):
         modified_cf = CfDef('Keyspace1', 'ToBeIndexed', column_metadata=[modified_cd])
         modified_cf.id = cfid
         client.system_update_column_family(modified_cf)
+        
         ks1 = client.describe_keyspace('Keyspace1')
         server_cf = [x for x in ks1.cf_defs if x.name=='ToBeIndexed'][0]
         assert server_cf
         assert server_cf.column_metadata[0].index_type == modified_cd.index_type
         assert server_cf.column_metadata[0].index_name == modified_cd.index_name
-
+        
+        # sleep a bit to give time for the index to build.
+        time.sleep(0.1)
+        
         # simple query on one index expression
         cp = ColumnParent('ToBeIndexed')
         sp = SlicePredicate(slice_range=SliceRange('', ''))

Modified: cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java?rev=1036937&r1=1036936&r2=1036937&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java (original)
+++ cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java Fri Nov 19 16:55:37 2010
@@ -312,7 +312,7 @@ public class ColumnFamilyStoreTest exten
         rm.apply();
 
         ColumnFamilyStore cfs = table.getColumnFamilyStore("Indexed2");
-        ColumnDefinition old = cfs.metadata.column_metadata.get(ByteBufferUtil.bytes("birthdate"));
+        ColumnDefinition old = cfs.metadata.getColumn_metadata().get(ByteBufferUtil.bytes("birthdate"));
         ColumnDefinition cd = new ColumnDefinition(old.name, old.validator.getClass().getName(), IndexType.KEYS, "birthdate_index");
         cfs.addIndex(cd);
         while (!SystemTable.isIndexBuilt("Keyspace1", cfs.getIndexedColumnFamilyStore(ByteBufferUtil.bytes("birthdate")).columnFamily))

Modified: cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/db/DefsTest.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/db/DefsTest.java?rev=1036937&r1=1036936&r2=1036937&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/db/DefsTest.java (original)
+++ cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/db/DefsTest.java Fri Nov 19 16:55:37 2010
@@ -144,7 +144,7 @@ public class DefsTest extends CleanupHel
         assert DatabaseDescriptor.getDefsVersion().equals(ver2);
         
         // drop it.
-        Migration m3 = new DropColumnFamily("Keyspace1", "MigrationCf_2", true);
+        Migration m3 = new DropColumnFamily("Keyspace1", "MigrationCf_2");
         m3.apply();
         UUID ver3 = m3.getVersion();
         assert DatabaseDescriptor.getDefsVersion().equals(ver3);
@@ -225,7 +225,7 @@ public class DefsTest extends CleanupHel
         store.getFlushPath();
         assert DefsTable.getFiles(cfm.tableName, cfm.cfName).size() > 0;
         
-        new DropColumnFamily(ks.name, cfm.cfName, true).apply();
+        new DropColumnFamily(ks.name, cfm.cfName).apply();
         
         assert !DatabaseDescriptor.getTableDefinition(ks.name).cfMetaData().containsKey(cfm.cfName);
         
@@ -344,7 +344,7 @@ public class DefsTest extends CleanupHel
         store.forceBlockingFlush();
         assert DefsTable.getFiles(cfm.tableName, cfm.cfName).size() > 0;
         
-        new DropKeyspace(ks.name, true).apply();
+        new DropKeyspace(ks.name).apply();
         
         assert DatabaseDescriptor.getTableDefinition(ks.name) == null;
         
@@ -545,156 +545,132 @@ public class DefsTest extends CleanupHel
         
         assert DatabaseDescriptor.getTableDefinition(cf.tableName) != null;
         assert DatabaseDescriptor.getTableDefinition(cf.tableName) == ksm;
+        assert DatabaseDescriptor.getCFMetaData(cf.tableName, cf.cfName) != null;
         
         // updating certain fields should fail.
-        CfDef cf_def = new CfDef();
-        cf_def.setId(cf.cfId);
-        cf_def.setKeyspace(cf.tableName);
-        cf_def.setName(cf.cfName);
-        cf_def.setColumn_type(cf.cfType.name());
-        cf_def.setComment(cf.comment);
-        cf_def.setComparator_type(cf.comparator.getClass().getName());
-        cf_def.setSubcomparator_type(null);
-        cf_def.setGc_grace_seconds(cf.gcGraceSeconds);
-        cf_def.setKey_cache_size(cf.keyCacheSize);
-        cf_def.setRead_repair_chance(cf.readRepairChance);
-        cf_def.setRow_cache_size(43.3);
-        cf_def.setColumn_metadata(new ArrayList<ColumnDef>());
-        cf_def.setDefault_validation_class("BytesType");
-        cf_def.setMin_compaction_threshold(5);
-        cf_def.setMax_compaction_threshold(31);
+        org.apache.cassandra.avro.CfDef cf_def = CFMetaData.convertToAvro(cf);
+        cf_def.row_cache_size = 43.3;
+        cf_def.column_metadata = new ArrayList<org.apache.cassandra.avro.ColumnDef>();
+        cf_def.default_validation_class ="BytesType";
+        cf_def.min_compaction_threshold = 5;
+        cf_def.max_compaction_threshold = 31;
         
         // test valid operations.
-        cf_def.setComment("Modified comment");
-        CFMetaData updateCfm = cf.apply(cf_def);
-        new UpdateColumnFamily(cf, updateCfm).apply();
-        cf = updateCfm;
-        
-        cf_def.setRow_cache_size(2d);
-        updateCfm = cf.apply(cf_def);
-        new UpdateColumnFamily(cf, updateCfm).apply();
-        cf = updateCfm;
-        
-        cf_def.setKey_cache_size(3d);
-        updateCfm = cf.apply(cf_def);
-        new UpdateColumnFamily(cf, updateCfm).apply();
-        cf = updateCfm;
-        
-        cf_def.setRead_repair_chance(0.23);
-        updateCfm = cf.apply(cf_def);
-        new UpdateColumnFamily(cf, updateCfm).apply();
-        cf = updateCfm;
-        
-        cf_def.setGc_grace_seconds(12);
-        updateCfm = cf.apply(cf_def);
-        new UpdateColumnFamily(cf, updateCfm).apply();
-        cf = updateCfm;
-        
-        cf_def.setDefault_validation_class("UTF8Type");
-        updateCfm = cf.apply(cf_def);
-        new UpdateColumnFamily(cf, updateCfm).apply();
-        cf = updateCfm;
-
-        cf_def.setMin_compaction_threshold(3);
-        updateCfm = cf.apply(cf_def);
-        new UpdateColumnFamily(cf, updateCfm).apply();
-        cf = updateCfm;
-
-        cf_def.setMax_compaction_threshold(33);
-        updateCfm = cf.apply(cf_def);
-        new UpdateColumnFamily(cf, updateCfm).apply();
-        cf = updateCfm;
+        cf_def.comment = "Modified comment";
+        new UpdateColumnFamily(cf_def).apply(); // doesn't get set back here.
+        
+        cf_def.row_cache_size = 2d;
+        new UpdateColumnFamily(cf_def).apply();
+        
+        cf_def.key_cache_size = 3d;
+        new UpdateColumnFamily(cf_def).apply();
+        
+        cf_def.read_repair_chance = 0.23;
+        new UpdateColumnFamily(cf_def).apply();
+        
+        cf_def.gc_grace_seconds = 12;
+        new UpdateColumnFamily(cf_def).apply();
+        
+        cf_def.default_validation_class = "UTF8Type";
+        new UpdateColumnFamily(cf_def).apply();
+
+        cf_def.min_compaction_threshold = 3;
+        new UpdateColumnFamily(cf_def).apply();
+
+        cf_def.max_compaction_threshold = 33;
+        new UpdateColumnFamily(cf_def).apply();
 
         // can't test changing the reconciler because there is only one impl.
         
         // check the cumulative affect.
-        assert DatabaseDescriptor.getCFMetaData(cf.tableName, cf.cfName).comment.equals(cf_def.comment);
-        assert DatabaseDescriptor.getCFMetaData(cf.tableName, cf.cfName).rowCacheSize == cf_def.row_cache_size;
-        assert DatabaseDescriptor.getCFMetaData(cf.tableName, cf.cfName).keyCacheSize == cf_def.key_cache_size;
-        assert DatabaseDescriptor.getCFMetaData(cf.tableName, cf.cfName).readRepairChance == cf_def.read_repair_chance;
-        assert DatabaseDescriptor.getCFMetaData(cf.tableName, cf.cfName).gcGraceSeconds == cf_def.gc_grace_seconds;
-        assert DatabaseDescriptor.getCFMetaData(cf.tableName, cf.cfName).defaultValidator == UTF8Type.instance;
+        assert DatabaseDescriptor.getCFMetaData(cf.tableName, cf.cfName).getComment().equals(cf_def.comment);
+        assert DatabaseDescriptor.getCFMetaData(cf.tableName, cf.cfName).getRowCacheSize() == cf_def.row_cache_size;
+        assert DatabaseDescriptor.getCFMetaData(cf.tableName, cf.cfName).getKeyCacheSize() == cf_def.key_cache_size;
+        assert DatabaseDescriptor.getCFMetaData(cf.tableName, cf.cfName).getReadRepairChance() == cf_def.read_repair_chance;
+        assert DatabaseDescriptor.getCFMetaData(cf.tableName, cf.cfName).getGcGraceSeconds() == cf_def.gc_grace_seconds;
+        assert DatabaseDescriptor.getCFMetaData(cf.tableName, cf.cfName).getDefaultValidator() == UTF8Type.instance;
         
+        // todo: we probably don't need to reset old values in the catches anymore.
         // make sure some invalid operations fail.
         int oldId = cf_def.id;
         try
         {
-            cf_def.setId(cf_def.getId() + 1);
-            updateCfm = cf.apply(cf_def);
+            cf_def.id++;
+            cf.apply(cf_def);
             throw new AssertionError("Should have blown up when you used a different id.");
         }
         catch (ConfigurationException expected) 
         {
-            cf_def.setId(oldId);    
+            cf_def.id = oldId;    
         }
         
-        String oldStr = cf_def.getName();
+        CharSequence oldStr = cf_def.name;
         try
         {
-            cf_def.setName(cf_def.getName() + "_renamed");
-            updateCfm = cf.apply(cf_def);
+            cf_def.name = cf_def.name + "_renamed";
+            cf.apply(cf_def);
             throw new AssertionError("Should have blown up when you used a different name.");
         }
         catch (ConfigurationException expected)
         {
-            cf_def.setName(oldStr);
+            cf_def.name = oldStr;
         }
         
-        oldStr = cf_def.getKeyspace();
+        oldStr = cf_def.keyspace;
         try
         {
-            cf_def.setKeyspace(oldStr + "_renamed");
-            updateCfm = cf.apply(cf_def);
+            cf_def.keyspace = oldStr + "_renamed";
+            cf.apply(cf_def);
             throw new AssertionError("Should have blown up when you used a different keyspace.");
         }
         catch (ConfigurationException expected)
         {
-            cf_def.setKeyspace(oldStr);
+            cf_def.keyspace = oldStr;
         }
         
         try
         {
-            cf_def.setColumn_type(ColumnFamilyType.Super.name());
-            updateCfm = cf.apply(cf_def);
+            cf_def.column_type = ColumnFamilyType.Super.name();
+            cf.apply(cf_def);
             throw new AssertionError("Should have blwon up when you used a different cf type.");
         }
         catch (ConfigurationException expected)
         {
-            cf_def.setColumn_type(ColumnFamilyType.Standard.name());
+            cf_def.column_type = ColumnFamilyType.Standard.name();
         }
         
-        oldStr = cf_def.getComparator_type();
+        oldStr = cf_def.comparator_type;
         try 
         {
-            cf_def.setComparator_type(BytesType.class.getSimpleName());
-            updateCfm = cf.apply(cf_def);
+            cf_def.comparator_type = BytesType.class.getSimpleName();
+            cf.apply(cf_def);
             throw new AssertionError("Should have blown up when you used a different comparator.");
         }
         catch (ConfigurationException expected)
         {
-            cf_def.setComparator_type(UTF8Type.class.getSimpleName());
+            cf_def.comparator_type = UTF8Type.class.getSimpleName();
         }
 
         try
         {
-            cf_def.setMin_compaction_threshold(34);
-            updateCfm = cf.apply(cf_def);
+            cf_def.min_compaction_threshold = 34;
+            cf.apply(cf_def);
             throw new AssertionError("Should have blown up when min > max.");
         }
         catch (ConfigurationException expected)
         {
-            cf_def.setMin_compaction_threshold(3);
+            cf_def.min_compaction_threshold = 3;
         }
 
         try
         {
-            cf_def.setMax_compaction_threshold(2);
-            updateCfm = cf.apply(cf_def);
+            cf_def.max_compaction_threshold = 2;
+            cf.apply(cf_def);
             throw new AssertionError("Should have blown up when max > min.");
         }
         catch (ConfigurationException expected)
         {
-            cf_def.setMax_compaction_threshold(33);
+            cf_def.max_compaction_threshold = 33;
         }
     }
 
@@ -720,5 +696,4 @@ public class DefsTest extends CleanupHel
                               CFMetaData.DEFAULT_MEMTABLE_OPERATIONS_IN_MILLIONS,
                               Collections.<ByteBuffer, ColumnDefinition>emptyMap());
     }
-
 }

Modified: cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java?rev=1036937&r1=1036936&r2=1036937&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java (original)
+++ cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java Fri Nov 19 16:55:37 2010
@@ -81,6 +81,7 @@ public class SSTableWriterTest extends C
         FileUtils.deleteWithConfirm(orig.descriptor.filenameFor(Component.FILTER));
 
         SSTableReader sstr = CompactionManager.instance.submitSSTableBuild(orig.descriptor).get();
+        assert sstr != null;
         ColumnFamilyStore cfs = Table.open("Keyspace1").getColumnFamilyStore("Indexed1");
         cfs.addSSTable(sstr);
         cfs.buildSecondaryIndexes(cfs.getSSTables(), cfs.getIndexedColumns());