You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2014/02/13 23:35:49 UTC

[1/2] Replace index_interval with min/max_index_interval

Updated Branches:
  refs/heads/trunk e9f8fc716 -> ee477cc4d


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/src/java/org/apache/cassandra/io/sstable/IndexSummary.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummary.java b/src/java/org/apache/cassandra/io/sstable/IndexSummary.java
index 6d8712a..e32d8db 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummary.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummary.java
@@ -51,24 +51,31 @@ public class IndexSummary implements Closeable
     private static final Logger logger = LoggerFactory.getLogger(IndexSummary.class);
 
     public static final IndexSummarySerializer serializer = new IndexSummarySerializer();
-    private final int indexInterval;
+
+    /**
+     * A lower bound for the average number of partitions in between each index summary entry. A lower value means
+     * that more partitions will have an entry in the index summary when at the full sampling level.
+     */
+    private final int minIndexInterval;
+
     private final IPartitioner partitioner;
     private final int summarySize;
     private final int sizeAtFullSampling;
     private final Memory bytes;
 
     /**
-     * A value between MIN_SAMPLING_LEVEL and BASE_SAMPLING_LEVEL that represents how many of the original
+     * A value between 1 and BASE_SAMPLING_LEVEL that represents how many of the original
      * index summary entries ((1 / indexInterval) * numKeys) have been retained.
      *
      * Thus, this summary contains (samplingLevel / BASE_SAMPLING_LEVEL) * ((1 / indexInterval) * numKeys)) entries.
      */
     private final int samplingLevel;
 
-    public IndexSummary(IPartitioner partitioner, Memory memory, int summarySize, int sizeAtFullSampling, int indexInterval, int samplingLevel)
+    public IndexSummary(IPartitioner partitioner, Memory memory, int summarySize, int sizeAtFullSampling,
+                        int minIndexInterval, int samplingLevel)
     {
         this.partitioner = partitioner;
-        this.indexInterval = indexInterval;
+        this.minIndexInterval = minIndexInterval;
         this.summarySize = summarySize;
         this.sizeAtFullSampling = sizeAtFullSampling;
         this.bytes = memory;
@@ -140,9 +147,22 @@ public class IndexSummary implements Closeable
         return index == (summarySize - 1) ? bytes.size() : getPositionInSummary(index + 1);
     }
 
-    public int getIndexInterval()
+    public int getMinIndexInterval()
     {
-        return indexInterval;
+        return minIndexInterval;
+    }
+
+    public double getEffectiveIndexInterval()
+    {
+        return (BASE_SAMPLING_LEVEL / (double) samplingLevel) * minIndexInterval;
+    }
+
+    /**
+     * Returns an estimate of the total number of keys in the SSTable.
+     */
+    public long getEstimatedKeyCount()
+    {
+        return ((long) getMaxNumberOfEntries() + 1) * minIndexInterval;
     }
 
     public int size()
@@ -157,7 +177,7 @@ public class IndexSummary implements Closeable
 
     /**
      * Returns the number of entries this summary would have if it were at the full sampling level, which is equal
-     * to the number of entries in the primary on-disk index divided by the index interval.
+     * to the number of entries in the primary on-disk index divided by the min index interval.
      */
     public int getMaxNumberOfEntries()
     {
@@ -184,14 +204,14 @@ public class IndexSummary implements Closeable
      */
     public int getEffectiveIndexIntervalAfterIndex(int index)
     {
-        return Downsampling.getEffectiveIndexIntervalAfterIndex(index, samplingLevel, indexInterval);
+        return Downsampling.getEffectiveIndexIntervalAfterIndex(index, samplingLevel, minIndexInterval);
     }
 
     public static class IndexSummarySerializer
     {
         public void serialize(IndexSummary t, DataOutputStream out, boolean withSamplingLevel) throws IOException
         {
-            out.writeInt(t.indexInterval);
+            out.writeInt(t.minIndexInterval);
             out.writeInt(t.summarySize);
             out.writeLong(t.bytes.size());
             if (withSamplingLevel)
@@ -202,14 +222,15 @@ public class IndexSummary implements Closeable
             FBUtilities.copy(new MemoryInputStream(t.bytes), out, t.bytes.size());
         }
 
-        public IndexSummary deserialize(DataInputStream in, IPartitioner partitioner, boolean haveSamplingLevel, int expectedIndexInterval) throws IOException
+        public IndexSummary deserialize(DataInputStream in, IPartitioner partitioner, boolean haveSamplingLevel, int expectedMinIndexInterval, int maxIndexInterval) throws IOException
         {
-            int indexInterval = in.readInt();
-            if (indexInterval != expectedIndexInterval)
+            int minIndexInterval = in.readInt();
+            if (minIndexInterval != expectedMinIndexInterval)
             {
-                throw new IOException(String.format("Cannot read index summary because Index Interval changed from %d to %d.",
-                                                           indexInterval, expectedIndexInterval));
+                throw new IOException(String.format("Cannot read index summary because min_index_interval changed from %d to %d.",
+                                                    minIndexInterval, expectedMinIndexInterval));
             }
+
             int summarySize = in.readInt();
             long offheapSize = in.readLong();
             int samplingLevel, fullSamplingSummarySize;
@@ -223,9 +244,17 @@ public class IndexSummary implements Closeable
                 samplingLevel = BASE_SAMPLING_LEVEL;
                 fullSamplingSummarySize = summarySize;
             }
+
+            int effectiveIndexInterval = (int) Math.ceil((BASE_SAMPLING_LEVEL / (double) samplingLevel) * minIndexInterval);
+            if (effectiveIndexInterval > maxIndexInterval)
+            {
+                throw new IOException(String.format("Rebuilding index summary because the effective index interval (%d) is higher than" +
+                                                    " the current max index interval (%d)", effectiveIndexInterval, maxIndexInterval));
+            }
+
             Memory memory = Memory.allocate(offheapSize);
             FBUtilities.copy(in, new MemoryOutputStream(memory), offheapSize);
-            return new IndexSummary(partitioner, memory, summarySize, fullSamplingSummarySize, indexInterval, samplingLevel);
+            return new IndexSummary(partitioner, memory, summarySize, fullSamplingSummarySize, minIndexInterval, samplingLevel);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java
index 3635e7e..d77e887 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java
@@ -28,7 +28,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.cassandra.io.sstable.Downsampling.BASE_SAMPLING_LEVEL;
-import static org.apache.cassandra.io.sstable.Downsampling.MIN_SAMPLING_LEVEL;
 
 public class IndexSummaryBuilder
 {
@@ -36,43 +35,44 @@ public class IndexSummaryBuilder
 
     private final ArrayList<Long> positions;
     private final ArrayList<byte[]> keys;
-    private final int indexInterval;
+    private final int minIndexInterval;
     private final int samplingLevel;
     private final int[] startPoints;
     private long keysWritten = 0;
     private long indexIntervalMatches = 0;
     private long offheapSize = 0;
 
-    public IndexSummaryBuilder(long expectedKeys, int indexInterval, int samplingLevel)
+    public IndexSummaryBuilder(long expectedKeys, int minIndexInterval, int samplingLevel)
     {
-        this.indexInterval = indexInterval;
         this.samplingLevel = samplingLevel;
         this.startPoints = Downsampling.getStartPoints(BASE_SAMPLING_LEVEL, samplingLevel);
 
-        long expectedEntries = expectedKeys / indexInterval;
-        if (expectedEntries > Integer.MAX_VALUE)
+        long maxExpectedEntries = expectedKeys / minIndexInterval;
+        if (maxExpectedEntries > Integer.MAX_VALUE)
         {
-            // that's a _lot_ of keys, and a very low interval
-            int effectiveInterval = (int) Math.ceil((double) Integer.MAX_VALUE / expectedKeys);
-            expectedEntries = expectedKeys / effectiveInterval;
-            assert expectedEntries <= Integer.MAX_VALUE : expectedEntries;
-            logger.warn("Index interval of {} is too low for {} expected keys; using interval of {} instead",
-                        indexInterval, expectedKeys, effectiveInterval);
+            // that's a _lot_ of keys, and a very low min index interval
+            int effectiveMinInterval = (int) Math.ceil((double) Integer.MAX_VALUE / expectedKeys);
+            maxExpectedEntries = expectedKeys / effectiveMinInterval;
+            assert maxExpectedEntries <= Integer.MAX_VALUE : maxExpectedEntries;
+            logger.warn("min_index_interval of {} is too low for {} expected keys; using interval of {} instead",
+                        minIndexInterval, expectedKeys, effectiveMinInterval);
+            this.minIndexInterval = effectiveMinInterval;
+        }
+        else
+        {
+            this.minIndexInterval = minIndexInterval;
         }
 
-        // adjust our estimates based on the sampling level
-        expectedEntries = (expectedEntries * samplingLevel) / BASE_SAMPLING_LEVEL;
-
-        positions = new ArrayList<>((int)expectedEntries);
-        keys = new ArrayList<>((int)expectedEntries);
+        // for initializing data structures, adjust our estimates based on the sampling level
+        maxExpectedEntries = (maxExpectedEntries * samplingLevel) / BASE_SAMPLING_LEVEL;
+        positions = new ArrayList<>((int)maxExpectedEntries);
+        keys = new ArrayList<>((int)maxExpectedEntries);
     }
 
     public IndexSummaryBuilder maybeAddEntry(DecoratedKey decoratedKey, long indexPosition)
     {
-        if (keysWritten % indexInterval == 0)
+        if (keysWritten % minIndexInterval == 0)
         {
-            indexIntervalMatches++;
-
             // see if we should skip this key based on our sampling level
             boolean shouldSkip = false;
             for (int start : startPoints)
@@ -92,6 +92,8 @@ public class IndexSummaryBuilder
                 positions.add(indexPosition);
                 offheapSize += TypeSizes.NATIVE.sizeof(indexPosition);
             }
+
+            indexIntervalMatches++;
         }
         keysWritten++;
 
@@ -124,17 +126,22 @@ public class IndexSummaryBuilder
             memory.setLong(keyPosition, actualIndexPosition);
             keyPosition += TypeSizes.NATIVE.sizeof(actualIndexPosition);
         }
-        int sizeAtFullSampling = (int) Math.ceil(keysWritten / (double) indexInterval);
-        return new IndexSummary(partitioner, memory, keys.size(), sizeAtFullSampling, indexInterval, samplingLevel);
+        int sizeAtFullSampling = (int) Math.ceil(keysWritten / (double) minIndexInterval);
+        return new IndexSummary(partitioner, memory, keys.size(), sizeAtFullSampling, minIndexInterval, samplingLevel);
     }
 
     public static int entriesAtSamplingLevel(int samplingLevel, int maxSummarySize)
     {
-        return (samplingLevel * maxSummarySize) / BASE_SAMPLING_LEVEL;
+        return (int) Math.ceil((samplingLevel * maxSummarySize) / (double) BASE_SAMPLING_LEVEL);
     }
 
-    public static int calculateSamplingLevel(int currentSamplingLevel, int currentNumEntries, long targetNumEntries)
+    public static int calculateSamplingLevel(int currentSamplingLevel, int currentNumEntries, long targetNumEntries, int minIndexInterval, int maxIndexInterval)
     {
+        // effective index interval == (BASE_SAMPLING_LEVEL / samplingLevel) * minIndexInterval
+        // so we can just solve for minSamplingLevel here:
+        // maxIndexInterval == (BASE_SAMPLING_LEVEL / minSamplingLevel) * minIndexInterval
+        int effectiveMinSamplingLevel = Math.max(1, (int) Math.ceil((BASE_SAMPLING_LEVEL * minIndexInterval) / (double) maxIndexInterval));
+
         // Algebraic explanation for calculating the new sampling level (solve for newSamplingLevel):
         // originalNumEntries = (baseSamplingLevel / currentSamplingLevel) * currentNumEntries
         // newSpaceUsed = (newSamplingLevel / baseSamplingLevel) * originalNumEntries
@@ -142,7 +149,7 @@ public class IndexSummaryBuilder
         // newSpaceUsed = (newSamplingLevel / currentSamplingLevel) * currentNumEntries
         // (newSpaceUsed * currentSamplingLevel) / currentNumEntries = newSamplingLevel
         int newSamplingLevel = (int) (targetNumEntries * currentSamplingLevel) / currentNumEntries;
-        return Math.min(BASE_SAMPLING_LEVEL, Math.max(MIN_SAMPLING_LEVEL, newSamplingLevel));
+        return Math.min(BASE_SAMPLING_LEVEL, Math.max(effectiveMinSamplingLevel, newSamplingLevel));
     }
 
     /**
@@ -153,7 +160,7 @@ public class IndexSummaryBuilder
      * @param partitioner the partitioner used for the index summary
      * @return a new IndexSummary
      */
-    public static IndexSummary downsample(IndexSummary existing, int newSamplingLevel, IPartitioner partitioner)
+    public static IndexSummary downsample(IndexSummary existing, int newSamplingLevel, int minIndexInterval, IPartitioner partitioner)
     {
         // To downsample the old index summary, we'll go through (potentially) several rounds of downsampling.
         // Conceptually, each round starts at position X and then removes every Nth item.  The value of X follows
@@ -162,6 +169,7 @@ public class IndexSummaryBuilder
 
         int currentSamplingLevel = existing.getSamplingLevel();
         assert currentSamplingLevel > newSamplingLevel;
+        assert minIndexInterval == existing.getMinIndexInterval();
 
         // calculate starting indexes for downsampling rounds
         int[] startPoints = Downsampling.getStartPoints(currentSamplingLevel, newSamplingLevel);
@@ -207,6 +215,7 @@ public class IndexSummaryBuilder
             memory.setBytes(keyPosition, entry, 0, entry.length);
             keyPosition += entry.length;
         }
-        return new IndexSummary(partitioner, memory, newKeyCount, existing.getMaxNumberOfEntries(), existing.getIndexInterval(), newSamplingLevel);
+        return new IndexSummary(partitioner, memory, newKeyCount, existing.getMaxNumberOfEntries(),
+                                minIndexInterval, newSamplingLevel);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
index 1b36baf..b35f5f4 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
@@ -29,6 +29,7 @@ import javax.management.ObjectName;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.*;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -40,6 +41,8 @@ import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.WrappedRunnable;
 
+import static org.apache.cassandra.io.sstable.Downsampling.BASE_SAMPLING_LEVEL;
+
 /**
  * Manages the fixed-size memory pool for index summaries, periodically resizing them
  * in order to give more memory to hot sstables and less memory to cold sstables.
@@ -146,22 +149,22 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
         return memoryPoolBytes / 1024L / 1024L;
     }
 
-    public Map<String, Double> getSamplingRatios()
+    public Map<String, Integer> getIndexIntervals()
     {
         List<SSTableReader> sstables = getAllSSTables();
-        Map<String, Double> ratios = new HashMap<>(sstables.size());
+        Map<String, Integer> intervals = new HashMap<>(sstables.size());
         for (SSTableReader sstable : sstables)
-            ratios.put(sstable.getFilename(), sstable.getIndexSummarySamplingLevel() / (double) Downsampling.BASE_SAMPLING_LEVEL);
+            intervals.put(sstable.getFilename(), (int) Math.round(sstable.getEffectiveIndexInterval()));
 
-        return ratios;
+        return intervals;
     }
 
-    public double getAverageSamplingRatio()
+    public double getAverageIndexInterval()
     {
         List<SSTableReader> sstables = getAllSSTables();
         double total = 0.0;
         for (SSTableReader sstable : sstables)
-            total += sstable.getIndexSummarySamplingLevel() / (double) Downsampling.BASE_SAMPLING_LEVEL;
+            total += sstable.getEffectiveIndexInterval();
         return total / sstables.size();
     }
 
@@ -302,6 +305,8 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
 
         List<ResampleEntry> toDownsample = new ArrayList<>(sstables.size() / 4);
         List<ResampleEntry> toUpsample = new ArrayList<>(sstables.size() / 4);
+        List<ResampleEntry> forceResample = new ArrayList<>();
+        List<ResampleEntry> forceUpsample = new ArrayList<>();
         List<SSTableReader> newSSTables = new ArrayList<>(sstables.size());
 
         // Going from the coldest to the hottest sstables, try to give each sstable an amount of space proportional
@@ -309,6 +314,9 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
         long remainingSpace = memoryPoolCapacity;
         for (SSTableReader sstable : sstables)
         {
+            int minIndexInterval = sstable.metadata.getMinIndexInterval();
+            int maxIndexInterval = sstable.metadata.getMaxIndexInterval();
+
             double readsPerSec = sstable.readMeter == null ? 0.0 : sstable.readMeter.fifteenMinuteRate();
             long idealSpace = Math.round(remainingSpace * (readsPerSec / totalReadsPerSec));
 
@@ -317,14 +325,50 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
             double avgEntrySize = sstable.getIndexSummaryOffHeapSize() / (double) currentNumEntries;
             long targetNumEntries = Math.max(1, Math.round(idealSpace / avgEntrySize));
             int currentSamplingLevel = sstable.getIndexSummarySamplingLevel();
-            int newSamplingLevel = IndexSummaryBuilder.calculateSamplingLevel(currentSamplingLevel, currentNumEntries, targetNumEntries);
-            int numEntriesAtNewSamplingLevel = IndexSummaryBuilder.entriesAtSamplingLevel(newSamplingLevel, sstable.getMaxIndexSummarySize());
+            int maxSummarySize = sstable.getMaxIndexSummarySize();
 
-            logger.trace("{} has {} reads/sec; ideal space for index summary: {} bytes ({} entries); considering moving from level " +
-                         "{} ({} entries) to level {} ({} entries)",
-                         sstable.getFilename(), readsPerSec, idealSpace, targetNumEntries, currentSamplingLevel, currentNumEntries, newSamplingLevel, numEntriesAtNewSamplingLevel);
+            // if the min_index_interval changed, calculate what our current sampling level would be under the new min
+            if (sstable.getMinIndexInterval() != minIndexInterval)
+            {
+                int effectiveSamplingLevel = (int) Math.round(currentSamplingLevel * (minIndexInterval / (double) sstable.getMinIndexInterval()));
+                maxSummarySize = (int) Math.round(maxSummarySize * (sstable.getMinIndexInterval() / (double) minIndexInterval));
+                logger.trace("min_index_interval changed from {} to {}, so the current sampling level for {} is effectively now {} (was {})",
+                             sstable.getMinIndexInterval(), minIndexInterval, sstable, effectiveSamplingLevel, currentSamplingLevel);
+                currentSamplingLevel = effectiveSamplingLevel;
+            }
 
-            if (targetNumEntries >= currentNumEntries * UPSAMPLE_THRESHOLD && newSamplingLevel > currentSamplingLevel)
+            int newSamplingLevel = IndexSummaryBuilder.calculateSamplingLevel(currentSamplingLevel, currentNumEntries, targetNumEntries,
+                    minIndexInterval, maxIndexInterval);
+            int numEntriesAtNewSamplingLevel = IndexSummaryBuilder.entriesAtSamplingLevel(newSamplingLevel, maxSummarySize);
+            double effectiveIndexInterval = sstable.getEffectiveIndexInterval();
+
+            logger.trace("{} has {} reads/sec; ideal space for index summary: {} bytes ({} entries); considering moving " +
+                    "from level {} ({} entries, {} bytes) to level {} ({} entries, {} bytes)",
+                    sstable.getFilename(), readsPerSec, idealSpace, targetNumEntries, currentSamplingLevel, currentNumEntries,
+                    currentNumEntries * avgEntrySize, newSamplingLevel, numEntriesAtNewSamplingLevel,
+                    numEntriesAtNewSamplingLevel * avgEntrySize);
+
+            if (effectiveIndexInterval < minIndexInterval)
+            {
+                // The min_index_interval was changed; re-sample to match it.
+                logger.debug("Forcing resample of {} because the current index interval ({}) is below min_index_interval ({})",
+                        sstable, effectiveIndexInterval, minIndexInterval);
+                long spaceUsed = (long) Math.ceil(avgEntrySize * numEntriesAtNewSamplingLevel);
+                forceResample.add(new ResampleEntry(sstable, spaceUsed, newSamplingLevel));
+                remainingSpace -= spaceUsed;
+            }
+            else if (effectiveIndexInterval > maxIndexInterval)
+            {
+                // The max_index_interval was lowered; force an upsample to the effective minimum sampling level
+                logger.debug("Forcing upsample of {} because the current index interval ({}) is above max_index_interval ({})",
+                        sstable, effectiveIndexInterval, maxIndexInterval);
+                newSamplingLevel = Math.max(1, (BASE_SAMPLING_LEVEL * minIndexInterval) / maxIndexInterval);
+                numEntriesAtNewSamplingLevel = IndexSummaryBuilder.entriesAtSamplingLevel(newSamplingLevel, sstable.getMaxIndexSummarySize());
+                long spaceUsed = (long) Math.ceil(avgEntrySize * numEntriesAtNewSamplingLevel);
+                forceUpsample.add(new ResampleEntry(sstable, spaceUsed, newSamplingLevel));
+                remainingSpace -= avgEntrySize * numEntriesAtNewSamplingLevel;
+            }
+            else if (targetNumEntries >= currentNumEntries * UPSAMPLE_THRESHOLD && newSamplingLevel > currentSamplingLevel)
             {
                 long spaceUsed = (long) Math.ceil(avgEntrySize * numEntriesAtNewSamplingLevel);
                 toUpsample.add(new ResampleEntry(sstable, spaceUsed, newSamplingLevel));
@@ -354,7 +398,9 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
         }
 
         // downsample first, then upsample
+        toDownsample.addAll(forceResample);
         toDownsample.addAll(toUpsample);
+        toDownsample.addAll(forceUpsample);
         Multimap<DataTracker, SSTableReader> replacedByTracker = HashMultimap.create();
         Multimap<DataTracker, SSTableReader> replacementsByTracker = HashMultimap.create();
         for (ResampleEntry entry : toDownsample)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/src/java/org/apache/cassandra/io/sstable/IndexSummaryManagerMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManagerMBean.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManagerMBean.java
index 1e115cb..3382350 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManagerMBean.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManagerMBean.java
@@ -32,14 +32,11 @@ public interface IndexSummaryManagerMBean
     public double getMemoryPoolSizeInMB();
 
     /**
-     * Returns a map of SSTable filenames to their current sampling ratio, where 1.0 indicates that all of the
-     * original index summary entries have been retained and 0.5 indicates that half of the original entries have
-     * been discarded.
-     * @return A map of SSTable filenames to their sampling ratios.
+     * Returns a map of SSTable filenames to their current effective index interval.
      */
-    public Map<String, Double> getSamplingRatios();
+    public Map<String, Integer> getIndexIntervals();
 
-    public double getAverageSamplingRatio();
+    public double getAverageIndexInterval();
 
     public void redistributeSummaries() throws IOException;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
index ffb7be1..c02e397 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
@@ -192,10 +192,7 @@ public class SSTableReader extends SSTable implements Closeable
         if (count < 0)
         {
             for (SSTableReader sstable : sstables)
-            {
-                // using getMaxIndexSummarySize() lets us ignore the current sampling level
-                count += (sstable.getMaxIndexSummarySize() + 1) * sstable.indexSummary.getSamplingLevel();
-            }
+                count += sstable.estimatedKeys();
         }
         return count;
     }
