You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2012/12/01 02:04:17 UTC

[1/3] git commit: make index_interval configurable per columnfamily patch by Radim Kolar; reviewed by jbellis for CASSANDRA-3961

Updated Branches:
  refs/heads/trunk 40d2ad14b -> 6a5c9ed87


make index_interval configurable per columnfamily
patch by Radim Kolar; reviewed by jbellis for CASSANDRA-3961


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6a5c9ed8
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6a5c9ed8
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6a5c9ed8

Branch: refs/heads/trunk
Commit: 6a5c9ed87995059e8cb824608fd5907432fa7ab4
Parents: 40d2ad1
Author: Jonathan Ellis <jb...@apache.org>
Authored: Sat Dec 1 09:42:18 2012 +0900
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Sat Dec 1 10:03:52 2012 +0900

----------------------------------------------------------------------
 CHANGES.txt                                        |    1 +
 conf/cassandra.yaml                                |   12 -
 examples/client_only/conf/cassandra.yaml           |    5 -
 interface/cassandra.thrift                         |    3 +-
 .../cassandra/thrift/AuthenticationException.java  |   23 +-
 .../cassandra/thrift/AuthenticationRequest.java    |   28 +-
 .../org/apache/cassandra/thrift/Cassandra.java     | 1288 +++++++--------
 .../org/apache/cassandra/thrift/CfDef.java         |  110 ++-
 .../org/apache/cassandra/thrift/Constants.java     |    2 +-
 .../org/apache/cassandra/thrift/IndexClause.java   |    2 +-
 .../org/apache/cassandra/thrift/KsDef.java         |    8 +-
 pylib/cqlshlib/cql3handling.py                     |    5 +-
 src/java/org/apache/cassandra/cli/CliClient.java   |    7 +
 .../org/apache/cassandra/config/CFMetaData.java    |   23 +-
 src/java/org/apache/cassandra/config/Config.java   |    2 -
 .../cassandra/config/DatabaseDescriptor.java       |    5 -
 .../cassandra/db/compaction/CompactionManager.java |    4 +-
 .../cassandra/db/compaction/CompactionTask.java    |    2 +-
 .../apache/cassandra/db/compaction/Scrubber.java   |    2 +-
 .../dht/AbstractByteOrderedPartitioner.java        |    2 +-
 .../cassandra/dht/OrderPreservingPartitioner.java  |    2 +-
 .../apache/cassandra/io/sstable/IndexSummary.java  |   34 +-
 .../apache/cassandra/io/sstable/SSTableReader.java |   34 +-
 .../apache/cassandra/io/sstable/SSTableWriter.java |    2 +-
 .../apache/cassandra/service/StorageService.java   |   10 +-
 .../apache/cassandra/thrift/CassandraServer.java   |    2 +-
 .../org/apache/cassandra/cli/CliHelp.yaml          |    5 +
 .../db/compaction/LongCompactionsTest.java         |    4 +-
 .../org/apache/cassandra/db/KeyCollisionTest.java  |    2 +-
 .../cassandra/db/compaction/CompactionsTest.java   |    2 +-
 .../cassandra/io/LazilyCompactedRowTest.java       |    2 +-
 .../cassandra/io/sstable/SSTableReaderTest.java    |    2 +-
 32 files changed, 882 insertions(+), 753 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6a5c9ed8/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 1861f88..623d39f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 1.3
+ * make index_interval configurable per columnfamily (CASSANDRA-3961)
  * add default_tim_to_live (CASSANDRA-3974)
  * add memtable_flush_period_in_ms (CASSANDRA-4237)
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6a5c9ed8/conf/cassandra.yaml
----------------------------------------------------------------------
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index 3a44b2d..48060a0 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -598,18 +598,6 @@ request_scheduler: org.apache.cassandra.scheduler.NoScheduler
 # the request scheduling. Currently the only valid option is keyspace.
 # request_scheduler_id: keyspace
 
-# index_interval controls the sampling of entries from the primrary
-# row index in terms of space versus time.  The larger the interval,
-# the smaller and less effective the sampling will be.  In technicial
-# terms, the interval coresponds to the number of index entries that
-# are skipped between taking each sample.  All the sampled entries
-# must fit in memory.  Generally, a value between 128 and 512 here
-# coupled with a large key cache size on CFs results in the best trade
-# offs.  This value is not often changed, however if you have many
-# very small rows (many to an OS page), then increasing this will
-# often lower memory usage without a impact on performance.
-index_interval: 128
-
 # Enable or disable inter-node encryption
 # Default settings are TLS v1, RSA 1024-bit keys (it is imperative that
 # users generate their own keys) TLS_RSA_WITH_AES_128_CBC_SHA as the cipher

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6a5c9ed8/examples/client_only/conf/cassandra.yaml
----------------------------------------------------------------------
diff --git a/examples/client_only/conf/cassandra.yaml b/examples/client_only/conf/cassandra.yaml
index 0edc473..1171316 100644
--- a/examples/client_only/conf/cassandra.yaml
+++ b/examples/client_only/conf/cassandra.yaml
@@ -307,11 +307,6 @@ request_scheduler: org.apache.cassandra.scheduler.NoScheduler
 # the request scheduling. Currently the only valid option is keyspace.
 # request_scheduler_id: keyspace
 