@@ -608,7 +605,7 @@ public class SSTableReader extends SSTable implements Closeable
 
             IndexSummaryBuilder summaryBuilder = null;
             if (!summaryLoaded)
-                summaryBuilder = new IndexSummaryBuilder(estimatedKeys, metadata.getIndexInterval(), samplingLevel);
+                summaryBuilder = new IndexSummaryBuilder(estimatedKeys, metadata.getMinIndexInterval(), samplingLevel);
 
             long indexPosition;
             while ((indexPosition = primaryIndex.getFilePointer()) != indexSize)
@@ -664,7 +661,7 @@ public class SSTableReader extends SSTable implements Closeable
         try
         {
             iStream = new DataInputStream(new FileInputStream(summariesFile));
-            indexSummary = IndexSummary.serializer.deserialize(iStream, partitioner, descriptor.version.hasSamplingLevel, metadata.getIndexInterval());
+            indexSummary = IndexSummary.serializer.deserialize(iStream, partitioner, descriptor.version.hasSamplingLevel, metadata.getMinIndexInterval(), metadata.getMaxIndexInterval());
             first = partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
             last = partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
             ibuilder.deserializeBounds(iStream);
@@ -738,10 +735,25 @@ public class SSTableReader extends SSTable implements Closeable
      */
     public SSTableReader cloneWithNewSummarySamplingLevel(int samplingLevel) throws IOException
     {
+        int minIndexInterval = metadata.getMinIndexInterval();
+        int maxIndexInterval = metadata.getMaxIndexInterval();
+        double effectiveInterval = indexSummary.getEffectiveIndexInterval();
+
         IndexSummary newSummary;
-        if (samplingLevel < indexSummary.getSamplingLevel())
+
+         // We have to rebuild the summary from the on-disk primary index in three cases:
+         // 1. The sampling level went up, so we need to read more entries off disk
+         // 2. The min_index_interval changed (in either direction); this changes what entries would be in the summary
+         //    at full sampling (and consequently at any other sampling level)
+         // 3. The max_index_interval was lowered, forcing us to raise the sampling level
+        if (samplingLevel > indexSummary.getSamplingLevel() || indexSummary.getMinIndexInterval() != minIndexInterval || effectiveInterval > maxIndexInterval)
         {
-            newSummary = IndexSummaryBuilder.downsample(indexSummary, samplingLevel, partitioner);
+            newSummary = buildSummaryAtLevel(samplingLevel);
+        }
+        else if (samplingLevel < indexSummary.getSamplingLevel())
+        {
+            // we can use the existing index summary to make a smaller one
+            newSummary = IndexSummaryBuilder.downsample(indexSummary, samplingLevel, minIndexInterval, partitioner);
 
             SegmentedFile.Builder ibuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode());
             SegmentedFile.Builder dbuilder = compression
@@ -749,13 +761,10 @@ public class SSTableReader extends SSTable implements Closeable
                                            : SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode());
             saveSummary(ibuilder, dbuilder, newSummary);
         }
-        else if (samplingLevel > indexSummary.getSamplingLevel())
-        {
-            newSummary = upsampleSummary(samplingLevel);
-        }
         else
         {
-            throw new AssertionError("Attempted to clone SSTableReader with the same index summary sampling level");
+            throw new AssertionError("Attempted to clone SSTableReader with the same index summary sampling level and " +
+                                     "no adjustments to min/max_index_interval");
         }
 
         markReplaced();
@@ -769,14 +778,14 @@ public class SSTableReader extends SSTable implements Closeable
         return replacement;
     }
 