-# The Index Interval determines how large the sampling of row keys
-#  is for a given SSTable. The larger the sampling, the more effective
-#  the index is at the cost of space.
-index_interval: 128
-
 # Keyspaces have ColumnFamilies.        (Usually 1 KS per application.)
 # ColumnFamilies have Rows.             (Dozens of CFs per KS.)
 # Rows contain Columns.                 (Many per CF.)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6a5c9ed8/interface/cassandra.thrift
----------------------------------------------------------------------
diff --git a/interface/cassandra.thrift b/interface/cassandra.thrift
index e1e28ab..69e9985 100644
--- a/interface/cassandra.thrift
+++ b/interface/cassandra.thrift
@@ -55,7 +55,7 @@ namespace rb CassandraThrift
 # An effort should be made not to break forward-client-compatibility either
 # (e.g. one should avoid removing obsolete fields from the IDL), but no
 # guarantees in this respect are made by the Cassandra project.
-const string VERSION = "19.35.0"
+const string VERSION = "19.36.0"
 
 
 #
@@ -444,6 +444,7 @@ struct CfDef {
     37: optional double dclocal_read_repair_chance = 0.0,
     38: optional i32 memtable_flush_period_in_ms,
     39: optional i32 default_time_to_live,
+    40: optional i32 index_interval,
 
     /* All of the following are now ignored and unsupplied. */
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6a5c9ed8/interface/thrift/gen-java/org/apache/cassandra/thrift/AuthenticationException.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/AuthenticationException.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/AuthenticationException.java
index 4bea125..5d88cd9 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/AuthenticationException.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/AuthenticationException.java
@@ -5,7 +5,7 @@
  */
 package org.apache.cassandra.thrift;
 /*
- *
+ * 
  * 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
@@ -13,25 +13,34 @@ package org.apache.cassandra.thrift;
  * 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.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
 import java.util.Map;
 import java.util.HashMap;
 import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
 import java.util.EnumSet;
 import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * invalid authentication request (invalid keyspace, user does not exist, or credentials invalid)
@@ -106,7 +115,7 @@ public class AuthenticationException extends Exception implements org.apache.thr
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-    tmpMap.put(_Fields.WHY, new org.apache.thrift.meta_data.FieldMetaData("why", org.apache.thrift.TFieldRequirementType.REQUIRED,
+    tmpMap.put(_Fields.WHY, new org.apache.thrift.meta_data.FieldMetaData("why", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AuthenticationException.class, metaDataMap);
@@ -267,14 +276,14 @@ public class AuthenticationException extends Exception implements org.apache.thr
     while (true)
     {
       field = iprot.readFieldBegin();
-      if (field.type == org.apache.thrift.protocol.TType.STOP) {
+      if (field.type == org.apache.thrift.protocol.TType.STOP) { 
         break;
       }
       switch (field.id) {
         case 1: // WHY
           if (field.type == org.apache.thrift.protocol.TType.STRING) {
             this.why = iprot.readString();
-          } else {
+          } else { 
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
           }
           break;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6a5c9ed8/interface/thrift/gen-java/org/apache/cassandra/thrift/AuthenticationRequest.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/AuthenticationRequest.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/AuthenticationRequest.java
index 42cce37..0f68426 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/AuthenticationRequest.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/AuthenticationRequest.java
@@ -5,7 +5,7 @@
  */
 package org.apache.cassandra.thrift;
 /*
- *
+ * 
  * 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
@@ -13,24 +13,34 @@ package org.apache.cassandra.thrift;
  * 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.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
 import java.util.Map;
 import java.util.HashMap;
 import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
 import java.util.EnumSet;
 import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Authentication requests can contain any data, dependent on the IAuthenticator used
@@ -105,9 +115,9 @@ public class AuthenticationRequest implements org.apache.thrift.TBase<Authentica
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-    tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.REQUIRED,
-        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
-            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING),
+    tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AuthenticationRequest.class, metaDataMap);
@@ -291,7 +301,7 @@ public class AuthenticationRequest implements org.apache.thrift.TBase<Authentica
     while (true)
     {
       field = iprot.readFieldBegin();
-      if (field.type == org.apache.thrift.protocol.TType.STOP) {
+      if (field.type == org.apache.thrift.protocol.TType.STOP) { 
         break;
       }
       switch (field.id) {
@@ -310,7 +320,7 @@ public class AuthenticationRequest implements org.apache.thrift.TBase<Authentica
               }
               iprot.readMapEnd();
             }
-          } else {
+          } else { 
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
           }
           break;