-    private IndexSummary upsampleSummary(int newSamplingLevel) throws IOException
+    private IndexSummary buildSummaryAtLevel(int newSamplingLevel) throws IOException
     {
         // we read the positions in a BRAF so we don't have to worry about an entry spanning a mmap boundary.
         RandomAccessReader primaryIndex = RandomAccessReader.open(new File(descriptor.filenameFor(Component.PRIMARY_INDEX)));
         try
         {
             long indexSize = primaryIndex.length();
-            IndexSummaryBuilder summaryBuilder = new IndexSummaryBuilder(estimatedKeys(), metadata.getIndexInterval(), newSamplingLevel);
+            IndexSummaryBuilder summaryBuilder = new IndexSummaryBuilder(estimatedKeys(), metadata.getMinIndexInterval(), newSamplingLevel);
 
             long indexPosition;
             while ((indexPosition = primaryIndex.getFilePointer()) != indexSize)
@@ -803,6 +812,16 @@ public class SSTableReader extends SSTable implements Closeable
         return indexSummary.getOffHeapSize();
     }
 
+    public int getMinIndexInterval()
+    {
+        return indexSummary.getMinIndexInterval();
+    }
+
+    public double getEffectiveIndexInterval()
+    {
+        return indexSummary.getEffectiveIndexInterval();
+    }
+
     public void releaseSummary() throws IOException
     {
         indexSummary.close();
@@ -880,11 +899,11 @@ public class SSTableReader extends SSTable implements Closeable
     }
 
     /**
-     * @return An estimate of the number of keys in this SSTable.
+     * @return An estimate of the number of keys in this SSTable based on the index summary.
      */
     public long estimatedKeys()
     {
-        return ((long) indexSummary.getMaxNumberOfEntries()) * indexSummary.getIndexInterval();
+        return indexSummary.getEstimatedKeyCount();
     }
 
     /**
@@ -898,8 +917,8 @@ public class SSTableReader extends SSTable implements Closeable
         for (Pair<Integer, Integer> sampleIndexRange : sampleIndexes)
             sampleKeyCount += (sampleIndexRange.right - sampleIndexRange.left + 1);
 
-        // adjust for the current sampling level
-        long estimatedKeys = sampleKeyCount * (Downsampling.BASE_SAMPLING_LEVEL * indexSummary.getIndexInterval()) / indexSummary.getSamplingLevel();
+        // adjust for the current sampling level: (BSL / SL) * index_interval_at_full_sampling
+        long estimatedKeys = sampleKeyCount * (Downsampling.BASE_SAMPLING_LEVEL * indexSummary.getMinIndexInterval()) / indexSummary.getSamplingLevel();
         return Math.max(1, estimatedKeys);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
index 9b50a18..ab1fe49 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
@@ -441,7 +441,7 @@ public class SSTableWriter extends SSTable
             indexFile = SequentialWriter.open(new File(descriptor.filenameFor(Component.PRIMARY_INDEX)),
                                               !metadata.populateIoCacheOnFlush());
             builder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode());
-            summary = new IndexSummaryBuilder(keyCount, metadata.getIndexInterval(), Downsampling.BASE_SAMPLING_LEVEL);
+            summary = new IndexSummaryBuilder(keyCount, metadata.getMinIndexInterval(), Downsampling.BASE_SAMPLING_LEVEL);
             bf = FilterFactory.getFilter(keyCount, metadata.getBloomFilterFpChance(), true);
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index cca104e..91b4a36 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -2823,7 +2823,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         ColumnFamilyStore cfs = t.getColumnFamilyStore(cfName);
         List<DecoratedKey> keys = keySamples(Collections.singleton(cfs), range);
 
-        final long totalRowCountEstimate = (keys.size() + 1) * metadata.getIndexInterval();
+        long totalRowCountEstimate = cfs.estimatedKeysForRange(range);
 
         // splitCount should be much smaller than number of key samples, to avoid huge sampling error
         final int minSamplesPerSplit = 4;
@@ -2831,22 +2831,20 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         final int splitCount = Math.max(1, Math.min(maxSplitCount, (int)(totalRowCountEstimate / keysPerSplit)));
 
         List<Token> tokens = keysToTokens(range, keys);
-        return getSplits(tokens, splitCount, metadata);
+        return getSplits(tokens, splitCount, metadata, cfs);
     }
 
-    private List<Pair<Range<Token>, Long>> getSplits(List<Token> tokens, int splitCount, CFMetaData metadata)
+    private List<Pair<Range<Token>, Long>> getSplits(List<Token> tokens, int splitCount, CFMetaData metadata, ColumnFamilyStore cfs)
     {
         final double step = (double) (tokens.size() - 1) / splitCount;
-        int prevIndex = 0;
         Token prevToken = tokens.get(0);
         List<Pair<Range<Token>, Long>> splits = Lists.newArrayListWithExpectedSize(splitCount);
         for (int i = 1; i <= splitCount; i++)
         {
             int index = (int) Math.round(i * step);
             Token token = tokens.get(index);
-            long rowCountEstimate = (index - prevIndex) * metadata.getIndexInterval();
-            splits.add(Pair.create(new Range<Token>(prevToken, token), rowCountEstimate));
-            prevIndex = index;
+            Range<Token> range = new Range<>(prevToken, token);
+            splits.add(Pair.create(range, cfs.estimatedKeysForRange(range)));
             prevToken = token;
         }
         return splits;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
index ce23a47..d68ba10 100644
--- a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
+++ b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
@@ -117,7 +117,7 @@ public class LongCompactionsTest extends SchemaLoader
         cfs.clearUnsafe();
 
         final int ROWS_PER_SSTABLE = 10;
-        final int SSTABLES = cfs.metadata.getIndexInterval() * 3 / ROWS_PER_SSTABLE;
+        final int SSTABLES = cfs.metadata.getMinIndexInterval() * 3 / ROWS_PER_SSTABLE;
 
         // disable compaction while flushing
         cfs.disableAutoCompaction();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/test/unit/org/apache/cassandra/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/SchemaLoader.java b/test/unit/org/apache/cassandra/SchemaLoader.java
index 5efc266..fbac746 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -169,7 +169,8 @@ public class SchemaLoader
                                            standardCFMD(ks1, "legacyleveled")
                                                                                .compactionStrategyClass(LeveledCompactionStrategy.class)
                                                                                .compactionStrategyOptions(leveledOptions),
-                                           standardCFMD(ks1, "StandardLowIndexInterval").indexInterval(8).caching(CFMetaData.Caching.NONE)));
+                                           standardCFMD(ks1, "StandardLowIndexInterval").minIndexInterval(8).maxIndexInterval(256).caching(CFMetaData.Caching.NONE)));
+
 
         // Keyspace 2
         schema.add(KSMetaData.testMetadata(ks2,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
index de449d8..d462d96 100644
--- a/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
@@ -74,7 +74,7 @@ public class BlacklistingCompactionsTest extends SchemaLoader
         final ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
 
         final int ROWS_PER_SSTABLE = 10;
-        final int SSTABLES = cfs.metadata.getIndexInterval() * 2 / ROWS_PER_SSTABLE;
+        final int SSTABLES = cfs.metadata.getMinIndexInterval() * 2 / ROWS_PER_SSTABLE;
 
         cfs.setCompactionStrategyClass(compactionStrategy);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
index 994a846..c0d9678 100644
--- a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
@@ -20,8 +20,12 @@ package org.apache.cassandra.io.sstable;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 import org.slf4j.Logger;
@@ -34,7 +38,6 @@ import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.metrics.RestorableMeter;
 
 import static org.apache.cassandra.io.sstable.Downsampling.BASE_SAMPLING_LEVEL;
-import static org.apache.cassandra.io.sstable.Downsampling.MIN_SAMPLING_LEVEL;
 import static org.apache.cassandra.io.sstable.IndexSummaryManager.DOWNSAMPLE_THESHOLD;
 import static org.apache.cassandra.io.sstable.IndexSummaryManager.UPSAMPLE_THRESHOLD;
 import static org.apache.cassandra.io.sstable.IndexSummaryManager.redistributeSummaries;
@@ -47,6 +50,33 @@ public class IndexSummaryManagerTest extends SchemaLoader
 {
     private static final Logger logger = LoggerFactory.getLogger(IndexSummaryManagerTest.class);
 
+    int originalMinIndexInterval;
+    int originalMaxIndexInterval;
+    long originalCapacity;
+
+    @Before
+    public void beforeTest()
+    {
+        String ksname = "Keyspace1";
+        String cfname = "StandardLowIndexInterval"; // index interval of 8, no key caching
+        Keyspace keyspace = Keyspace.open(ksname);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
+        originalMinIndexInterval = cfs.metadata.getMinIndexInterval();
+        originalMaxIndexInterval = cfs.metadata.getMaxIndexInterval();
+        originalCapacity = IndexSummaryManager.instance.getMemoryPoolCapacityInMB();
+    }
+
+    @After
+    public void afterTest()
+    {
+        String ksname = "Keyspace1";
+        String cfname = "StandardLowIndexInterval"; // index interval of 8, no key caching
+        Keyspace keyspace = Keyspace.open(ksname);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
+        cfs.metadata.minIndexInterval(originalMinIndexInterval);
+        cfs.metadata.maxIndexInterval(originalMaxIndexInterval);
+        IndexSummaryManager.instance.setMemoryPoolCapacityInMB(originalCapacity);
+    }
 
     private static long totalOffHeapSize(List<SSTableReader> sstables)
     {
@@ -73,11 +103,11 @@ public class IndexSummaryManagerTest extends SchemaLoader
     {
         for (int i = 0; i < numRows; i++)
         {
-            DecoratedKey key = Util.dk(String.valueOf(i));
+            DecoratedKey key = Util.dk(String.format("%3d", i));
             QueryFilter filter = QueryFilter.getIdentityFilter(key, cfs.getColumnFamilyName(), System.currentTimeMillis());
             ColumnFamily row = cfs.getColumnFamily(filter);
             assertNotNull(row);
-            Cell cell = row.getColumn(Util.cellname("cell"));
+            Cell cell = row.getColumn(Util.cellname("column"));
             assertNotNull(cell);
             assertEquals(100, cell.value().array().length);
         }
@@ -91,37 +121,171 @@ public class IndexSummaryManagerTest extends SchemaLoader
         }
     };
 
-    @Test
-    public void testRedistributeSummaries() throws IOException
+    private void createSSTables(String ksname, String cfname, int numSSTables, int numRows)
     {
-        String ksname = "Keyspace1";
-        String cfname = "StandardLowIndexInterval"; // index interval of 8, no key caching
         Keyspace keyspace = Keyspace.open(ksname);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
         cfs.truncateBlocking();
         cfs.disableAutoCompaction();
 
+        ArrayList<Future> futures = new ArrayList<>(numSSTables);
         ByteBuffer value = ByteBuffer.wrap(new byte[100]);
-
-        int numSSTables = 4;
-        int numRows = 256;
         for (int sstable = 0; sstable < numSSTables; sstable++)
         {
             for (int row = 0; row < numRows; row++)
             {
-                DecoratedKey key = Util.dk(String.valueOf(row));
+                DecoratedKey key = Util.dk(String.format("%3d", row));
                 Mutation rm = new Mutation(ksname, key.key);
                 rm.add(cfname, Util.cellname("column"), value, 0);
-                rm.apply();
+                rm.applyUnsafe();
             }
-            cfs.forceBlockingFlush();
+            futures.add(cfs.forceFlush());
         }
+        for (Future future : futures)
+        {
+            try
+            {
+                future.get();
+            } catch (InterruptedException e)
+            {
+                throw new RuntimeException(e);
+            }
+            catch (ExecutionException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+        assertEquals(numSSTables, cfs.getSSTables().size());
+        validateData(cfs, numRows);
+    }
+
+    @Test
+    public void testChangeMinIndexInterval() throws IOException
+    {
+        String ksname = "Keyspace1";
+        String cfname = "StandardLowIndexInterval"; // index interval of 8, no key caching
+        Keyspace keyspace = Keyspace.open(ksname);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
+        int numSSTables = 1;
+        int numRows = 256;
+        createSSTables(ksname, cfname, numSSTables, numRows);
 
         List<SSTableReader> sstables = new ArrayList<>(cfs.getSSTables());
-        assertEquals(numSSTables, sstables.size());
-        validateData(cfs, numRows);
+        for (SSTableReader sstable : sstables)
+            sstable.readMeter = new RestorableMeter(100.0, 100.0);
 
         for (SSTableReader sstable : sstables)
+            assertEquals(cfs.metadata.getMinIndexInterval(), sstable.getEffectiveIndexInterval(), 0.001);
+
+        // double the min_index_interval
+        cfs.metadata.minIndexInterval(originalMinIndexInterval * 2);
+        IndexSummaryManager.instance.redistributeSummaries();
+        for (SSTableReader sstable : cfs.getSSTables())
+        {
+            assertEquals(cfs.metadata.getMinIndexInterval(), sstable.getEffectiveIndexInterval(), 0.001);
+            assertEquals(numRows / cfs.metadata.getMinIndexInterval(), sstable.getIndexSummarySize());
+        }
+
+        // return min_index_interval to its original value
+        cfs.metadata.minIndexInterval(originalMinIndexInterval);
+        IndexSummaryManager.instance.redistributeSummaries();
+        for (SSTableReader sstable : cfs.getSSTables())
+        {
+            assertEquals(cfs.metadata.getMinIndexInterval(), sstable.getEffectiveIndexInterval(), 0.001);
+            assertEquals(numRows / cfs.metadata.getMinIndexInterval(), sstable.getIndexSummarySize());
+        }
+
+        // halve the min_index_interval, but constrain the available space to exactly what we have now; as a result,
+        // the summary shouldn't change
+        cfs.metadata.minIndexInterval(originalMinIndexInterval / 2);
+        SSTableReader sstable = cfs.getSSTables().iterator().next();
+        long summarySpace = sstable.getIndexSummaryOffHeapSize();
+        IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, Arrays.asList(sstable), summarySpace);
+        sstable = cfs.getSSTables().iterator().next();
+        assertEquals(originalMinIndexInterval, sstable.getEffectiveIndexInterval(), 0.001);
+        assertEquals(numRows / originalMinIndexInterval, sstable.getIndexSummarySize());
+
+        // keep the min_index_interval the same, but now give the summary enough space to grow by 50%
+        double previousInterval = sstable.getEffectiveIndexInterval();
+        int previousSize = sstable.getIndexSummarySize();
+        IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, Arrays.asList(sstable), (long) Math.ceil(summarySpace * 1.5));
+        sstable = cfs.getSSTables().iterator().next();
+        assertEquals(previousSize * 1.5, (double) sstable.getIndexSummarySize(), 1);
+        assertEquals(previousInterval * (1.0 / 1.5), sstable.getEffectiveIndexInterval(), 0.001);
+
+        // return min_index_interval to it's original value (double it), but only give the summary enough space
+        // to have an effective index interval of twice the new min
+        cfs.metadata.minIndexInterval(originalMinIndexInterval);
+        IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, Arrays.asList(sstable), (long) Math.ceil(summarySpace / 2));
+        sstable = cfs.getSSTables().iterator().next();
+        assertEquals(originalMinIndexInterval * 2, sstable.getEffectiveIndexInterval(), 0.001);
+        assertEquals(numRows / (originalMinIndexInterval * 2), sstable.getIndexSummarySize());
+
+        // raise the min_index_interval above our current effective interval, but set the max_index_interval lower
+        // than what we actually have space for (meaning the index summary would ideally be smaller, but this would
+        // result in an effective interval above the new max)
+        cfs.metadata.minIndexInterval(originalMinIndexInterval * 4);
+        cfs.metadata.maxIndexInterval(originalMinIndexInterval * 4);
+        IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, Arrays.asList(sstable), 10);
+        sstable = cfs.getSSTables().iterator().next();
+        assertEquals(cfs.metadata.getMinIndexInterval(), sstable.getEffectiveIndexInterval(), 0.001);
+    }
+
+    @Test
+    public void testChangeMaxIndexInterval() throws IOException
+    {
+        String ksname = "Keyspace1";
+        String cfname = "StandardLowIndexInterval"; // index interval of 8, no key caching
+        Keyspace keyspace = Keyspace.open(ksname);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
+        int numSSTables = 1;
+        int numRows = 256;
+        createSSTables(ksname, cfname, numSSTables, numRows);
+
+        List<SSTableReader> sstables = new ArrayList<>(cfs.getSSTables());
+        for (SSTableReader sstable : sstables)
+            sstable.readMeter = new RestorableMeter(100.0, 100.0);
+
+        IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, sstables, 1);
+        sstables = new ArrayList<>(cfs.getSSTables());
+        for (SSTableReader sstable : sstables)
+            assertEquals(cfs.metadata.getMaxIndexInterval(), sstable.getEffectiveIndexInterval(), 0.01);
+
+        // halve the max_index_interval
+        cfs.metadata.maxIndexInterval(cfs.metadata.getMaxIndexInterval() / 2);
+        IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, sstables, 1);
+        sstables = new ArrayList<>(cfs.getSSTables());
+        for (SSTableReader sstable : sstables)
+        {
+            assertEquals(cfs.metadata.getMaxIndexInterval(), sstable.getEffectiveIndexInterval(), 0.01);
+            assertEquals(numRows / cfs.metadata.getMaxIndexInterval(), sstable.getIndexSummarySize());
+        }
+
+        // return max_index_interval to its original value
+        cfs.metadata.maxIndexInterval(cfs.metadata.getMaxIndexInterval() * 2);
+        IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, sstables, 1);
+        for (SSTableReader sstable : cfs.getSSTables())
+        {
+            assertEquals(cfs.metadata.getMaxIndexInterval(), sstable.getEffectiveIndexInterval(), 0.01);
+            assertEquals(numRows / cfs.metadata.getMaxIndexInterval(), sstable.getIndexSummarySize());
+        }
+    }
+
+    @Test(timeout = 10000)
+    public void testRedistributeSummaries() throws IOException
+    {
+        String ksname = "Keyspace1";
+        String cfname = "StandardLowIndexInterval"; // index interval of 8, no key caching
+        Keyspace keyspace = Keyspace.open(ksname);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
+        int numSSTables = 4;
+        int numRows = 256;
+        createSSTables(ksname, cfname, numSSTables, numRows);
+
+        int minSamplingLevel = (BASE_SAMPLING_LEVEL * cfs.metadata.getMinIndexInterval()) / cfs.metadata.getMaxIndexInterval();
+
+        List<SSTableReader> sstables = new ArrayList<>(cfs.getSSTables());
+        for (SSTableReader sstable : sstables)
             sstable.readMeter = new RestorableMeter(100.0, 100.0);
 
         long singleSummaryOffHeapSpace = sstables.get(0).getIndexSummaryOffHeapSize();
@@ -147,7 +311,7 @@ public class IndexSummaryManagerTest extends SchemaLoader
         validateData(cfs, numRows);
 
         // upsample back up to half
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables,(singleSummaryOffHeapSpace * (numSSTables / 2)));
+        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables,(singleSummaryOffHeapSpace * (numSSTables / 2) + 4));
         assert sstables.size() == 4;
         for (SSTableReader sstable : sstables)
             assertEquals(BASE_SAMPLING_LEVEL / 2, sstable.getIndexSummarySamplingLevel());
@@ -194,7 +358,7 @@ public class IndexSummaryManagerTest extends SchemaLoader
         sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (singleSummaryOffHeapSpace * 3) + 50);
         Collections.sort(sstables, hotnessComparator);
 
-        if (sstables.get(0).getIndexSummarySamplingLevel() == MIN_SAMPLING_LEVEL)
+        if (sstables.get(0).getIndexSummarySamplingLevel() == minSamplingLevel)
             assertEquals(BASE_SAMPLING_LEVEL, sstables.get(1).getIndexSummarySamplingLevel());
         else
             assertEquals(BASE_SAMPLING_LEVEL, sstables.get(0).getIndexSummarySamplingLevel());
@@ -205,26 +369,26 @@ public class IndexSummaryManagerTest extends SchemaLoader
 
 
         // Cause a mix of upsampling and downsampling. We'll leave enough space for two full index summaries. The two
-        // coldest sstables will get downsampled to 8/128 of their size, leaving us with 1 and 112/128th index
+        // coldest sstables will get downsampled to 4/128 of their size, leaving us with 1 and 92/128th index
         // summaries worth of space.  The hottest sstable should get a full index summary, and the one in the middle
         // should get the remainder.
         sstables.get(0).readMeter = new RestorableMeter(0.0, 0.0);
         sstables.get(1).readMeter = new RestorableMeter(0.0, 0.0);
-        sstables.get(2).readMeter = new RestorableMeter(100, 100);
+        sstables.get(2).readMeter = new RestorableMeter(92, 92);
         sstables.get(3).readMeter = new RestorableMeter(128.0, 128.0);
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (long) (singleSummaryOffHeapSpace + (singleSummaryOffHeapSpace * (100.0 / BASE_SAMPLING_LEVEL))));
+        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (long) (singleSummaryOffHeapSpace + (singleSummaryOffHeapSpace * (92.0 / BASE_SAMPLING_LEVEL))));
         Collections.sort(sstables, hotnessComparator);
-        assertEquals(MIN_SAMPLING_LEVEL, sstables.get(0).getIndexSummarySamplingLevel());
-        assertEquals(MIN_SAMPLING_LEVEL, sstables.get(1).getIndexSummarySamplingLevel());
-        assertTrue(sstables.get(2).getIndexSummarySamplingLevel() > MIN_SAMPLING_LEVEL);
+        assertEquals(1, sstables.get(0).getIndexSummarySize());  // at the min sampling level
+        assertEquals(1, sstables.get(0).getIndexSummarySize());  // at the min sampling level
+        assertTrue(sstables.get(2).getIndexSummarySamplingLevel() > minSamplingLevel);
         assertTrue(sstables.get(2).getIndexSummarySamplingLevel() < BASE_SAMPLING_LEVEL);
         assertEquals(BASE_SAMPLING_LEVEL, sstables.get(3).getIndexSummarySamplingLevel());
         validateData(cfs, numRows);
 
         // Don't leave enough space for even the minimal index summaries
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, 100);
+        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, 10);
         for (SSTableReader sstable : sstables)
-            assertEquals(MIN_SAMPLING_LEVEL, sstable.getIndexSummarySamplingLevel());
+            assertEquals(1, sstable.getIndexSummarySize());  // at the min sampling level
         validateData(cfs, numRows);
     }
 
@@ -255,11 +419,11 @@ public class IndexSummaryManagerTest extends SchemaLoader
         SSTableReader original = sstables.get(0);
 
         SSTableReader sstable = original;
-        for (int samplingLevel = MIN_SAMPLING_LEVEL; samplingLevel < BASE_SAMPLING_LEVEL; samplingLevel++)
+        for (int samplingLevel = 1; samplingLevel < BASE_SAMPLING_LEVEL; samplingLevel++)
         {
             sstable = sstable.cloneWithNewSummarySamplingLevel(samplingLevel);
             assertEquals(samplingLevel, sstable.getIndexSummarySamplingLevel());
-            int expectedSize = (numRows * samplingLevel) / (sstable.metadata.getIndexInterval() * BASE_SAMPLING_LEVEL);
+            int expectedSize = (numRows * samplingLevel) / (sstable.metadata.getMinIndexInterval() * BASE_SAMPLING_LEVEL);
             assertEquals(expectedSize, sstable.getIndexSummarySize(), 1);
         }
 
@@ -311,19 +475,20 @@ public class IndexSummaryManagerTest extends SchemaLoader
             cfs.forceBlockingFlush();
         }
 
-        assertEquals(1.0, manager.getAverageSamplingRatio(), 0.001);
-        Map<String, Double> samplingRatios = manager.getSamplingRatios();
-        for (Map.Entry<String, Double> entry : samplingRatios.entrySet())
-            assertEquals(1.0, entry.getValue(), 0.001);
+        assertTrue(manager.getAverageIndexInterval() >= cfs.metadata.getMinIndexInterval());
+        Map<String, Integer> intervals = manager.getIndexIntervals();
+        for (Map.Entry<String, Integer> entry : intervals.entrySet())
+            if (entry.getKey().contains("StandardLowIndexInterval"))
+                assertEquals(cfs.metadata.getMinIndexInterval(), entry.getValue(), 0.001);
 
         manager.setMemoryPoolCapacityInMB(0);
         manager.redistributeSummaries();
-        assertTrue(manager.getAverageSamplingRatio() < 0.99);
-        samplingRatios = manager.getSamplingRatios();
-        for (Map.Entry<String, Double> entry : samplingRatios.entrySet())
+        assertTrue(manager.getAverageIndexInterval() > cfs.metadata.getMinIndexInterval());
+        intervals = manager.getIndexIntervals();
+        for (Map.Entry<String, Integer> entry : intervals.entrySet())
         {
             if (entry.getKey().contains("StandardLowIndexInterval"))
-                assertTrue(entry.getValue() < 0.9);
+                assertTrue(entry.getValue() >= cfs.metadata.getMinIndexInterval());
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java
index 2253302..ff991b4 100644
--- a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java
@@ -39,7 +39,6 @@ import org.apache.cassandra.utils.Pair;
 import static org.apache.cassandra.io.sstable.IndexSummaryBuilder.downsample;
 import static org.apache.cassandra.io.sstable.IndexSummaryBuilder.entriesAtSamplingLevel;
 import static org.apache.cassandra.io.sstable.Downsampling.BASE_SAMPLING_LEVEL;
-import static org.apache.cassandra.io.sstable.Downsampling.MIN_SAMPLING_LEVEL;
 
 import static org.junit.Assert.*;
 
@@ -81,7 +80,7 @@ public class IndexSummaryTest
         dos.writeUTF("JUNK");
         FileUtils.closeQuietly(dos);
         DataInputStream dis = new DataInputStream(new ByteArrayInputStream(aos.toByteArray()));
-        IndexSummary is = IndexSummary.serializer.deserialize(dis, DatabaseDescriptor.getPartitioner(), false, 1);
+        IndexSummary is = IndexSummary.serializer.deserialize(dis, DatabaseDescriptor.getPartitioner(), false, 1, 1);
         for (int i = 0; i < 100; i++)
             assertEquals(i, is.binarySearch(random.left.get(i)));
         // read the junk
@@ -105,7 +104,7 @@ public class IndexSummaryTest
         DataOutputStream dos = new DataOutputStream(aos);
         IndexSummary.serializer.serialize(summary, dos, false);
         DataInputStream dis = new DataInputStream(new ByteArrayInputStream(aos.toByteArray()));
-        IndexSummary loaded = IndexSummary.serializer.deserialize(dis, p, false, 1);
+        IndexSummary loaded = IndexSummary.serializer.deserialize(dis, p, false, 1, 1);
 
         assertEquals(1, loaded.size());
         assertEquals(summary.getPosition(0), loaded.getPosition(0));
@@ -135,10 +134,10 @@ public class IndexSummaryTest
         assertEquals(Arrays.asList(0), Downsampling.getSamplingPattern(0));
         assertEquals(Arrays.asList(0), Downsampling.getSamplingPattern(1));
 
-        assertEquals(Arrays.asList(0, 1), Downsampling.getSamplingPattern(2));
-        assertEquals(Arrays.asList(0, 2, 1, 3), Downsampling.getSamplingPattern(4));
-        assertEquals(Arrays.asList(0, 4, 2, 6, 1, 5, 3, 7), Downsampling.getSamplingPattern(8));
-        assertEquals(Arrays.asList(0, 8, 4, 12, 2, 10, 6, 14, 1, 9, 5, 13, 3, 11, 7, 15), Downsampling.getSamplingPattern(16));
+        assertEquals(Arrays.asList(1, 0), Downsampling.getSamplingPattern(2));
+        assertEquals(Arrays.asList(3, 1, 2, 0), Downsampling.getSamplingPattern(4));
+        assertEquals(Arrays.asList(7, 3, 5, 1, 6, 2, 4, 0), Downsampling.getSamplingPattern(8));
+        assertEquals(Arrays.asList(15, 7, 11, 3, 13, 5, 9, 1, 14, 6, 10, 2, 12, 4, 8, 0), Downsampling.getSamplingPattern(16));
     }
 
     private static boolean shouldSkip(int index, List<Integer> startPoints)
@@ -171,9 +170,9 @@ public class IndexSummaryTest
 
         // downsample by one level, then two levels, then three levels...
         int downsamplingRound = 1;
-        for (int samplingLevel = BASE_SAMPLING_LEVEL - 1; samplingLevel >= MIN_SAMPLING_LEVEL; samplingLevel--)
+        for (int samplingLevel = BASE_SAMPLING_LEVEL - 1; samplingLevel >= 1; samplingLevel--)
         {
-            IndexSummary downsampled = downsample(original, samplingLevel, DatabaseDescriptor.getPartitioner());
+            IndexSummary downsampled = downsample(original, samplingLevel, 128, DatabaseDescriptor.getPartitioner());
             assertEquals(entriesAtSamplingLevel(samplingLevel, original.getMaxNumberOfEntries()), downsampled.size());
 
             int sampledCount = 0;
@@ -192,9 +191,9 @@ public class IndexSummaryTest
         // downsample one level each time
         IndexSummary previous = original;
         downsamplingRound = 1;
-        for (int downsampleLevel = BASE_SAMPLING_LEVEL - 1; downsampleLevel >= MIN_SAMPLING_LEVEL; downsampleLevel--)
+        for (int downsampleLevel = BASE_SAMPLING_LEVEL - 1; downsampleLevel >= 1; downsampleLevel--)
         {
-            IndexSummary downsampled = downsample(previous, downsampleLevel, DatabaseDescriptor.getPartitioner());
+            IndexSummary downsampled = downsample(previous, downsampleLevel, 128, DatabaseDescriptor.getPartitioner());
             assertEquals(entriesAtSamplingLevel(downsampleLevel, original.getMaxNumberOfEntries()), downsampled.size());
 
             int sampledCount = 0;
@@ -216,7 +215,7 @@ public class IndexSummaryTest
     @Test
     public void testOriginalIndexLookup()
     {
-        for (int i = BASE_SAMPLING_LEVEL; i >= MIN_SAMPLING_LEVEL; i--)
+        for (int i = BASE_SAMPLING_LEVEL; i >= 1; i--)
             assertEquals(i, Downsampling.getOriginalIndexes(i).size());
 
         ArrayList<Integer> full = new ArrayList<>();
@@ -224,13 +223,13 @@ public class IndexSummaryTest
             full.add(i);
 
         assertEquals(full, Downsampling.getOriginalIndexes(BASE_SAMPLING_LEVEL));
-        // the entry at index 0 is the first to go
-        assertEquals(full.subList(1, full.size()), Downsampling.getOriginalIndexes(BASE_SAMPLING_LEVEL - 1));
+        // the entry at index 127 is the first to go
+        assertEquals(full.subList(0, full.size() - 1), Downsampling.getOriginalIndexes(BASE_SAMPLING_LEVEL - 1));
 
         // spot check a few values (these depend on BASE_SAMPLING_LEVEL being 128)
         assert BASE_SAMPLING_LEVEL == 128;
-        assertEquals(Arrays.asList(31, 63, 95, 127), Downsampling.getOriginalIndexes(4));
-        assertEquals(Arrays.asList(63, 127), Downsampling.getOriginalIndexes(2));
+        assertEquals(Arrays.asList(0, 32, 64, 96), Downsampling.getOriginalIndexes(4));
+        assertEquals(Arrays.asList(0, 64), Downsampling.getOriginalIndexes(2));
         assertEquals(Arrays.asList(), Downsampling.getOriginalIndexes(0));
     }
 
@@ -241,13 +240,13 @@ public class IndexSummaryTest
         for (int i = 0; i < BASE_SAMPLING_LEVEL; i++)
             assertEquals(indexInterval, Downsampling.getEffectiveIndexIntervalAfterIndex(i, BASE_SAMPLING_LEVEL, indexInterval));
 
-        // with one round of downsampling, only the first summary has been removed, so only the last index will have
+        // with one round of downsampling, only the last summary entry has been removed, so only the last index will have
         // double the gap until the next sample
         for (int i = 0; i < BASE_SAMPLING_LEVEL - 2; i++)
             assertEquals(indexInterval, Downsampling.getEffectiveIndexIntervalAfterIndex(i, BASE_SAMPLING_LEVEL - 1, indexInterval));
         assertEquals(indexInterval * 2, Downsampling.getEffectiveIndexIntervalAfterIndex(BASE_SAMPLING_LEVEL - 2, BASE_SAMPLING_LEVEL - 1, indexInterval));
 
-        // at samplingLevel=2, the retained summary points are [63, 127] (assumes BASE_SAMPLING_LEVEL is 128)
+        // at samplingLevel=2, the retained summary points are [0, 64] (assumes BASE_SAMPLING_LEVEL is 128)
         assert BASE_SAMPLING_LEVEL == 128;
         assertEquals(64 * indexInterval, Downsampling.getEffectiveIndexIntervalAfterIndex(0, 2, indexInterval));
         assertEquals(64 * indexInterval, Downsampling.getEffectiveIndexIntervalAfterIndex(1, 2, indexInterval));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
index 4b05e17..a7b8a3a 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
@@ -242,7 +242,7 @@ public class SSTableReaderTest extends SchemaLoader
 
         DecoratedKey firstKey = null, lastKey = null;
         long timestamp = System.currentTimeMillis();
-        for (int i = 0; i < store.metadata.getIndexInterval(); i++)
+        for (int i = 0; i < store.metadata.getMinIndexInterval(); i++)
         {
             DecoratedKey key = Util.dk(String.valueOf(i));
             if (firstKey == null)
@@ -359,7 +359,7 @@ public class SSTableReaderTest extends SchemaLoader
         final ColumnFamilyStore store = keyspace.getColumnFamilyStore("StandardLowIndexInterval"); // index interval of 8, no key caching
         CompactionManager.instance.disableAutoCompaction();
 
-        final int NUM_ROWS = 1000;
+        final int NUM_ROWS = 512;
         for (int j = 0; j < NUM_ROWS; j++)
         {
             ByteBuffer key = ByteBufferUtil.bytes(String.format("%3d", j));
@@ -402,7 +402,7 @@ public class SSTableReaderTest extends SchemaLoader
             }));
         }
 
-        SSTableReader replacement = sstable.cloneWithNewSummarySamplingLevel(Downsampling.MIN_SAMPLING_LEVEL);
+        SSTableReader replacement = sstable.cloneWithNewSummarySamplingLevel(1);
         store.getDataTracker().replaceReaders(Arrays.asList(sstable), Arrays.asList(replacement));
         for (Future future : futures)
             future.get();


[2/2] git commit: Replace index_interval with min/max_index_interval

Posted by al...@apache.org.
Replace index_interval with min/max_index_interval

patch by Tyler Hobbs; reviewed by Aleksey Yeschenko for CASSANDRA-6379


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

Branch: refs/heads/trunk
Commit: ee477cc4ddd1ab8ba4c7c2338d31060119735e7a
Parents: e9f8fc7
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Fri Feb 14 01:34:47 2014 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Fri Feb 14 01:34:47 2014 +0300

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |   6 +-
 interface/cassandra.thrift                      |   7 +-
 .../org/apache/cassandra/thrift/CfDef.java      | 478 +++++++++++++------
 .../cassandra/thrift/cassandraConstants.java    |   2 +-
 pylib/cqlshlib/cql3handling.py                  |   5 +-
 pylib/cqlshlib/test/test_cqlsh_output.py        |   3 +-
 .../org/apache/cassandra/config/CFMetaData.java | 102 ++--
 .../cassandra/cql3/statements/CFPropDefs.java   |  18 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |   8 +
 .../org/apache/cassandra/db/SystemKeyspace.java |  32 ++
 .../db/compaction/CompactionManager.java        |   4 +-
 .../cassandra/db/compaction/CompactionTask.java |   2 +-
 .../cassandra/db/compaction/Scrubber.java       |   2 +-
 .../cassandra/db/compaction/Upgrader.java       |   2 +-
 .../dht/OrderPreservingPartitioner.java         |   2 +-
 .../cassandra/io/sstable/Downsampling.java      |  47 +-
 .../cassandra/io/sstable/IndexSummary.java      |  59 ++-
 .../io/sstable/IndexSummaryBuilder.java         |  63 +--
 .../io/sstable/IndexSummaryManager.java         |  70 ++-
 .../io/sstable/IndexSummaryManagerMBean.java    |   9 +-
 .../cassandra/io/sstable/SSTableReader.java     |  57 ++-
 .../cassandra/io/sstable/SSTableWriter.java     |   2 +-
 .../cassandra/service/StorageService.java       |  12 +-
 .../db/compaction/LongCompactionsTest.java      |   2 +-
 .../unit/org/apache/cassandra/SchemaLoader.java |   3 +-
 .../compaction/BlacklistingCompactionsTest.java |   2 +-
 .../io/sstable/IndexSummaryManagerTest.java     | 235 +++++++--
 .../cassandra/io/sstable/IndexSummaryTest.java  |  35 +-
 .../cassandra/io/sstable/SSTableReaderTest.java |   6 +-
 30 files changed, 924 insertions(+), 352 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index dc9cff5..42c8bd9 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -31,6 +31,7 @@
  * Add option to use row cache with a given amount of rows (CASSANDRA-5357)
  * Avoid repairing already repaired data (CASSANDRA-5351)
  * Reject counter updates with USING TTL/TIMESTAMP (CASSANDRA-6649)
+ * Replace index_interval with min/max_index_interval (CASSANDRA-6379)
 
 
 2.0.6

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 3b3a64a..865830c 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -33,7 +33,7 @@ New features
 
 Upgrading
 ---------
-   - Rolling upgrades from anything pre-2.0.5 is not supported.
+   - Rolling upgrades from anything pre-2.0.6 is not supported.
    - For leveled compaction users, 2.0 must be atleast started before
      upgrading to 2.1 due to the fact that the old JSON leveled
      manifest is migrated into the sstable metadata files on startup
@@ -48,6 +48,10 @@ Upgrading
      to change your data model to accomodate the new implementation.
      (See https://issues.apache.org/jira/browse/CASSANDRA-6504 and the dev
      blog post at http://www.datastax.com/dev/blog/<PLACEHOLDER> for details).
+    - (per-table) index_interval parameter has been replaced with
+     min_index_interval and max_index_interval paratemeters. index_interval
+     has been deprecated.
+
 
 2.0.5
 =====

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/interface/cassandra.thrift
----------------------------------------------------------------------
diff --git a/interface/cassandra.thrift b/interface/cassandra.thrift
index 5f551ce..a223eb5 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.38.1"
+const string VERSION = "19.39.0"
 
 
 #
@@ -470,10 +470,11 @@ struct CfDef {
     38: optional bool populate_io_cache_on_flush,
     39: optional i32 memtable_flush_period_in_ms,
     40: optional i32 default_time_to_live,
-    41: optional i32 index_interval,
     42: optional string speculative_retry="NONE",
     43: optional list<TriggerDef> triggers,
     44: optional string cells_per_row_to_cache = "100",
+    45: optional i32 min_index_interval,
+    46: optional i32 max_index_interval,
 
     /* All of the following are now ignored and unsupplied. */
 
@@ -499,6 +500,8 @@ struct CfDef {
     27: optional string row_cache_provider,
     /** @deprecated */
     31: optional i32 row_cache_keys_to_save,
+    /** @deprecated */
+    41: optional i32 index_interval,
 }
 
 /* describes a keyspace. */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java
index 990b17e..4f6912e 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java
@@ -81,10 +81,11 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
   private static final org.apache.thrift.protocol.TField POPULATE_IO_CACHE_ON_FLUSH_FIELD_DESC = new org.apache.thrift.protocol.TField("populate_io_cache_on_flush", org.apache.thrift.protocol.TType.BOOL, (short)38);
   private static final org.apache.thrift.protocol.TField MEMTABLE_FLUSH_PERIOD_IN_MS_FIELD_DESC = new org.apache.thrift.protocol.TField("memtable_flush_period_in_ms", org.apache.thrift.protocol.TType.I32, (short)39);
   private static final org.apache.thrift.protocol.TField DEFAULT_TIME_TO_LIVE_FIELD_DESC = new org.apache.thrift.protocol.TField("default_time_to_live", org.apache.thrift.protocol.TType.I32, (short)40);
-  private static final org.apache.thrift.protocol.TField INDEX_INTERVAL_FIELD_DESC = new org.apache.thrift.protocol.TField("index_interval", org.apache.thrift.protocol.TType.I32, (short)41);
   private static final org.apache.thrift.protocol.TField SPECULATIVE_RETRY_FIELD_DESC = new org.apache.thrift.protocol.TField("speculative_retry", org.apache.thrift.protocol.TType.STRING, (short)42);
   private static final org.apache.thrift.protocol.TField TRIGGERS_FIELD_DESC = new org.apache.thrift.protocol.TField("triggers", org.apache.thrift.protocol.TType.LIST, (short)43);
   private static final org.apache.thrift.protocol.TField CELLS_PER_ROW_TO_CACHE_FIELD_DESC = new org.apache.thrift.protocol.TField("cells_per_row_to_cache", org.apache.thrift.protocol.TType.STRING, (short)44);
+  private static final org.apache.thrift.protocol.TField MIN_INDEX_INTERVAL_FIELD_DESC = new org.apache.thrift.protocol.TField("min_index_interval", org.apache.thrift.protocol.TType.I32, (short)45);
+  private static final org.apache.thrift.protocol.TField MAX_INDEX_INTERVAL_FIELD_DESC = new org.apache.thrift.protocol.TField("max_index_interval", org.apache.thrift.protocol.TType.I32, (short)46);
   private static final org.apache.thrift.protocol.TField ROW_CACHE_SIZE_FIELD_DESC = new org.apache.thrift.protocol.TField("row_cache_size", org.apache.thrift.protocol.TType.DOUBLE, (short)9);
   private static final org.apache.thrift.protocol.TField KEY_CACHE_SIZE_FIELD_DESC = new org.apache.thrift.protocol.TField("key_cache_size", org.apache.thrift.protocol.TType.DOUBLE, (short)11);
   private static final org.apache.thrift.protocol.TField ROW_CACHE_SAVE_PERIOD_IN_SECONDS_FIELD_DESC = new org.apache.thrift.protocol.TField("row_cache_save_period_in_seconds", org.apache.thrift.protocol.TType.I32, (short)19);
@@ -96,6 +97,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
   private static final org.apache.thrift.protocol.TField MERGE_SHARDS_CHANCE_FIELD_DESC = new org.apache.thrift.protocol.TField("merge_shards_chance", org.apache.thrift.protocol.TType.DOUBLE, (short)25);
   private static final org.apache.thrift.protocol.TField ROW_CACHE_PROVIDER_FIELD_DESC = new org.apache.thrift.protocol.TField("row_cache_provider", org.apache.thrift.protocol.TType.STRING, (short)27);
   private static final org.apache.thrift.protocol.TField ROW_CACHE_KEYS_TO_SAVE_FIELD_DESC = new org.apache.thrift.protocol.TField("row_cache_keys_to_save", org.apache.thrift.protocol.TType.I32, (short)31);
+  private static final org.apache.thrift.protocol.TField INDEX_INTERVAL_FIELD_DESC = new org.apache.thrift.protocol.TField("index_interval", org.apache.thrift.protocol.TType.I32, (short)41);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -127,10 +129,11 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
   public boolean populate_io_cache_on_flush; // optional
   public int memtable_flush_period_in_ms; // optional
   public int default_time_to_live; // optional
-  public int index_interval; // optional
   public String speculative_retry; // optional
   public List<TriggerDef> triggers; // optional
   public String cells_per_row_to_cache; // optional
+  public int min_index_interval; // optional
+  public int max_index_interval; // optional
   /**
    * @deprecated
    */
@@ -175,6 +178,10 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
    * @deprecated
    */
   public int row_cache_keys_to_save; // optional
+  /**
+   * @deprecated
+   */
+  public int index_interval; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -202,10 +209,11 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     POPULATE_IO_CACHE_ON_FLUSH((short)38, "populate_io_cache_on_flush"),
     MEMTABLE_FLUSH_PERIOD_IN_MS((short)39, "memtable_flush_period_in_ms"),
     DEFAULT_TIME_TO_LIVE((short)40, "default_time_to_live"),
-    INDEX_INTERVAL((short)41, "index_interval"),
     SPECULATIVE_RETRY((short)42, "speculative_retry"),
     TRIGGERS((short)43, "triggers"),
     CELLS_PER_ROW_TO_CACHE((short)44, "cells_per_row_to_cache"),
+    MIN_INDEX_INTERVAL((short)45, "min_index_interval"),
+    MAX_INDEX_INTERVAL((short)46, "max_index_interval"),
     /**
      * @deprecated
      */
@@ -249,7 +257,11 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     /**
      * @deprecated
      */
-    ROW_CACHE_KEYS_TO_SAVE((short)31, "row_cache_keys_to_save");
+    ROW_CACHE_KEYS_TO_SAVE((short)31, "row_cache_keys_to_save"),
+    /**
+     * @deprecated
+     */
+    INDEX_INTERVAL((short)41, "index_interval");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -312,14 +324,16 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
           return MEMTABLE_FLUSH_PERIOD_IN_MS;
         case 40: // DEFAULT_TIME_TO_LIVE
           return DEFAULT_TIME_TO_LIVE;
-        case 41: // INDEX_INTERVAL
-          return INDEX_INTERVAL;
         case 42: // SPECULATIVE_RETRY
           return SPECULATIVE_RETRY;
         case 43: // TRIGGERS
           return TRIGGERS;
         case 44: // CELLS_PER_ROW_TO_CACHE
           return CELLS_PER_ROW_TO_CACHE;
+        case 45: // MIN_INDEX_INTERVAL
+          return MIN_INDEX_INTERVAL;
+        case 46: // MAX_INDEX_INTERVAL
+          return MAX_INDEX_INTERVAL;
         case 9: // ROW_CACHE_SIZE
           return ROW_CACHE_SIZE;
         case 11: // KEY_CACHE_SIZE
@@ -342,6 +356,8 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
           return ROW_CACHE_PROVIDER;
         case 31: // ROW_CACHE_KEYS_TO_SAVE
           return ROW_CACHE_KEYS_TO_SAVE;
+        case 41: // INDEX_INTERVAL
+          return INDEX_INTERVAL;
         default:
           return null;
       }
@@ -392,19 +408,21 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
   private static final int __POPULATE_IO_CACHE_ON_FLUSH_ISSET_ID = 7;
   private static final int __MEMTABLE_FLUSH_PERIOD_IN_MS_ISSET_ID = 8;
   private static final int __DEFAULT_TIME_TO_LIVE_ISSET_ID = 9;
-  private static final int __INDEX_INTERVAL_ISSET_ID = 10;
-  private static final int __ROW_CACHE_SIZE_ISSET_ID = 11;
-  private static final int __KEY_CACHE_SIZE_ISSET_ID = 12;
-  private static final int __ROW_CACHE_SAVE_PERIOD_IN_SECONDS_ISSET_ID = 13;
-  private static final int __KEY_CACHE_SAVE_PERIOD_IN_SECONDS_ISSET_ID = 14;
-  private static final int __MEMTABLE_FLUSH_AFTER_MINS_ISSET_ID = 15;
-  private static final int __MEMTABLE_THROUGHPUT_IN_MB_ISSET_ID = 16;
-  private static final int __MEMTABLE_OPERATIONS_IN_MILLIONS_ISSET_ID = 17;
-  private static final int __REPLICATE_ON_WRITE_ISSET_ID = 18;
-  private static final int __MERGE_SHARDS_CHANCE_ISSET_ID = 19;
-  private static final int __ROW_CACHE_KEYS_TO_SAVE_ISSET_ID = 20;
+  private static final int __MIN_INDEX_INTERVAL_ISSET_ID = 10;
+  private static final int __MAX_INDEX_INTERVAL_ISSET_ID = 11;
+  private static final int __ROW_CACHE_SIZE_ISSET_ID = 12;
+  private static final int __KEY_CACHE_SIZE_ISSET_ID = 13;
+  private static final int __ROW_CACHE_SAVE_PERIOD_IN_SECONDS_ISSET_ID = 14;
+  private static final int __KEY_CACHE_SAVE_PERIOD_IN_SECONDS_ISSET_ID = 15;
+  private static final int __MEMTABLE_FLUSH_AFTER_MINS_ISSET_ID = 16;
+  private static final int __MEMTABLE_THROUGHPUT_IN_MB_ISSET_ID = 17;
+  private static final int __MEMTABLE_OPERATIONS_IN_MILLIONS_ISSET_ID = 18;
+  private static final int __REPLICATE_ON_WRITE_ISSET_ID = 19;
+  private static final int __MERGE_SHARDS_CHANCE_ISSET_ID = 20;
+  private static final int __ROW_CACHE_KEYS_TO_SAVE_ISSET_ID = 21;
+  private static final int __INDEX_INTERVAL_ISSET_ID = 22;
   private int __isset_bitfield = 0;
-  private _Fields optionals[] = {_Fields.COLUMN_TYPE,_Fields.COMPARATOR_TYPE,_Fields.SUBCOMPARATOR_TYPE,_Fields.COMMENT,_Fields.READ_REPAIR_CHANCE,_Fields.COLUMN_METADATA,_Fields.GC_GRACE_SECONDS,_Fields.DEFAULT_VALIDATION_CLASS,_Fields.ID,_Fields.MIN_COMPACTION_THRESHOLD,_Fields.MAX_COMPACTION_THRESHOLD,_Fields.KEY_VALIDATION_CLASS,_Fields.KEY_ALIAS,_Fields.COMPACTION_STRATEGY,_Fields.COMPACTION_STRATEGY_OPTIONS,_Fields.COMPRESSION_OPTIONS,_Fields.BLOOM_FILTER_FP_CHANCE,_Fields.CACHING,_Fields.DCLOCAL_READ_REPAIR_CHANCE,_Fields.POPULATE_IO_CACHE_ON_FLUSH,_Fields.MEMTABLE_FLUSH_PERIOD_IN_MS,_Fields.DEFAULT_TIME_TO_LIVE,_Fields.INDEX_INTERVAL,_Fields.SPECULATIVE_RETRY,_Fields.TRIGGERS,_Fields.CELLS_PER_ROW_TO_CACHE,_Fields.ROW_CACHE_SIZE,_Fields.KEY_CACHE_SIZE,_Fields.ROW_CACHE_SAVE_PERIOD_IN_SECONDS,_Fields.KEY_CACHE_SAVE_PERIOD_IN_SECONDS,_Fields.MEMTABLE_FLUSH_AFTER_MINS,_Fields.MEMTABLE_THROUGHPUT_IN_MB,_Fields.MEMTABLE_OPERATIONS_IN_MILLIONS,_Fields.REPLICATE_ON_WRITE,_Fields.ME
 RGE_SHARDS_CHANCE,_Fields.ROW_CACHE_PROVIDER,_Fields.ROW_CACHE_KEYS_TO_SAVE};
+  private _Fields optionals[] = {_Fields.COLUMN_TYPE,_Fields.COMPARATOR_TYPE,_Fields.SUBCOMPARATOR_TYPE,_Fields.COMMENT,_Fields.READ_REPAIR_CHANCE,_Fields.COLUMN_METADATA,_Fields.GC_GRACE_SECONDS,_Fields.DEFAULT_VALIDATION_CLASS,_Fields.ID,_Fields.MIN_COMPACTION_THRESHOLD,_Fields.MAX_COMPACTION_THRESHOLD,_Fields.KEY_VALIDATION_CLASS,_Fields.KEY_ALIAS,_Fields.COMPACTION_STRATEGY,_Fields.COMPACTION_STRATEGY_OPTIONS,_Fields.COMPRESSION_OPTIONS,_Fields.BLOOM_FILTER_FP_CHANCE,_Fields.CACHING,_Fields.DCLOCAL_READ_REPAIR_CHANCE,_Fields.POPULATE_IO_CACHE_ON_FLUSH,_Fields.MEMTABLE_FLUSH_PERIOD_IN_MS,_Fields.DEFAULT_TIME_TO_LIVE,_Fields.SPECULATIVE_RETRY,_Fields.TRIGGERS,_Fields.CELLS_PER_ROW_TO_CACHE,_Fields.MIN_INDEX_INTERVAL,_Fields.MAX_INDEX_INTERVAL,_Fields.ROW_CACHE_SIZE,_Fields.KEY_CACHE_SIZE,_Fields.ROW_CACHE_SAVE_PERIOD_IN_SECONDS,_Fields.KEY_CACHE_SAVE_PERIOD_IN_SECONDS,_Fields.MEMTABLE_FLUSH_AFTER_MINS,_Fields.MEMTABLE_THROUGHPUT_IN_MB,_Fields.MEMTABLE_OPERATIONS_IN_MILLIONS,_Field
 s.REPLICATE_ON_WRITE,_Fields.MERGE_SHARDS_CHANCE,_Fields.ROW_CACHE_PROVIDER,_Fields.ROW_CACHE_KEYS_TO_SAVE,_Fields.INDEX_INTERVAL};
   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);
@@ -461,8 +479,6 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
     tmpMap.put(_Fields.DEFAULT_TIME_TO_LIVE, new org.apache.thrift.meta_data.FieldMetaData("default_time_to_live", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
-    tmpMap.put(_Fields.INDEX_INTERVAL, new org.apache.thrift.meta_data.FieldMetaData("index_interval", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
     tmpMap.put(_Fields.SPECULATIVE_RETRY, new org.apache.thrift.meta_data.FieldMetaData("speculative_retry", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.TRIGGERS, new org.apache.thrift.meta_data.FieldMetaData("triggers", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
@@ -470,6 +486,10 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
             new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TriggerDef.class))));
     tmpMap.put(_Fields.CELLS_PER_ROW_TO_CACHE, new org.apache.thrift.meta_data.FieldMetaData("cells_per_row_to_cache", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.MIN_INDEX_INTERVAL, new org.apache.thrift.meta_data.FieldMetaData("min_index_interval", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.MAX_INDEX_INTERVAL, new org.apache.thrift.meta_data.FieldMetaData("max_index_interval", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
     tmpMap.put(_Fields.ROW_CACHE_SIZE, new org.apache.thrift.meta_data.FieldMetaData("row_cache_size", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
     tmpMap.put(_Fields.KEY_CACHE_SIZE, new org.apache.thrift.meta_data.FieldMetaData("key_cache_size", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
@@ -492,6 +512,8 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.ROW_CACHE_KEYS_TO_SAVE, new org.apache.thrift.meta_data.FieldMetaData("row_cache_keys_to_save", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.INDEX_INTERVAL, new org.apache.thrift.meta_data.FieldMetaData("index_interval", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CfDef.class, metaDataMap);
   }
@@ -584,7 +606,6 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     this.populate_io_cache_on_flush = other.populate_io_cache_on_flush;
     this.memtable_flush_period_in_ms = other.memtable_flush_period_in_ms;
     this.default_time_to_live = other.default_time_to_live;
-    this.index_interval = other.index_interval;
     if (other.isSetSpeculative_retry()) {
       this.speculative_retry = other.speculative_retry;
     }
@@ -598,6 +619,8 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     if (other.isSetCells_per_row_to_cache()) {
       this.cells_per_row_to_cache = other.cells_per_row_to_cache;
     }
+    this.min_index_interval = other.min_index_interval;
+    this.max_index_interval = other.max_index_interval;
     this.row_cache_size = other.row_cache_size;
     this.key_cache_size = other.key_cache_size;
     this.row_cache_save_period_in_seconds = other.row_cache_save_period_in_seconds;
@@ -611,6 +634,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
       this.row_cache_provider = other.row_cache_provider;
     }
     this.row_cache_keys_to_save = other.row_cache_keys_to_save;
+    this.index_interval = other.index_interval;
   }
 
   public CfDef deepCopy() {
@@ -656,13 +680,15 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     this.memtable_flush_period_in_ms = 0;
     setDefault_time_to_liveIsSet(false);
     this.default_time_to_live = 0;
-    setIndex_intervalIsSet(false);
-    this.index_interval = 0;
     this.speculative_retry = "NONE";
 
     this.triggers = null;
     this.cells_per_row_to_cache = "100";
 
+    setMin_index_intervalIsSet(false);
+    this.min_index_interval = 0;
+    setMax_index_intervalIsSet(false);
+    this.max_index_interval = 0;
     setRow_cache_sizeIsSet(false);
     this.row_cache_size = 0.0;
     setKey_cache_sizeIsSet(false);
@@ -684,6 +710,8 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     this.row_cache_provider = null;
     setRow_cache_keys_to_saveIsSet(false);
     this.row_cache_keys_to_save = 0;
+    setIndex_intervalIsSet(false);
+    this.index_interval = 0;
   }
 
   public String getKeyspace() {
@@ -1299,29 +1327,6 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __DEFAULT_TIME_TO_LIVE_ISSET_ID, value);
   }
 
-  public int getIndex_interval() {
-    return this.index_interval;
-  }
-
-  public CfDef setIndex_interval(int index_interval) {
-    this.index_interval = index_interval;
-    setIndex_intervalIsSet(true);
-    return this;
-  }
-
-  public void unsetIndex_interval() {
-    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __INDEX_INTERVAL_ISSET_ID);
-  }
-
-  /** Returns true if field index_interval is set (has been assigned a value) and false otherwise */
-  public boolean isSetIndex_interval() {
-    return EncodingUtils.testBit(__isset_bitfield, __INDEX_INTERVAL_ISSET_ID);
-  }
-
-  public void setIndex_intervalIsSet(boolean value) {
-    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __INDEX_INTERVAL_ISSET_ID, value);
-  }
-
   public String getSpeculative_retry() {
     return this.speculative_retry;
   }
@@ -1409,6 +1414,52 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     }
   }
 
+  public int getMin_index_interval() {
+    return this.min_index_interval;
+  }
+
+  public CfDef setMin_index_interval(int min_index_interval) {
+    this.min_index_interval = min_index_interval;
+    setMin_index_intervalIsSet(true);
+    return this;
+  }
+
+  public void unsetMin_index_interval() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MIN_INDEX_INTERVAL_ISSET_ID);
+  }
+
+  /** Returns true if field min_index_interval is set (has been assigned a value) and false otherwise */
+  public boolean isSetMin_index_interval() {
+    return EncodingUtils.testBit(__isset_bitfield, __MIN_INDEX_INTERVAL_ISSET_ID);
+  }
+
+  public void setMin_index_intervalIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MIN_INDEX_INTERVAL_ISSET_ID, value);
+  }
+
+  public int getMax_index_interval() {
+    return this.max_index_interval;
+  }
+
+  public CfDef setMax_index_interval(int max_index_interval) {
+    this.max_index_interval = max_index_interval;
+    setMax_index_intervalIsSet(true);
+    return this;
+  }
+
+  public void unsetMax_index_interval() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAX_INDEX_INTERVAL_ISSET_ID);
+  }
+
+  /** Returns true if field max_index_interval is set (has been assigned a value) and false otherwise */
+  public boolean isSetMax_index_interval() {
+    return EncodingUtils.testBit(__isset_bitfield, __MAX_INDEX_INTERVAL_ISSET_ID);
+  }
+
+  public void setMax_index_intervalIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAX_INDEX_INTERVAL_ISSET_ID, value);
+  }
+
   /**
    * @deprecated
    */
@@ -1729,6 +1780,35 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ROW_CACHE_KEYS_TO_SAVE_ISSET_ID, value);
   }
 
+  /**
+   * @deprecated
+   */
+  public int getIndex_interval() {
+    return this.index_interval;
+  }
+
+  /**
+   * @deprecated
+   */
+  public CfDef setIndex_interval(int index_interval) {
+    this.index_interval = index_interval;
+    setIndex_intervalIsSet(true);
+    return this;
+  }
+
+  public void unsetIndex_interval() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __INDEX_INTERVAL_ISSET_ID);
+  }
+
+  /** Returns true if field index_interval is set (has been assigned a value) and false otherwise */
+  public boolean isSetIndex_interval() {
+    return EncodingUtils.testBit(__isset_bitfield, __INDEX_INTERVAL_ISSET_ID);
+  }
+
+  public void setIndex_intervalIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __INDEX_INTERVAL_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case KEYSPACE:
@@ -1923,14 +2003,6 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
       }
       break;
 
-    case INDEX_INTERVAL:
-      if (value == null) {
-        unsetIndex_interval();
-      } else {
-        setIndex_interval((Integer)value);
-      }
-      break;
-
     case SPECULATIVE_RETRY:
       if (value == null) {
         unsetSpeculative_retry();
@@ -1955,6 +2027,22 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
       }
       break;
 
+    case MIN_INDEX_INTERVAL:
+      if (value == null) {
+        unsetMin_index_interval();
+      } else {
+        setMin_index_interval((Integer)value);
+      }
+      break;
+
+    case MAX_INDEX_INTERVAL:
+      if (value == null) {
+        unsetMax_index_interval();
+      } else {
+        setMax_index_interval((Integer)value);
+      }
+      break;
+
     case ROW_CACHE_SIZE:
       if (value == null) {
         unsetRow_cache_size();
@@ -2043,6 +2131,14 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
       }
       break;
 
+    case INDEX_INTERVAL:
+      if (value == null) {
+        unsetIndex_interval();
+      } else {
+        setIndex_interval((Integer)value);
+      }
+      break;
+
     }
   }
 
@@ -2120,9 +2216,6 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     case DEFAULT_TIME_TO_LIVE:
       return Integer.valueOf(getDefault_time_to_live());
 
-    case INDEX_INTERVAL:
-      return Integer.valueOf(getIndex_interval());
-
     case SPECULATIVE_RETRY:
       return getSpeculative_retry();
 
@@ -2132,6 +2225,12 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     case CELLS_PER_ROW_TO_CACHE:
       return getCells_per_row_to_cache();
 
+    case MIN_INDEX_INTERVAL:
+      return Integer.valueOf(getMin_index_interval());
+
+    case MAX_INDEX_INTERVAL:
+      return Integer.valueOf(getMax_index_interval());
+
     case ROW_CACHE_SIZE:
       return Double.valueOf(getRow_cache_size());
 
@@ -2165,6 +2264,9 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     case ROW_CACHE_KEYS_TO_SAVE:
       return Integer.valueOf(getRow_cache_keys_to_save());
 
+    case INDEX_INTERVAL:
+      return Integer.valueOf(getIndex_interval());
+
     }
     throw new IllegalStateException();
   }
@@ -2224,14 +2326,16 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
       return isSetMemtable_flush_period_in_ms();
     case DEFAULT_TIME_TO_LIVE:
       return isSetDefault_time_to_live();
-    case INDEX_INTERVAL:
-      return isSetIndex_interval();
     case SPECULATIVE_RETRY:
       return isSetSpeculative_retry();
     case TRIGGERS:
       return isSetTriggers();
     case CELLS_PER_ROW_TO_CACHE:
       return isSetCells_per_row_to_cache();
+    case MIN_INDEX_INTERVAL:
+      return isSetMin_index_interval();
+    case MAX_INDEX_INTERVAL:
+      return isSetMax_index_interval();
     case ROW_CACHE_SIZE:
       return isSetRow_cache_size();
     case KEY_CACHE_SIZE:
@@ -2254,6 +2358,8 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
       return isSetRow_cache_provider();
     case ROW_CACHE_KEYS_TO_SAVE:
       return isSetRow_cache_keys_to_save();
+    case INDEX_INTERVAL:
+      return isSetIndex_interval();
     }
     throw new IllegalStateException();
   }
@@ -2487,15 +2593,6 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
         return false;
     }
 
-    boolean this_present_index_interval = true && this.isSetIndex_interval();
-    boolean that_present_index_interval = true && that.isSetIndex_interval();
-    if (this_present_index_interval || that_present_index_interval) {
-      if (!(this_present_index_interval && that_present_index_interval))
-        return false;
-      if (this.index_interval != that.index_interval)
-        return false;
-    }
-
     boolean this_present_speculative_retry = true && this.isSetSpeculative_retry();
     boolean that_present_speculative_retry = true && that.isSetSpeculative_retry();
     if (this_present_speculative_retry || that_present_speculative_retry) {
@@ -2523,6 +2620,24 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
         return false;
     }
 
+    boolean this_present_min_index_interval = true && this.isSetMin_index_interval();
+    boolean that_present_min_index_interval = true && that.isSetMin_index_interval();
+    if (this_present_min_index_interval || that_present_min_index_interval) {
+      if (!(this_present_min_index_interval && that_present_min_index_interval))
+        return false;
+      if (this.min_index_interval != that.min_index_interval)
+        return false;
+    }
+
+    boolean this_present_max_index_interval = true && this.isSetMax_index_interval();
+    boolean that_present_max_index_interval = true && that.isSetMax_index_interval();
+    if (this_present_max_index_interval || that_present_max_index_interval) {
+      if (!(this_present_max_index_interval && that_present_max_index_interval))
+        return false;
+      if (this.max_index_interval != that.max_index_interval)
+        return false;
+    }
+
     boolean this_present_row_cache_size = true && this.isSetRow_cache_size();
     boolean that_present_row_cache_size = true && that.isSetRow_cache_size();
     if (this_present_row_cache_size || that_present_row_cache_size) {
@@ -2622,6 +2737,15 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
         return false;
     }
 
+    boolean this_present_index_interval = true && this.isSetIndex_interval();
+    boolean that_present_index_interval = true && that.isSetIndex_interval();
+    if (this_present_index_interval || that_present_index_interval) {
+      if (!(this_present_index_interval && that_present_index_interval))
+        return false;
+      if (this.index_interval != that.index_interval)
+        return false;
+    }
+
     return true;
   }
 
@@ -2749,11 +2873,6 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     if (present_default_time_to_live)
       builder.append(default_time_to_live);
 
-    boolean present_index_interval = true && (isSetIndex_interval());
-    builder.append(present_index_interval);
-    if (present_index_interval)
-      builder.append(index_interval);
-
     boolean present_speculative_retry = true && (isSetSpeculative_retry());
     builder.append(present_speculative_retry);
     if (present_speculative_retry)
@@ -2769,6 +2888,16 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     if (present_cells_per_row_to_cache)
       builder.append(cells_per_row_to_cache);
 
+    boolean present_min_index_interval = true && (isSetMin_index_interval());
+    builder.append(present_min_index_interval);
+    if (present_min_index_interval)
+      builder.append(min_index_interval);
+
+    boolean present_max_index_interval = true && (isSetMax_index_interval());
+    builder.append(present_max_index_interval);
+    if (present_max_index_interval)
+      builder.append(max_index_interval);
+
     boolean present_row_cache_size = true && (isSetRow_cache_size());
     builder.append(present_row_cache_size);
     if (present_row_cache_size)
@@ -2824,6 +2953,11 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     if (present_row_cache_keys_to_save)
       builder.append(row_cache_keys_to_save);
 
+    boolean present_index_interval = true && (isSetIndex_interval());
+    builder.append(present_index_interval);
+    if (present_index_interval)
+      builder.append(index_interval);
+
     return builder.toHashCode();
   }
 
@@ -3075,16 +3209,6 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(isSetIndex_interval()).compareTo(other.isSetIndex_interval());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetIndex_interval()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.index_interval, other.index_interval);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
     lastComparison = Boolean.valueOf(isSetSpeculative_retry()).compareTo(other.isSetSpeculative_retry());
     if (lastComparison != 0) {
       return lastComparison;
@@ -3115,6 +3239,26 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetMin_index_interval()).compareTo(other.isSetMin_index_interval());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetMin_index_interval()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.min_index_interval, other.min_index_interval);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetMax_index_interval()).compareTo(other.isSetMax_index_interval());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetMax_index_interval()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.max_index_interval, other.max_index_interval);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     lastComparison = Boolean.valueOf(isSetRow_cache_size()).compareTo(other.isSetRow_cache_size());
     if (lastComparison != 0) {
       return lastComparison;
@@ -3225,6 +3369,16 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetIndex_interval()).compareTo(other.isSetIndex_interval());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetIndex_interval()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.index_interval, other.index_interval);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -3440,12 +3594,6 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
       sb.append(this.default_time_to_live);
       first = false;
     }
-    if (isSetIndex_interval()) {
-      if (!first) sb.append(", ");
-      sb.append("index_interval:");
-      sb.append(this.index_interval);
-      first = false;
-    }
     if (isSetSpeculative_retry()) {
       if (!first) sb.append(", ");
       sb.append("speculative_retry:");
@@ -3476,6 +3624,18 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
       }
       first = false;
     }
+    if (isSetMin_index_interval()) {
+      if (!first) sb.append(", ");
+      sb.append("min_index_interval:");
+      sb.append(this.min_index_interval);
+      first = false;
+    }
+    if (isSetMax_index_interval()) {
+      if (!first) sb.append(", ");
+      sb.append("max_index_interval:");
+      sb.append(this.max_index_interval);
+      first = false;
+    }
     if (isSetRow_cache_size()) {
       if (!first) sb.append(", ");
       sb.append("row_cache_size:");
@@ -3546,6 +3706,12 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
       sb.append(this.row_cache_keys_to_save);
       first = false;
     }
+    if (isSetIndex_interval()) {
+      if (!first) sb.append(", ");
+      sb.append("index_interval:");
+      sb.append(this.index_interval);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -3824,14 +3990,6 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 41: // INDEX_INTERVAL
-            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-              struct.index_interval = iprot.readI32();
-              struct.setIndex_intervalIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
           case 42: // SPECULATIVE_RETRY
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.speculative_retry = iprot.readString();
@@ -3867,6 +4025,22 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 45: // MIN_INDEX_INTERVAL
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.min_index_interval = iprot.readI32();
+              struct.setMin_index_intervalIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 46: // MAX_INDEX_INTERVAL
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.max_index_interval = iprot.readI32();
+              struct.setMax_index_intervalIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           case 9: // ROW_CACHE_SIZE
             if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
               struct.row_cache_size = iprot.readDouble();
@@ -3955,6 +4129,14 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 41: // INDEX_INTERVAL
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.index_interval = iprot.readI32();
+              struct.setIndex_intervalIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -4227,6 +4409,16 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
           oprot.writeFieldEnd();
         }
       }
+      if (struct.isSetMin_index_interval()) {
+        oprot.writeFieldBegin(MIN_INDEX_INTERVAL_FIELD_DESC);
+        oprot.writeI32(struct.min_index_interval);
+        oprot.writeFieldEnd();
+      }
+      if (struct.isSetMax_index_interval()) {
+        oprot.writeFieldBegin(MAX_INDEX_INTERVAL_FIELD_DESC);
+        oprot.writeI32(struct.max_index_interval);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -4313,52 +4505,58 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
       if (struct.isSetDefault_time_to_live()) {
         optionals.set(21);
       }
-      if (struct.isSetIndex_interval()) {
+      if (struct.isSetSpeculative_retry()) {
         optionals.set(22);
       }
-      if (struct.isSetSpeculative_retry()) {
+      if (struct.isSetTriggers()) {
         optionals.set(23);
       }
-      if (struct.isSetTriggers()) {
+      if (struct.isSetCells_per_row_to_cache()) {
         optionals.set(24);
       }
-      if (struct.isSetCells_per_row_to_cache()) {
+      if (struct.isSetMin_index_interval()) {
         optionals.set(25);
       }
-      if (struct.isSetRow_cache_size()) {
+      if (struct.isSetMax_index_interval()) {
         optionals.set(26);
       }
-      if (struct.isSetKey_cache_size()) {
+      if (struct.isSetRow_cache_size()) {
         optionals.set(27);
       }
-      if (struct.isSetRow_cache_save_period_in_seconds()) {
+      if (struct.isSetKey_cache_size()) {
         optionals.set(28);
       }
-      if (struct.isSetKey_cache_save_period_in_seconds()) {
+      if (struct.isSetRow_cache_save_period_in_seconds()) {
         optionals.set(29);
       }
-      if (struct.isSetMemtable_flush_after_mins()) {
+      if (struct.isSetKey_cache_save_period_in_seconds()) {
         optionals.set(30);
       }
-      if (struct.isSetMemtable_throughput_in_mb()) {
+      if (struct.isSetMemtable_flush_after_mins()) {
         optionals.set(31);
       }
-      if (struct.isSetMemtable_operations_in_millions()) {
+      if (struct.isSetMemtable_throughput_in_mb()) {
         optionals.set(32);
       }
-      if (struct.isSetReplicate_on_write()) {
+      if (struct.isSetMemtable_operations_in_millions()) {
         optionals.set(33);
       }
-      if (struct.isSetMerge_shards_chance()) {
+      if (struct.isSetReplicate_on_write()) {
         optionals.set(34);
       }
-      if (struct.isSetRow_cache_provider()) {
+      if (struct.isSetMerge_shards_chance()) {
         optionals.set(35);
       }
-      if (struct.isSetRow_cache_keys_to_save()) {
+      if (struct.isSetRow_cache_provider()) {
         optionals.set(36);
       }
-      oprot.writeBitSet(optionals, 37);
+      if (struct.isSetRow_cache_keys_to_save()) {
+        optionals.set(37);
+      }
+      if (struct.isSetIndex_interval()) {
+        optionals.set(38);
+      }
+      oprot.writeBitSet(optionals, 39);
       if (struct.isSetColumn_type()) {
         oprot.writeString(struct.column_type);
       }
@@ -4445,9 +4643,6 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
       if (struct.isSetDefault_time_to_live()) {
         oprot.writeI32(struct.default_time_to_live);
       }
-      if (struct.isSetIndex_interval()) {
-        oprot.writeI32(struct.index_interval);
-      }
       if (struct.isSetSpeculative_retry()) {
         oprot.writeString(struct.speculative_retry);
       }
@@ -4463,6 +4658,12 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
       if (struct.isSetCells_per_row_to_cache()) {
         oprot.writeString(struct.cells_per_row_to_cache);
       }
+      if (struct.isSetMin_index_interval()) {
+        oprot.writeI32(struct.min_index_interval);
+      }
+      if (struct.isSetMax_index_interval()) {
+        oprot.writeI32(struct.max_index_interval);
+      }
       if (struct.isSetRow_cache_size()) {
         oprot.writeDouble(struct.row_cache_size);
       }
@@ -4496,6 +4697,9 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
       if (struct.isSetRow_cache_keys_to_save()) {
         oprot.writeI32(struct.row_cache_keys_to_save);
       }
+      if (struct.isSetIndex_interval()) {
+        oprot.writeI32(struct.index_interval);
+      }
     }
 
     @Override
@@ -4505,7 +4709,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
       struct.setKeyspaceIsSet(true);
       struct.name = iprot.readString();
       struct.setNameIsSet(true);
-      BitSet incoming = iprot.readBitSet(37);
+      BitSet incoming = iprot.readBitSet(39);
       if (incoming.get(0)) {
         struct.column_type = iprot.readString();
         struct.setColumn_typeIsSet(true);
@@ -4627,14 +4831,10 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
         struct.setDefault_time_to_liveIsSet(true);
       }
       if (incoming.get(22)) {
-        struct.index_interval = iprot.readI32();
-        struct.setIndex_intervalIsSet(true);
-      }
-      if (incoming.get(23)) {
         struct.speculative_retry = iprot.readString();
         struct.setSpeculative_retryIsSet(true);
       }
-      if (incoming.get(24)) {
+      if (incoming.get(23)) {
         {
           org.apache.thrift.protocol.TList _list143 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
           struct.triggers = new ArrayList<TriggerDef>(_list143.size);
@@ -4648,54 +4848,66 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
         }
         struct.setTriggersIsSet(true);
       }
-      if (incoming.get(25)) {
+      if (incoming.get(24)) {
         struct.cells_per_row_to_cache = iprot.readString();
         struct.setCells_per_row_to_cacheIsSet(true);
       }
+      if (incoming.get(25)) {
+        struct.min_index_interval = iprot.readI32();
+        struct.setMin_index_intervalIsSet(true);
+      }
       if (incoming.get(26)) {
+        struct.max_index_interval = iprot.readI32();
+        struct.setMax_index_intervalIsSet(true);
+      }
+      if (incoming.get(27)) {
         struct.row_cache_size = iprot.readDouble();
         struct.setRow_cache_sizeIsSet(true);
       }
-      if (incoming.get(27)) {
+      if (incoming.get(28)) {
         struct.key_cache_size = iprot.readDouble();
         struct.setKey_cache_sizeIsSet(true);
       }
-      if (incoming.get(28)) {
+      if (incoming.get(29)) {
         struct.row_cache_save_period_in_seconds = iprot.readI32();
         struct.setRow_cache_save_period_in_secondsIsSet(true);
       }
-      if (incoming.get(29)) {
+      if (incoming.get(30)) {
         struct.key_cache_save_period_in_seconds = iprot.readI32();
         struct.setKey_cache_save_period_in_secondsIsSet(true);
       }
-      if (incoming.get(30)) {
+      if (incoming.get(31)) {
         struct.memtable_flush_after_mins = iprot.readI32();
         struct.setMemtable_flush_after_minsIsSet(true);
       }
-      if (incoming.get(31)) {
+      if (incoming.get(32)) {
         struct.memtable_throughput_in_mb = iprot.readI32();
         struct.setMemtable_throughput_in_mbIsSet(true);
       }
-      if (incoming.get(32)) {
+      if (incoming.get(33)) {
         struct.memtable_operations_in_millions = iprot.readDouble();
         struct.setMemtable_operations_in_millionsIsSet(true);
       }
-      if (incoming.get(33)) {
+      if (incoming.get(34)) {
         struct.replicate_on_write = iprot.readBool();
         struct.setReplicate_on_writeIsSet(true);
       }
-      if (incoming.get(34)) {
+      if (incoming.get(35)) {
         struct.merge_shards_chance = iprot.readDouble();
         struct.setMerge_shards_chanceIsSet(true);
       }
-      if (incoming.get(35)) {
+      if (incoming.get(36)) {
         struct.row_cache_provider = iprot.readString();
         struct.setRow_cache_providerIsSet(true);
       }
-      if (incoming.get(36)) {
+      if (incoming.get(37)) {
         struct.row_cache_keys_to_save = iprot.readI32();
         struct.setRow_cache_keys_to_saveIsSet(true);
       }
+      if (incoming.get(38)) {
+        struct.index_interval = iprot.readI32();
+        struct.setIndex_intervalIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/interface/thrift/gen-java/org/apache/cassandra/thrift/cassandraConstants.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/cassandraConstants.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/cassandraConstants.java
index a795fc4..81b0ffb 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/cassandraConstants.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/cassandraConstants.java
@@ -56,6 +56,6 @@ import org.slf4j.LoggerFactory;
 
 public class cassandraConstants {
 
-  public static final String VERSION = "19.38.1";
+  public static final String VERSION = "19.39.0";
 
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 1e1b4d5..cf26e6e 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -67,7 +67,8 @@ class Cql3ParsingRuleSet(CqlParsingRuleSet):
         ('comment', None),
         ('dclocal_read_repair_chance', 'local_read_repair_chance'),
         ('gc_grace_seconds', None),
-        ('index_interval', None),
+        ('min_index_interval', None),
+        ('max_index_interval', None),
         ('read_repair_chance', None),
         ('populate_io_cache_on_flush', None),
         ('default_time_to_live', None),
@@ -439,7 +440,7 @@ def cf_prop_val_completer(ctxt, cass):
     if this_opt in ('replicate_on_write', 'populate_io_cache_on_flush'):
         return ["'yes'", "'no'"]
     if this_opt in ('min_compaction_threshold', 'max_compaction_threshold',
-                    'gc_grace_seconds', 'index_interval'):
+                    'gc_grace_seconds', 'min_index_interval', 'max_index_interval'):
         return [Hint('<integer>')]
     return [Hint('<option_value>')]
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/pylib/cqlshlib/test/test_cqlsh_output.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/test/test_cqlsh_output.py b/pylib/cqlshlib/test/test_cqlsh_output.py
index 102a040..a8c1158 100644
--- a/pylib/cqlshlib/test/test_cqlsh_output.py
+++ b/pylib/cqlshlib/test/test_cqlsh_output.py
@@ -659,7 +659,8 @@ class TestCqlshOutput(BaseTestCase):
               comment='' AND
               dclocal_read_repair_chance=0.000000 AND
               gc_grace_seconds=864000 AND
-              index_interval=128 AND
+              min_index_interval=128 AND
+              max_index_interval=2048 AND
               read_repair_chance=0.100000 AND
               populate_io_cache_on_flush='false' AND
               default_time_to_live=0 AND

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index 5c09961..010a277 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -82,7 +82,8 @@ public final class CFMetaData
     public final static Caching DEFAULT_CACHING_STRATEGY = Caching.KEYS_ONLY;
     public final static int DEFAULT_DEFAULT_TIME_TO_LIVE = 0;
     public final static SpeculativeRetry DEFAULT_SPECULATIVE_RETRY = new SpeculativeRetry(SpeculativeRetry.RetryType.PERCENTILE, 0.99);
-    public final static int DEFAULT_INDEX_INTERVAL = 128;
+    public final static int DEFAULT_MIN_INDEX_INTERVAL = 128;
+    public final static int DEFAULT_MAX_INDEX_INTERVAL = 2048;
     public final static boolean DEFAULT_POPULATE_IO_CACHE_ON_FLUSH = false;
     public final static RowsPerPartitionToCache DEFAULT_ROWS_PER_PARTITION_TO_CACHE = new RowsPerPartitionToCache(100, RowsPerPartitionToCache.Type.HEAD);
 
@@ -145,6 +146,8 @@ public final class CFMetaData
                                                                     + "speculative_retry text,"
                                                                     + "populate_io_cache_on_flush boolean,"
                                                                     + "index_interval int,"
+                                                                    + "min_index_interval int,"
+                                                                    + "max_index_interval int,"
                                                                     + "dropped_columns map<text, bigint>,"
                                                                     + "rows_per_partition_to_cache text,"
                                                                     + "PRIMARY KEY (keyspace_name, columnfamily_name)"
@@ -444,7 +447,8 @@ public final class CFMetaData
     private volatile int maxCompactionThreshold = DEFAULT_MAX_COMPACTION_THRESHOLD;
     private volatile Double bloomFilterFpChance = null;
     private volatile Caching caching = DEFAULT_CACHING_STRATEGY;
-    private volatile int indexInterval = DEFAULT_INDEX_INTERVAL;
+    private volatile int minIndexInterval = DEFAULT_MIN_INDEX_INTERVAL;
+    private volatile int maxIndexInterval = DEFAULT_MAX_INDEX_INTERVAL;
     private int memtableFlushPeriod = 0;
     private volatile int defaultTimeToLive = DEFAULT_DEFAULT_TIME_TO_LIVE;
     private volatile SpeculativeRetry speculativeRetry = DEFAULT_SPECULATIVE_RETRY;
@@ -475,6 +479,7 @@ public final class CFMetaData
 
     public volatile CompressionParameters compressionParameters = new CompressionParameters(null);
 
+    // attribute setters that return the modified CFMetaData instance
     public CFMetaData comment(String prop) { comment = enforceCommentNotNull(prop); return this;}
     public CFMetaData readRepairChance(double prop) {readRepairChance = prop; return this;}
     public CFMetaData dcLocalReadRepairChance(double prop) {dcLocalReadRepairChance = prop; return this;}
@@ -488,7 +493,8 @@ public final class CFMetaData
     public CFMetaData compressionParameters(CompressionParameters prop) {compressionParameters = prop; return this;}
     public CFMetaData bloomFilterFpChance(Double prop) {bloomFilterFpChance = prop; return this;}
     public CFMetaData caching(Caching prop) {caching = prop; return this;}
-    public CFMetaData indexInterval(int prop) {indexInterval = prop; return this;}
+    public CFMetaData minIndexInterval(int prop) {minIndexInterval = prop; return this;}
+    public CFMetaData maxIndexInterval(int prop) {maxIndexInterval = prop; return this;}
     public CFMetaData memtableFlushPeriod(int prop) {memtableFlushPeriod = prop; return this;}
     public CFMetaData defaultTimeToLive(int prop) {defaultTimeToLive = prop; return this;}
     public CFMetaData speculativeRetry(SpeculativeRetry prop) {speculativeRetry = prop; return this;}
@@ -676,7 +682,8 @@ public final class CFMetaData
                       .bloomFilterFpChance(oldCFMD.bloomFilterFpChance)
                       .caching(oldCFMD.caching)
                       .defaultTimeToLive(oldCFMD.defaultTimeToLive)
-                      .indexInterval(oldCFMD.indexInterval)
+                      .minIndexInterval(oldCFMD.minIndexInterval)
+                      .maxIndexInterval(oldCFMD.maxIndexInterval)
                       .speculativeRetry(oldCFMD.speculativeRetry)
                       .memtableFlushPeriod(oldCFMD.memtableFlushPeriod)
                       .populateIoCacheOnFlush(oldCFMD.populateIoCacheOnFlush)
@@ -860,14 +867,19 @@ public final class CFMetaData
         return caching;
     }
 
-    public RowsPerPartitionToCache getRowsPerPartitionToCache()
+    public int getMinIndexInterval()
     {
-        return rowsPerPartitionToCache;
+        return minIndexInterval;
+    }
+
+    public int getMaxIndexInterval()
+    {
+        return maxIndexInterval;
     }
 
-    public int getIndexInterval()
+    public RowsPerPartitionToCache getRowsPerPartitionToCache()
     {
-        return indexInterval;
+        return rowsPerPartitionToCache;
     }
 
     public SpeculativeRetry getSpeculativeRetry()
@@ -922,7 +934,8 @@ public final class CFMetaData
             && Objects.equal(memtableFlushPeriod, other.memtableFlushPeriod)
             && Objects.equal(caching, other.caching)
             && Objects.equal(defaultTimeToLive, other.defaultTimeToLive)
-            && Objects.equal(indexInterval, other.indexInterval)
+            && Objects.equal(minIndexInterval, other.minIndexInterval)
+            && Objects.equal(maxIndexInterval, other.maxIndexInterval)
             && Objects.equal(speculativeRetry, other.speculativeRetry)
             && Objects.equal(populateIoCacheOnFlush, other.populateIoCacheOnFlush)
             && Objects.equal(droppedColumns, other.droppedColumns)
@@ -955,7 +968,8 @@ public final class CFMetaData
             .append(memtableFlushPeriod)
             .append(caching)
             .append(defaultTimeToLive)
-            .append(indexInterval)
+            .append(minIndexInterval)
+            .append(maxIndexInterval)
             .append(speculativeRetry)
             .append(populateIoCacheOnFlush)
             .append(droppedColumns)
@@ -1017,6 +1031,21 @@ public final class CFMetaData
             cf_def.setDefault_time_to_live(CFMetaData.DEFAULT_DEFAULT_TIME_TO_LIVE);
         if (!cf_def.isSetDclocal_read_repair_chance())
             cf_def.setDclocal_read_repair_chance(CFMetaData.DEFAULT_DCLOCAL_READ_REPAIR_CHANCE);
+
+        // if index_interval was set, use that for the min_index_interval default
+        if (!cf_def.isSetMin_index_interval())
+        {
+            if (cf_def.isSetIndex_interval())
+                cf_def.setMin_index_interval(cf_def.getIndex_interval());
+            else
+                cf_def.setMin_index_interval(CFMetaData.DEFAULT_MIN_INDEX_INTERVAL);
+        }
+        if (!cf_def.isSetMax_index_interval())
+        {
+            // ensure the max is at least as large as the min
+            cf_def.setMax_index_interval(Math.max(cf_def.min_index_interval, CFMetaData.DEFAULT_MAX_INDEX_INTERVAL));
+        }
+
         if (!cf_def.isSetCells_per_row_to_cache())
             cf_def.setCells_per_row_to_cache(CFMetaData.DEFAULT_ROWS_PER_PARTITION_TO_CACHE.toString());
     }
@@ -1069,8 +1098,10 @@ public final class CFMetaData
                 newCFMD.defaultTimeToLive(cf_def.default_time_to_live);
             if (cf_def.isSetDclocal_read_repair_chance())
                 newCFMD.dcLocalReadRepairChance(cf_def.dclocal_read_repair_chance);
-            if (cf_def.isSetIndex_interval())
-                newCFMD.indexInterval(cf_def.index_interval);
+            if (cf_def.isSetMin_index_interval())
+                newCFMD.minIndexInterval(cf_def.min_index_interval);
+            if (cf_def.isSetMax_index_interval())
+                newCFMD.maxIndexInterval(cf_def.max_index_interval);
             if (cf_def.isSetSpeculative_retry())
                 newCFMD.speculativeRetry(SpeculativeRetry.fromString(cf_def.speculative_retry));
             if (cf_def.isSetPopulate_io_cache_on_flush())
@@ -1172,8 +1203,10 @@ public final class CFMetaData
         maxCompactionThreshold = cfm.maxCompactionThreshold;
 
         bloomFilterFpChance = cfm.bloomFilterFpChance;
-        memtableFlushPeriod = cfm.memtableFlushPeriod;
         caching = cfm.caching;
+        minIndexInterval = cfm.minIndexInterval;
+        maxIndexInterval = cfm.maxIndexInterval;
+        memtableFlushPeriod = cfm.memtableFlushPeriod;
         rowsPerPartitionToCache = cfm.rowsPerPartitionToCache;
         defaultTimeToLive = cfm.defaultTimeToLive;
         speculativeRetry = cfm.speculativeRetry;
@@ -1320,7 +1353,8 @@ public final class CFMetaData
         def.setCompression_options(compressionParameters.asThriftOptions());
         if (bloomFilterFpChance != null)
             def.setBloom_filter_fp_chance(bloomFilterFpChance);
-        def.setIndex_interval(indexInterval);
+        def.setMin_index_interval(minIndexInterval);
+        def.setMax_index_interval(maxIndexInterval);
         def.setMemtable_flush_period_in_ms(memtableFlushPeriod);
         def.setCaching(caching.toString());
         def.setCells_per_row_to_cache(rowsPerPartitionToCache.toString());
@@ -1514,6 +1548,8 @@ public final class CFMetaData
         if (bloomFilterFpChance != null && bloomFilterFpChance == 0)
             throw new ConfigurationException("Zero false positives is impossible; bloom filter false positive chance bffpc must be 0 < bffpc <= 1");
 
+        validateIndexIntervalThresholds();
+
         return this;
     }
 
@@ -1546,6 +1582,15 @@ public final class CFMetaData
                                                             minCompactionThreshold, maxCompactionThreshold));
     }
 
+    private void validateIndexIntervalThresholds() throws ConfigurationException
+    {
+        if (minIndexInterval <= 0)
+            throw new ConfigurationException(String.format("Min index interval must be greater than 0 (got %d).", minIndexInterval));
+        if (maxIndexInterval < minIndexInterval)
+            throw new ConfigurationException(String.format("Max index interval (%d) must be greater than the min index " +
+                                                           "interval (%d).", maxIndexInterval, minIndexInterval));
+    }
+
     /**
      * Create schema mutations to update this metadata to provided new state.
      *
@@ -1683,7 +1728,9 @@ public final class CFMetaData
         adder.add("compaction_strategy_class", compactionStrategyClass.getName());
         adder.add("compression_parameters", json(compressionParameters.asThriftOptions()));
         adder.add("compaction_strategy_options", json(compactionStrategyOptions));
-        adder.add("index_interval", indexInterval);
+        adder.add("min_index_interval", minIndexInterval);
+        adder.add("max_index_interval", maxIndexInterval);
+        adder.add("index_interval", null);
         adder.add("speculative_retry", speculativeRetry.toString());
 
         for (Map.Entry<ColumnIdentifier, Long> entry : droppedColumns.entrySet())
@@ -1753,18 +1800,16 @@ public final class CFMetaData
             cfm.compactionStrategyClass(createCompactionStrategy(result.getString("compaction_strategy_class")));
             cfm.compressionParameters(CompressionParameters.create(fromJsonMap(result.getString("compression_parameters"))));
             cfm.compactionStrategyOptions(fromJsonMap(result.getString("compaction_strategy_options")));
-            if (result.has("index_interval"))
-            {
-                cfm.indexInterval(result.getInt("index_interval"));
-            }
-            else
-            {
-                if (DatabaseDescriptor.getIndexInterval() != null)
-                {
-                    // use index_interval set in cassandra.yaml as default value (in memory only)
-                    cfm.indexInterval(DatabaseDescriptor.getIndexInterval());
-                }
-            }
+
+            // migrate old index_interval values to min_index_interval, if present
+            if (result.has("min_index_interval"))
+                cfm.minIndexInterval(result.getInt("min_index_interval"));
+            else if (result.has("index_interval"))
+                cfm.minIndexInterval(result.getInt("index_interval"));
+
+            if (result.has("max_index_interval"))
+                cfm.maxIndexInterval(result.getInt("max_index_interval"));
+
             if (result.has("populate_io_cache_on_flush"))
                 cfm.populateIoCacheOnFlush(result.getBoolean("populate_io_cache_on_flush"));
 
@@ -2226,7 +2271,8 @@ public final class CFMetaData
             .append("memtableFlushPeriod", memtableFlushPeriod)
             .append("caching", caching)
             .append("defaultTimeToLive", defaultTimeToLive)
-            .append("indexInterval", indexInterval)
+            .append("minIndexInterval", minIndexInterval)
+            .append("maxIndexInterval", maxIndexInterval)
             .append("speculativeRetry", speculativeRetry)
             .append("populateIoCacheOnFlush", populateIoCacheOnFlush)
             .append("droppedColumns", droppedColumns)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java b/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
index 50fabf3..f473e22 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
@@ -38,7 +38,8 @@ public class CFPropDefs extends PropertyDefinitions
     public static final String KW_CACHING = "caching";
     public static final String KW_ROWS_PER_PARTITION_TO_CACHE = "rows_per_partition_to_cache";
     public static final String KW_DEFAULT_TIME_TO_LIVE = "default_time_to_live";
-    public static final String KW_INDEX_INTERVAL = "index_interval";
+    public static final String KW_MIN_INDEX_INTERVAL = "min_index_interval";
+    public static final String KW_MAX_INDEX_INTERVAL = "max_index_interval";
     public static final String KW_SPECULATIVE_RETRY = "speculative_retry";
     public static final String KW_POPULATE_IO_CACHE_ON_FLUSH = "populate_io_cache_on_flush";
     public static final String KW_BF_FP_CHANCE = "bloom_filter_fp_chance";
@@ -61,7 +62,8 @@ public class CFPropDefs extends PropertyDefinitions
         keywords.add(KW_CACHING);
         keywords.add(KW_ROWS_PER_PARTITION_TO_CACHE);
         keywords.add(KW_DEFAULT_TIME_TO_LIVE);
-        keywords.add(KW_INDEX_INTERVAL);
+        keywords.add(KW_MIN_INDEX_INTERVAL);
+        keywords.add(KW_MAX_INDEX_INTERVAL);
         keywords.add(KW_SPECULATIVE_RETRY);
         keywords.add(KW_POPULATE_IO_CACHE_ON_FLUSH);
         keywords.add(KW_BF_FP_CHANCE);
@@ -69,6 +71,7 @@ public class CFPropDefs extends PropertyDefinitions
         keywords.add(KW_COMPRESSION);
         keywords.add(KW_MEMTABLE_FLUSH_PERIOD);
 
+        obsoleteKeywords.add("index_interval");
         obsoleteKeywords.add("replicate_on_write");
     }
 
@@ -115,7 +118,13 @@ public class CFPropDefs extends PropertyDefinitions
         }
 
         validateMinimumInt(KW_DEFAULT_TIME_TO_LIVE, 0, CFMetaData.DEFAULT_DEFAULT_TIME_TO_LIVE);
-        validateMinimumInt(KW_INDEX_INTERVAL, 1, CFMetaData.DEFAULT_INDEX_INTERVAL);
+
+        Integer minIndexInterval = getInt(KW_MIN_INDEX_INTERVAL, null);
+        Integer maxIndexInterval = getInt(KW_MAX_INDEX_INTERVAL, null);
+        if (minIndexInterval != null && minIndexInterval < 1)
+            throw new ConfigurationException(KW_MIN_INDEX_INTERVAL + " must be greater than 0");
+        if (maxIndexInterval != null && minIndexInterval != null && maxIndexInterval < minIndexInterval)
+            throw new ConfigurationException(KW_MAX_INDEX_INTERVAL + " must be greater than " + KW_MIN_INDEX_INTERVAL);
 
         SpeculativeRetry.fromString(getString(KW_SPECULATIVE_RETRY, SpeculativeRetry.RetryType.NONE.name()));
     }
@@ -165,7 +174,8 @@ public class CFPropDefs extends PropertyDefinitions
         cfm.speculativeRetry(CFMetaData.SpeculativeRetry.fromString(getString(KW_SPECULATIVE_RETRY, cfm.getSpeculativeRetry().toString())));
         cfm.memtableFlushPeriod(getInt(KW_MEMTABLE_FLUSH_PERIOD, cfm.getMemtableFlushPeriod()));
         cfm.populateIoCacheOnFlush(getBoolean(KW_POPULATE_IO_CACHE_ON_FLUSH, cfm.populateIoCacheOnFlush()));
-        cfm.indexInterval(getInt(KW_INDEX_INTERVAL, cfm.getIndexInterval()));
+        cfm.minIndexInterval(getInt(KW_MIN_INDEX_INTERVAL, cfm.getMinIndexInterval()));
+        cfm.maxIndexInterval(getInt(KW_MAX_INDEX_INTERVAL, cfm.getMaxIndexInterval()));
 
         if (compactionStrategyClass != null)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 0b3f64e..ca4ff0a 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -2293,6 +2293,14 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         return Iterables.concat(samples);
     }
 
+    public long estimatedKeysForRange(Range<Token> range)
+    {
+        long count = 0;
+        for (SSTableReader sstable : getSSTables())
+            count += sstable.estimatedKeysForRanges(Collections.singleton(range));
+        return count;
+    }
+
     /**
      * For testing.  No effort is made to clear historical or even the current memtables, nor for
      * thread safety.  All we do is wipe the sstable containers clean, while leaving the actual

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/src/java/org/apache/cassandra/db/SystemKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index 1a00cc0..292d3da 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -111,6 +111,8 @@ public class SystemKeyspace
     {
         setupVersion();
 
+        migrateIndexInterval();
+
         // add entries to system schema columnfamilies for the hardcoded system definitions
         for (String ksname : Schema.systemKeyspaceNames)
         {
@@ -143,6 +145,36 @@ public class SystemKeyspace
                                          DatabaseDescriptor.getPartitioner().getClass().getName()));
     }
 
+    // TODO: In 3.0, remove this and the index_interval column from system.schema_columnfamilies
+    /** Migrates index_interval values to min_index_interval and sets index_interval to null */
+    private static void migrateIndexInterval()
+    {
+        for (UntypedResultSet.Row row : processInternal(String.format("SELECT * FROM system.%s", SCHEMA_COLUMNFAMILIES_CF)))
+        {
+            if (!row.has("index_interval"))
+                continue;
+
+            logger.debug("Migrating index_interval to min_index_interval");
+
+            CFMetaData table = CFMetaData.fromSchema(row);
+            String query = String.format("SELECT writetime(type) "
+                    + "FROM system.%s "
+                    + "WHERE keyspace_name = '%s' AND columnfamily_name = '%s'",
+                    SCHEMA_COLUMNFAMILIES_CF,
+                    table.ksName,
+                    table.cfName);
+            long timestamp = processInternal(query).one().getLong("writetime(type)");
+            try
+            {
+                table.toSchema(timestamp).apply();
+            }
+            catch (ConfigurationException e)
+            {
+                // shouldn't happen
+            }
+        }
+    }
+
     /**
      * Write compaction log, except columfamilies under system keyspace.
      *

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index effaf1c..6c09efb 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -619,7 +619,7 @@ public class CompactionManager implements CompactionManagerMBean
 
             long totalkeysWritten = 0;
 
-            int expectedBloomFilterSize = Math.max(cfs.metadata.getIndexInterval(),
+            int expectedBloomFilterSize = Math.max(cfs.metadata.getMinIndexInterval(),
                                                    (int) (SSTableReader.getApproximateKeyCount(Arrays.asList(sstable))));
             if (logger.isDebugEnabled())
                 logger.debug("Expected bloom filter size : {}", expectedBloomFilterSize);
@@ -905,7 +905,7 @@ public class CompactionManager implements CompactionManagerMBean
         int repairedKeyCount = 0;
         int unrepairedKeyCount = 0;
         // TODO(5351): we can do better here:
-        int expectedBloomFilterSize = Math.max(cfs.metadata.getIndexInterval(), (int)(SSTableReader.getApproximateKeyCount(repairedSSTables)));
+        int expectedBloomFilterSize = Math.max(cfs.metadata.getMinIndexInterval(), (int)(SSTableReader.getApproximateKeyCount(repairedSSTables)));
         logger.info("Performing anticompaction on {} sstables", repairedSSTables.size());
         // iterate over sstables to check if the repaired / unrepaired ranges intersect them.
         for (SSTableReader sstable : repairedSSTables)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 84b22d3..b2cb641 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -132,7 +132,7 @@ public class CompactionTask extends AbstractCompactionTask
 
         long start = System.nanoTime();
         long totalKeysWritten = 0;
-        long estimatedTotalKeys = Math.max(cfs.metadata.getIndexInterval(), SSTableReader.getApproximateKeyCount(actuallyCompact));
+        long estimatedTotalKeys = Math.max(cfs.metadata.getMinIndexInterval(), SSTableReader.getApproximateKeyCount(actuallyCompact));
         long estimatedSSTables = Math.max(1, SSTableReader.getTotalBytes(actuallyCompact) / strategy.getMaxSSTableBytes());
         long keysPerSSTable = (long) Math.ceil((double) estimatedTotalKeys / estimatedSSTables);
         logger.debug("Expected bloom filter size : {}", keysPerSSTable);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/Scrubber.java b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
index ff04723..e1da811 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -88,7 +88,7 @@ public class Scrubber implements Closeable
                         ? new ScrubController(cfs)
                         : new CompactionController(cfs, Collections.singleton(sstable), CompactionManager.getDefaultGcBefore(cfs));
         this.isCommutative = cfs.metadata.isCounter();
-        this.expectedBloomFilterSize = Math.max(cfs.metadata.getIndexInterval(), (int)(SSTableReader.getApproximateKeyCount(toScrub)));
+        this.expectedBloomFilterSize = Math.max(cfs.metadata.getMinIndexInterval(), (int)(SSTableReader.getApproximateKeyCount(toScrub)));
 
         // loop through each row, deserializing to check for damage.
         // we'll also loop through the index at the same time, using the position from the index to recover if the

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/src/java/org/apache/cassandra/db/compaction/Upgrader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/Upgrader.java b/src/java/org/apache/cassandra/db/compaction/Upgrader.java
index 022a3c9..dfadfcd 100644
--- a/src/java/org/apache/cassandra/db/compaction/Upgrader.java
+++ b/src/java/org/apache/cassandra/db/compaction/Upgrader.java
@@ -56,7 +56,7 @@ public class Upgrader
         this.controller = new UpgradeController(cfs);
 
         this.strategy = cfs.getCompactionStrategy();
-        long estimatedTotalKeys = Math.max(cfs.metadata.getIndexInterval(), SSTableReader.getApproximateKeyCount(toUpgrade));
+        long estimatedTotalKeys = Math.max(cfs.metadata.getMinIndexInterval(), SSTableReader.getApproximateKeyCount(toUpgrade));
         long estimatedSSTables = Math.max(1, SSTableReader.getTotalBytes(this.toUpgrade) / strategy.getMaxSSTableBytes());
         this.estimatedRows = (long) Math.ceil((double) estimatedTotalKeys / estimatedSSTables);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java b/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
index 190851c..d812d3e 100644
--- a/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
@@ -199,7 +199,7 @@ public class OrderPreservingPartitioner extends AbstractPartitioner<StringToken>
                 for (Range<Token> r : sortedRanges)
                 {
                     // Looping over every KS:CF:Range, get the splits size and add it to the count
-                    allTokens.put(r.right, allTokens.get(r.right) + StorageService.instance.getSplits(ks, cfmd.cfName, r, cfmd.getIndexInterval(), cfmd).size());
+                    allTokens.put(r.right, allTokens.get(r.right) + StorageService.instance.getSplits(ks, cfmd.cfName, r, cfmd.getMinIndexInterval(), cfmd).size());
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ee477cc4/src/java/org/apache/cassandra/io/sstable/Downsampling.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/Downsampling.java b/src/java/org/apache/cassandra/io/sstable/Downsampling.java
index 62ca1be..c83db76 100644
--- a/src/java/org/apache/cassandra/io/sstable/Downsampling.java
+++ b/src/java/org/apache/cassandra/io/sstable/Downsampling.java
@@ -15,13 +15,6 @@ public class Downsampling
      */
     public static final int BASE_SAMPLING_LEVEL = 128;
 
-    /**
-     * The lowest level we will downsample to: the coarsest summary will have (MSL / BSL) entries left.
-     *
-     * This can be anywhere from 1 to the base sampling level.
-     */
-    public static final int MIN_SAMPLING_LEVEL = 8;
-
     private static final Map<Integer, List<Integer>> samplePatternCache = new HashMap<>();
 
     private static final Map<Integer, List<Integer>> originalIndexCache = new HashMap<>();
@@ -43,24 +36,22 @@ public class Downsampling
         if (samplingLevel <= 1)
             return Arrays.asList(0);
 
-        ArrayList<Integer> startIndices = new ArrayList<>(samplingLevel);
-        startIndices.add(0);
+        int[] odds = new int[samplingLevel / 2];
+        int[] evens = new int[samplingLevel / 2];
+        for (int i = 1; i < samplingLevel; i += 2)
+            odds[i/2] = i;
+        for (int i = 0; i < samplingLevel; i += 2)
+            evens[i/2] = i;
 
-        int spread = samplingLevel;
-        while (spread >= 2)
-        {
-            ArrayList<Integer> roundIndices = new ArrayList<>(samplingLevel / spread);
-            for (int i = spread / 2; i < samplingLevel; i += spread)
-                roundIndices.add(i);
-
-            // especially for latter rounds, it's important that we spread out the start points, so we'll
-            // make a recursive call to get an ordering for this list of start points
-            List<Integer> roundIndicesOrdering = getSamplingPattern(roundIndices.size());
-            for (int i = 0; i < roundIndices.size(); ++i)
-                startIndices.add(roundIndices.get(roundIndicesOrdering.get(i)));
+        // especially for latter rounds, it's important that we spread out the start points, so we'll
+        // make a recursive call to get an ordering for this list of start points
+        List<Integer> ordering = getSamplingPattern(samplingLevel/2);
+        List<Integer> startIndices = new ArrayList<>(samplingLevel);
 
-            spread /= 2;
-        }
+        for (Integer index : ordering)
+            startIndices.add(odds[index]);
+        for (Integer index : ordering)
+            startIndices.add(evens[index]);
 
         samplePatternCache.put(samplingLevel, startIndices);
         return startIndices;
@@ -101,25 +92,25 @@ public class Downsampling
      * the index summary.  If samplingLevel == BASE_SAMPLING_LEVEL, this will be equal to the index interval.
      * @param index an index into an IndexSummary
      * @param samplingLevel the current sampling level for that IndexSummary
-     * @param indexInterval the index interval
+     * @param minIndexInterval the min index interval (effective index interval at full sampling)
      * @return the number of partitions before the next index summary entry, inclusive on one end
      */
-    public static int getEffectiveIndexIntervalAfterIndex(int index, int samplingLevel, int indexInterval)
+    public static int getEffectiveIndexIntervalAfterIndex(int index, int samplingLevel, int minIndexInterval)
     {
         assert index >= -1;
         List<Integer> originalIndexes = getOriginalIndexes(samplingLevel);
         if (index == -1)
-            return originalIndexes.get(0) * indexInterval;
+            return originalIndexes.get(0) * minIndexInterval;
 
         index %= samplingLevel;
         if (index == originalIndexes.size() - 1)
         {
             // account for partitions after the "last" entry as well as partitions before the "first" entry
-            return ((BASE_SAMPLING_LEVEL - originalIndexes.get(index)) + originalIndexes.get(0)) * indexInterval;
+            return ((BASE_SAMPLING_LEVEL - originalIndexes.get(index)) + originalIndexes.get(0)) * minIndexInterval;
         }
         else
         {
-            return (originalIndexes.get(index + 1) - originalIndexes.get(index)) * indexInterval;
+            return (originalIndexes.get(index + 1) - originalIndexes.get(index)) * minIndexInterval;
         }
     }