You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by be...@apache.org on 2015/09/30 20:48:19 UTC

[01/16] cassandra git commit: Fix Mmapped File Boundaries

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 f6cab37d5 -> c37562e34
  refs/heads/cassandra-2.2 7452b2050 -> 25de92e32
  refs/heads/cassandra-3.0 a039b7df5 -> e3d58448b
  refs/heads/trunk 092281fee -> b06e703d7


Fix Mmapped File Boundaries

This patch fixes two bugs with mmap segment boundary
tracking, and introduces automated correction of
this bug on startup

patch by benedict; reviewed by tjake for CASSANDRA-10357


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

Branch: refs/heads/cassandra-2.1
Commit: c37562e345c24720c55428a8644191df68319812
Parents: f6cab37
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Wed Sep 16 18:09:32 2015 +0100
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Wed Sep 30 19:45:49 2015 +0100

----------------------------------------------------------------------
 .../io/sstable/AbstractSSTableSimpleWriter.java |   2 +
 .../cassandra/io/sstable/CQLSSTableWriter.java  |  15 +
 .../cassandra/io/sstable/SSTableReader.java     |  34 +-
 .../io/sstable/SSTableSimpleUnsortedWriter.java |   6 +
 .../io/sstable/SSTableSimpleWriter.java         |  12 +
 .../cassandra/io/util/MappedFileDataInput.java  |   8 +-
 .../cassandra/io/util/MmappedSegmentedFile.java | 270 +++++++++++++---
 .../apache/cassandra/io/util/SegmentedFile.java |   1 +
 .../sstable/LongSegmentedFileBoundaryTest.java  | 322 +++++++++++++++++++
 9 files changed, 601 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
index 165a4b2..557c3de 100644
--- a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
@@ -200,4 +200,6 @@ public abstract class AbstractSSTableSimpleWriter implements Closeable
     protected abstract void writeRow(DecoratedKey key, ColumnFamily columnFamily) throws IOException;
 
     protected abstract ColumnFamily getColumnFamily() throws IOException;
+
+    public abstract Descriptor getCurrentDescriptor();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
index b211a90..c364171 100644
--- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
@@ -271,6 +271,16 @@ public class CQLSSTableWriter implements Closeable
         writer.close();
     }
 
+    public Descriptor getCurrentDescriptor()
+    {
+        return writer.getCurrentDescriptor();
+    }
+
+    public CFMetaData getCFMetaData()
+    {
+        return writer.metadata;
+    }
+
     /**
      * A Builder for a CQLSSTableWriter object.
      */
@@ -366,6 +376,11 @@ public class CQLSSTableWriter implements Closeable
             }
         }
 
+        CFMetaData metadata()
+        {
+            return schema;
+        }
+
         /**
          * Adds the specified column family to the specified keyspace.
          *

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/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 0f307b0..84add6f 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
@@ -17,13 +17,7 @@
  */
 package org.apache.cassandra.io.sstable;
 
-import java.io.BufferedInputStream;
-import java.io.DataInputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.IOException;
+import java.io.*;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.*;
@@ -70,20 +64,14 @@ import org.apache.cassandra.dht.LocalPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.FSError;
+import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.io.compress.CompressionMetadata;
 import org.apache.cassandra.io.sstable.metadata.CompactionMetadata;
 import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
 import org.apache.cassandra.io.sstable.metadata.MetadataType;
 import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
 import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
-import org.apache.cassandra.io.util.BufferedSegmentedFile;
-import org.apache.cassandra.io.util.CompressedSegmentedFile;
-import org.apache.cassandra.io.util.DataOutputStreamAndChannel;
-import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.io.util.ICompressedFile;
-import org.apache.cassandra.io.util.RandomAccessReader;
-import org.apache.cassandra.io.util.SegmentedFile;
+import org.apache.cassandra.io.util.*;
 import org.apache.cassandra.metrics.RestorableMeter;
 import org.apache.cassandra.metrics.StorageMetrics;
 import org.apache.cassandra.service.ActiveRepairService;
@@ -162,6 +150,7 @@ import static org.apache.cassandra.db.Directories.SECONDARY_INDEX_NAME_SEPARATOR
 public class SSTableReader extends SSTable implements SelfRefCounted<SSTableReader>
 {
     private static final Logger logger = LoggerFactory.getLogger(SSTableReader.class);
+    private static final int ACCURATE_BOUNDARIES_MAGIC_NUMBER = 248923458;
 
     private static final ScheduledThreadPoolExecutor syncExecutor = new ScheduledThreadPoolExecutor(1);
     static
@@ -892,6 +881,19 @@ public class SSTableReader extends SSTable implements SelfRefCounted<SSTableRead
             last = partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
             ibuilder.deserializeBounds(iStream);
             dbuilder.deserializeBounds(iStream);
+
+            boolean checkForRepair = true;
+            try
+            {
+                int v = iStream.readInt();
+                // check for our magic number, indicating this summary has been sampled correctly
+                checkForRepair = v != ACCURATE_BOUNDARIES_MAGIC_NUMBER;
+            }
+            catch (Throwable t) {}
+
+            // fix CASSANDRA-10357 on-the-fly
+            if (checkForRepair && MmappedSegmentedFile.maybeRepair(metadata, descriptor, indexSummary, ibuilder, dbuilder))
+                saveSummary(ibuilder, dbuilder);
         }
         catch (IOException e)
         {
@@ -992,6 +994,8 @@ public class SSTableReader extends SSTable implements SelfRefCounted<SSTableRead
             ByteBufferUtil.writeWithLength(last.getKey(), oStream);
             ibuilder.serializeBounds(oStream);
             dbuilder.serializeBounds(oStream);
+            // write a magic number, to indicate this summary has been sampled correctly
+            oStream.writeInt(ACCURATE_BOUNDARIES_MAGIC_NUMBER);
         }
         catch (IOException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
index 9ee9ea1..25ec354 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
@@ -140,6 +140,12 @@ public class SSTableSimpleUnsortedWriter extends AbstractSSTableSimpleWriter
         return previous;
     }
 
+    public Descriptor getCurrentDescriptor()
+    {
+        // can be implemented, but isn't necessary
+        throw new UnsupportedOperationException();
+    }
+
     protected ColumnFamily createColumnFamily() throws IOException
     {
         return ArrayBackedSortedColumns.factory.create(metadata);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
index 87c8e33..23da501 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
@@ -65,6 +65,13 @@ public class SSTableSimpleWriter extends AbstractSSTableSimpleWriter
         writer = getWriter();
     }
 
+    SSTableReader closeAndOpenReader()
+    {
+        if (currentKey != null)
+            writeRow(currentKey, columnFamily);
+        return writer.closeAndOpenReader();
+    }
+
     public void close()
     {
         try
@@ -89,4 +96,9 @@ public class SSTableSimpleWriter extends AbstractSSTableSimpleWriter
     {
         return ArrayBackedSortedColumns.factory.create(metadata);
     }
+
+    public Descriptor getCurrentDescriptor()
+    {
+        return writer.descriptor;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java b/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java
index d056240..f93ce72 100644
--- a/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java
+++ b/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java
@@ -51,12 +51,18 @@ public class MappedFileDataInput extends AbstractDataInput implements FileDataIn
     public void seek(long pos) throws IOException
     {
         long inSegmentPos = pos - segmentOffset;
-        if (inSegmentPos < 0 || inSegmentPos > buffer.capacity())
+        if (!contains(pos))
             throw new IOException(String.format("Seek position %d is not within mmap segment (seg offs: %d, length: %d)", pos, segmentOffset, buffer.capacity()));
 
         seekInternal((int) inSegmentPos);
     }
 
+    public boolean contains(long pos)
+    {
+        long inSegmentPos = pos - segmentOffset;
+        return inSegmentPos >= 0 && inSegmentPos < buffer.capacity();
+    }
+
     public long getFilePointer()
     {
         return segmentOffset + (long)position;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
index 1b23343..623f65a 100644
--- a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
@@ -24,11 +24,17 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.RowIndexEntry;
 import org.apache.cassandra.io.FSReadError;
-import org.apache.cassandra.io.sstable.SSTableWriter;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummary;
+import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 
 public class MmappedSegmentedFile extends SegmentedFile
@@ -135,52 +141,220 @@ public class MmappedSegmentedFile extends SegmentedFile
         }
     }
 
+    // see CASSANDRA-10357
+    public static boolean maybeRepair(CFMetaData metadata, Descriptor descriptor, IndexSummary indexSummary, SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
+    {
+        boolean mayNeedRepair = false;
+        if (ibuilder instanceof Builder)
+            mayNeedRepair = ((Builder) ibuilder).mayNeedRepair(descriptor.filenameFor(Component.PRIMARY_INDEX));
+        if (dbuilder instanceof Builder)
+            mayNeedRepair |= ((Builder) dbuilder).mayNeedRepair(descriptor.filenameFor(Component.DATA));
+
+        if (mayNeedRepair)
+            forceRepair(metadata, descriptor, indexSummary, ibuilder, dbuilder);
+        return mayNeedRepair;
+    }
+
+    // if one of the index/data files have boundaries larger than we can mmap, and they were written by a version that did not guarantee correct boundaries were saved,
+    // rebuild the boundaries and save them again
+    private static void forceRepair(CFMetaData metadata, Descriptor descriptor, IndexSummary indexSummary, SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
+    {
+        if (ibuilder instanceof Builder)
+            ((Builder) ibuilder).boundaries.clear();
+        if (dbuilder instanceof Builder)
+            ((Builder) dbuilder).boundaries.clear();
+
+        try (RandomAccessFile raf = new RandomAccessFile(descriptor.filenameFor(Component.PRIMARY_INDEX), "r");)
+        {
+            long iprev = 0, dprev = 0;
+            for (int i = 0; i < indexSummary.size(); i++)
+            {
+                // first read the position in the summary, and read the corresponding position in the data file
+                long icur = indexSummary.getPosition(i);
+                raf.seek(icur);
+                ByteBufferUtil.readWithShortLength(raf);
+                RowIndexEntry rie = metadata.comparator.rowIndexEntrySerializer().deserialize(raf, descriptor.version);
+                long dcur = rie.position;
+
+                // if these positions are small enough to map out a segment from the prior version (i.e. less than 2Gb),
+                // just add these as a boundary and proceed to the next index summary record; most scenarios will be
+                // served by this, keeping the cost of rebuild to a minimum.
+
+                if (Math.max(icur - iprev , dcur - dprev) > MAX_SEGMENT_SIZE)
+                {
+                    // otherwise, loop over its index block, providing each RIE as a potential boundary for both files
+                    raf.seek(iprev);
+                    while (raf.getFilePointer() < icur)
+                    {
+                        // add the position of this record in the index file as an index file boundary
+                        ibuilder.addPotentialBoundary(raf.getFilePointer());
+                        // then read the RIE, and add its data file position as a boundary for the data file
+                        ByteBufferUtil.readWithShortLength(raf);
+                        rie = metadata.comparator.rowIndexEntrySerializer().deserialize(raf, descriptor.version);
+                        dbuilder.addPotentialBoundary(rie.position);
+                    }
+                }
+
+                ibuilder.addPotentialBoundary(icur);
+                dbuilder.addPotentialBoundary(dcur);
+
+                iprev = icur;
+                dprev = dcur;
+            }
+        }
+        catch (IOException e)
+        {
+            logger.error("Failed to recalculate boundaries for {}; mmap access may degrade to buffered for this file", descriptor);
+        }
+    }
+
     /**
      * Overrides the default behaviour to create segments of a maximum size.
      */
-    static class Builder extends SegmentedFile.Builder
+    public static class Builder extends SegmentedFile.Builder
     {
-        // planned segment boundaries
-        private List<Long> boundaries;
+        @VisibleForTesting
+        public static class Boundaries
+        {
+            private long[] boundaries;
+
+            // number of boundaries we have "fixed" (i.e. have determined the final value of)
+            private int fixedCount;
+
+            public Boundaries()
+            {
+                // we always have a boundary of zero, so we start with a fixedCount of 1
+                this(new long[8], 1);
+            }
+
+            public Boundaries(long[] boundaries, int fixedCount)
+            {
+                init(boundaries, fixedCount);
+            }
+
+            void init(long[] boundaries, int fixedCount)
+            {
+                this.boundaries = boundaries;
+                this.fixedCount = fixedCount;
+            }
+
+            public void addCandidate(long candidate)
+            {
+                // we make sure we have room before adding another element, so that we can share the addCandidate logic statically
+                boundaries = ensureCapacity(boundaries, fixedCount);
+                fixedCount = addCandidate(boundaries, fixedCount, candidate);
+            }
+
+            private static int addCandidate(long[] boundaries, int fixedCount, long candidate)
+            {
+                // check how far we are from the last fixed boundary
+                long delta = candidate - boundaries[fixedCount - 1];
+                assert delta >= 0;
+                if (delta != 0)
+                {
+                    if (delta <= MAX_SEGMENT_SIZE)
+                        // overwrite the unfixed (potential) boundary if the resultant segment would still be mmappable
+                        boundaries[fixedCount] = candidate;
+                    else if (boundaries[fixedCount] == 0)
+                        // or, if it is not initialised, we cannot make an mmapped segment here, so this is the fixed boundary
+                        boundaries[fixedCount++] = candidate;
+                    else
+                        // otherwise, fix the prior boundary and initialise our unfixed boundary
+                        boundaries[++fixedCount] = candidate;
+                }
+                return fixedCount;
+            }
+
+            // ensures there is room for another fixed boundary AND an unfixed candidate boundary, i.e. fixedCount + 2 items
+            private static long[] ensureCapacity(long[] boundaries, int fixedCount)
+            {
+                if (fixedCount + 1 >= boundaries.length)
+                    return Arrays.copyOf(boundaries, boundaries.length * 2);
+                return boundaries;
+            }
+
+            void clear()
+            {
+                fixedCount = 1;
+                Arrays.fill(boundaries, 0);
+            }
+
+            // returns the fixed boundaries, truncated to a correctly sized long[]
+            public long[] truncate()
+            {
+                return Arrays.copyOf(boundaries, fixedCount);
+            }
 
-        // offset of the open segment (first segment begins at 0).
-        private long currentStart = 0;
+            // returns the finished boundaries for the provided length, truncated to a correctly sized long[]
+            public long[] finish(long length, boolean isFinal)
+            {
+                assert length > 0;
+                // ensure there's room for the length to be added
+                boundaries = ensureCapacity(boundaries, fixedCount);
+
+                // clone our current contents, so we don't corrupt them
+                int fixedCount = this.fixedCount;
+                long[] boundaries = this.boundaries.clone();
+
+                // if we're finishing early, our length may be before some of our boundaries,
+                // so walk backwards until our boundaries are <= length
+                while (boundaries[fixedCount - 1] >= length)
+                    boundaries[fixedCount--] = 0;
+                if (boundaries[fixedCount] >= length)
+                    boundaries[fixedCount] = 0;
+
+                // add our length as a boundary
+                fixedCount = addCandidate(boundaries, fixedCount, length);
+
+                // if we have any unfixed boundary at the end, it's now fixed, since we're done
+                if (boundaries[fixedCount] != 0)
+                    fixedCount++;
+
+                boundaries = Arrays.copyOf(boundaries, fixedCount);
+                if (isFinal)
+                {
+                    // if this is the final one, save it
+                    this.boundaries = boundaries;
+                    this.fixedCount = fixedCount;
+                }
+                return boundaries;
+            }
+        }
 
-        // current length of the open segment.
-        // used to allow merging multiple too-large-to-mmap segments, into a single buffered segment.
-        private long currentSize = 0;
+        private final Boundaries boundaries = new Boundaries();
 
         public Builder()
         {
             super();
-            boundaries = new ArrayList<>();
-            boundaries.add(0L);
         }
 
-        public void addPotentialBoundary(long boundary)
+        public long[] boundaries()
         {
-            if (boundary - currentStart <= MAX_SEGMENT_SIZE)
-            {
-                // boundary fits into current segment: expand it
-                currentSize = boundary - currentStart;
-                return;
-            }
+            return boundaries.truncate();
+        }
 
-            // close the current segment to try and make room for the boundary
-            if (currentSize > 0)
-            {
-                currentStart += currentSize;
-                boundaries.add(currentStart);
-            }
-            currentSize = boundary - currentStart;
+        // indicates if we may need to repair the mmapped file boundaries. this is a cheap check to see if there
+        // are any spans larger than an mmap segment size, which should be rare to occur in practice.
+        boolean mayNeedRepair(String path)
+        {
+            // old boundaries were created without the length, so add it as a candidate
+            long length = new File(path).length();
+            boundaries.addCandidate(length);
+            long[] boundaries = this.boundaries.truncate();
 
-            // if we couldn't make room, the boundary needs its own segment
-            if (currentSize > MAX_SEGMENT_SIZE)
+            long prev = 0;
+            for (long boundary : boundaries)
             {
-                currentStart = boundary;
-                boundaries.add(currentStart);
-                currentSize = 0;
+                if (boundary - prev > MAX_SEGMENT_SIZE)
+                    return true;
+                prev = boundary;
             }
+            return false;
+        }
+
+        public void addPotentialBoundary(long boundary)
+        {
+            boundaries.addCandidate(boundary);
         }
 
         public SegmentedFile complete(String path, long overrideLength, boolean isFinal)
@@ -188,10 +362,10 @@ public class MmappedSegmentedFile extends SegmentedFile
             assert !isFinal || overrideLength <= 0;
             long length = overrideLength > 0 ? overrideLength : new File(path).length();
             // create the segments
-            return new MmappedSegmentedFile(path, length, createSegments(path, length));
+            return new MmappedSegmentedFile(path, length, createSegments(path, length, isFinal));
         }
 
-        private Segment[] createSegments(String path, long length)
+        private Segment[] createSegments(String path, long length, boolean isFinal)
         {
             RandomAccessFile raf;
             try
@@ -203,27 +377,17 @@ public class MmappedSegmentedFile extends SegmentedFile
                 throw new RuntimeException(e);
             }
 
-            // if we're early finishing a range that doesn't span multiple segments, but the finished file now does,
-            // we remove these from the end (we loop incase somehow this spans multiple segments, but that would
-            // be a loco dataset
-            while (length < boundaries.get(boundaries.size() - 1))
-                boundaries.remove(boundaries.size() -1);
-
-            // add a sentinel value == length
-            List<Long> boundaries = new ArrayList<>(this.boundaries);
-            if (length != boundaries.get(boundaries.size() - 1))
-                boundaries.add(length);
-
+            long[] boundaries = this.boundaries.finish(length, isFinal);
 
-            int segcount = boundaries.size() - 1;
+            int segcount = boundaries.length - 1;
             Segment[] segments = new Segment[segcount];
 
             try
             {
                 for (int i = 0; i < segcount; i++)
                 {
-                    long start = boundaries.get(i);
-                    long size = boundaries.get(i + 1) - start;
+                    long start = boundaries[i];
+                    long size = boundaries[i + 1] - start;
                     MappedByteBuffer segment = size <= MAX_SEGMENT_SIZE
                                                ? raf.getChannel().map(FileChannel.MapMode.READ_ONLY, start, size)
                                                : null;
@@ -245,9 +409,10 @@ public class MmappedSegmentedFile extends SegmentedFile
         public void serializeBounds(DataOutput out) throws IOException
         {
             super.serializeBounds(out);
-            out.writeInt(boundaries.size());
-            for (long position: boundaries)
-                out.writeLong(position);
+            long[] boundaries = this.boundaries.truncate();
+            out.writeInt(boundaries.length);
+            for (long boundary : boundaries)
+                out.writeLong(boundary);
         }
 
         @Override
@@ -256,12 +421,11 @@ public class MmappedSegmentedFile extends SegmentedFile
             super.deserializeBounds(in);
 
             int size = in.readInt();
-            List<Long> temp = new ArrayList<>(size);
-            
+            long[] boundaries = new long[size];
             for (int i = 0; i < size; i++)
-                temp.add(in.readLong());
+                boundaries[i] = in.readLong();
 
-            boundaries = temp;
+            this.boundaries.init(boundaries, size);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/util/SegmentedFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/SegmentedFile.java b/src/java/org/apache/cassandra/io/util/SegmentedFile.java
index c65ecbf..23454bc 100644
--- a/src/java/org/apache/cassandra/io/util/SegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/SegmentedFile.java
@@ -23,6 +23,7 @@ import java.io.File;
 import java.io.IOException;
 import java.nio.MappedByteBuffer;
 import java.util.Iterator;
+import java.util.List;
 import java.util.NoSuchElementException;
 
 import com.google.common.util.concurrent.RateLimiter;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
----------------------------------------------------------------------
diff --git a/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java b/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
new file mode 100644
index 0000000..e17c6a7
--- /dev/null
+++ b/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
@@ -0,0 +1,322 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied.  See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+package org.apache.cassandra.io.sstable;
+
+import java.io.*;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Random;
+
+import com.google.common.io.Files;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.MmappedSegmentedFile;
+import org.apache.cassandra.io.util.MmappedSegmentedFile.Builder.Boundaries;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class LongSegmentedFileBoundaryTest
+{
+    @BeforeClass
+    public static void setup() throws Exception
+    {
+        SchemaLoader.cleanupAndLeaveDirs();
+        Keyspace.setInitialized();
+        StorageService.instance.initServer();
+    }
+
+    @AfterClass
+    public static void tearDown()
+    {
+        Config.setClientMode(false);
+    }
+
+    @Test
+    public void testRandomBoundaries()
+    {
+        long[] candidates = new long[1 + (1 << 16)];
+        int[] indexesToCheck = new int[1 << 8];
+        Random random = new Random();
+
+        for (int run = 0; run < 100; run++)
+        {
+
+            long seed = random.nextLong();
+            random.setSeed(seed);
+            System.out.println("Seed: " + seed);
+
+            // at least 1Ki, and as many as 256Ki, boundaries
+            int candidateCount = (1 + random.nextInt(candidates.length >> 10)) << 10;
+            generateBoundaries(random, candidateCount, candidates, indexesToCheck);
+
+            Boundaries builder = new Boundaries();
+            int nextIndexToCheck = indexesToCheck[0];
+            int checkCount = 0;
+            System.out.printf("[0..%d)", candidateCount);
+            for (int i = 1; i < candidateCount - 1; i++)
+            {
+                if (i == nextIndexToCheck)
+                {
+                    if (checkCount % 20 == 0)
+                        System.out.printf(" %d", i);
+                    // grow number of samples logarithmically; work will still increase superlinearly, as size of dataset grows linearly
+                    int sampleCount = 1 << (31 - Integer.numberOfLeadingZeros(++checkCount));
+                    checkBoundarySample(random, candidates, i, sampleCount, builder);
+                    // select out next index to check (there may be dups, so skip them)
+                    while ((nextIndexToCheck = checkCount == indexesToCheck.length ? candidateCount : indexesToCheck[checkCount]) == i)
+                        checkCount++;
+                }
+
+                builder.addCandidate(candidates[i]);
+            }
+            System.out.println();
+            checkBoundaries(candidates, candidateCount - 1, builder, candidates[candidateCount - 1]);
+            Assert.assertEquals(candidateCount, nextIndexToCheck);
+        }
+    }
+
+    private static void generateBoundaries(Random random, int candidateCount, long[] candidates, int[] indexesToCheck)
+    {
+        // average averageBoundarySize is 4MiB, max 4GiB, min 4KiB
+        long averageBoundarySize = (4L << 10) * random.nextInt(1 << 20);
+        long prev = 0;
+        for (int i = 1 ; i < candidateCount ; i++)
+            candidates[i] = prev += Math.max(1, averageBoundarySize + (random.nextGaussian() * averageBoundarySize));
+
+        // generate indexes we will corroborate our behaviour on
+        for (int i = 0 ; i < indexesToCheck.length ; i++)
+            indexesToCheck[i] = 1 + random.nextInt(candidateCount - 2);
+        Arrays.sort(indexesToCheck);
+    }
+
+    private static void checkBoundarySample(Random random, long[] candidates, int candidateCount, int sampleCount, Boundaries builder)
+    {
+        for (int i = 0 ; i < sampleCount ; i++)
+        {
+            // pick a number exponentially less likely to be near the beginning, since we test that area earlier
+            int position = 0 ;
+            while (position <= 0)
+                position = candidateCount / (Integer.lowestOneBit(random.nextInt()));
+            long upperBound = candidates[position];
+            long lowerBound = random.nextBoolean() ? (rand(random, 0, upperBound) / (Integer.lowestOneBit(random.nextInt())))
+                                                   : candidates[Math.max(0, position - random.nextInt(64))];
+            long length = rand(random, lowerBound, upperBound);
+            checkBoundaries(candidates, candidateCount, builder, length);
+        }
+        checkBoundaries(candidates, candidateCount, builder, candidates[candidateCount]);
+    }
+
+    private static long rand(Random random, long lowerBound, long upperBound)
+    {
+        if (upperBound == lowerBound)
+            return upperBound;
+        return lowerBound + ((random.nextLong() & Long.MAX_VALUE) % (upperBound - lowerBound));
+    }
+
+    private static void checkBoundaries(long[] candidates, int candidateCount, Boundaries builder, long length)
+    {
+        if (length == 0)
+            return;
+
+        long[] boundaries = new long[(int) (10 + 2 * (length / Integer.MAX_VALUE))];
+        int count = 1;
+        int prev = 0;
+        while (true)
+        {
+            int p = candidates[prev + 1] - boundaries[count - 1] >= Integer.MAX_VALUE
+                    ? prev + 1
+                    : Arrays.binarySearch(candidates, prev, candidateCount, boundaries[count - 1] + Integer.MAX_VALUE);
+            if (p < 0) p = -2 -p;
+            if (p >= candidateCount - 1 || candidates[p] >= length)
+                break;
+            boundaries[count++] = candidates[p];
+            if (candidates[p + 1] >= length)
+                break;
+            prev = p;
+        }
+        if (candidates[candidateCount - 1] < length && length - boundaries[count - 1] >= Integer.MAX_VALUE)
+            boundaries[count++] = candidates[candidateCount - 1];
+        boundaries[count++] = length;
+        final long[] canon = Arrays.copyOf(boundaries, count);
+        final long[] check = builder.finish(length, false);
+        if (!Arrays.equals(canon, check))
+            Assert.assertTrue("\n" + Arrays.toString(canon) + "\n" + Arrays.toString(check), Arrays.equals(canon, check));
+    }
+
+    @Test
+    public void testBoundariesAndRepairSmall() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1, 1 << 16);
+    }
+
+    @Test
+    public void testBoundariesAndRepairMedium() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1, 1 << 20);
+    }
+
+    @Test
+    public void testBoundariesAndRepairLarge() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1, 100 << 20);
+    }
+
+    @Test
+    public void testBoundariesAndRepairHuge() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1, Integer.MAX_VALUE - 1024);
+    }
+
+    @Test
+    public void testBoundariesAndRepairTooHuge() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1, Integer.MAX_VALUE);
+    }
+
+    @Test
+    public void testBoundariesAndRepairHugeIndex() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1 << 7, 1 << 15);
+    }
+
+    @Test
+    public void testBoundariesAndRepairReallyHugeIndex() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1 << 14, 1 << 15);
+    }
+
+    private void testBoundariesAndRepair(int rows, int rowSize) throws InvalidRequestException, IOException
+    {
+        String KS = "cql_keyspace";
+        String TABLE = "table1";
+
+        File tempdir = Files.createTempDir();
+        try
+        {
+            Assert.assertTrue(DatabaseDescriptor.getColumnIndexSize() < rowSize);
+            Assert.assertTrue(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap);
+            Assert.assertTrue(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap);
+            Assert.assertTrue(StorageService.getPartitioner() instanceof ByteOrderedPartitioner);
+            File dataDir = new File(tempdir.getAbsolutePath() + File.separator + KS + File.separator + TABLE);
+            Assert.assertTrue(dataDir.mkdirs());
+
+            String schema = "CREATE TABLE cql_keyspace.table" + (rows > 1 ? "2" : "1") + " (k bigint, v1 blob, v2 blob, v3 blob, v4 blob, v5 blob, PRIMARY KEY (k" + (rows > 1 ? ", v1" : "") + ")) WITH compression = { 'sstable_compression':'' };";
+            String insert = "INSERT INTO cql_keyspace.table" + (rows > 1 ? "2" : "1") + " (k, v1, v2, v3, v4, v5) VALUES (?, ?, ?, ?, ?, ?)";
+
+            CQLSSTableWriter.Builder builder = CQLSSTableWriter.builder()
+                                                      .inDirectory(dataDir)
+                                                      .forTable(schema)
+                                                      .withPartitioner(StorageService.getPartitioner())
+                                                      .using(insert)
+                                                      .sorted();
+            CQLSSTableWriter writer = builder.build();
+
+            // write 8Gb of decorated keys
+            ByteBuffer[] value = new ByteBuffer[rows];
+            for (int row = 0 ; row < rows ; row++)
+            {
+                // if we're using clustering columns, the clustering key is replicated across every other column
+                value[row] = ByteBuffer.allocate(rowSize / (rows > 1 ? 8 : 5));
+                value[row].putInt(0, row);
+            }
+            long targetSize = 8L << 30;
+            long dk = 0;
+            long size = 0;
+            long dkSize = rowSize * rows;
+            while (size < targetSize)
+            {
+                for (int row = 0 ; row < rows ; row++)
+                    writer.addRow(dk, value[row], value[row], value[row], value[row], value[row]);
+                size += dkSize;
+                dk++;
+            }
+
+            Descriptor descriptor = writer.getCurrentDescriptor().asType(Descriptor.Type.FINAL);
+            writer.close();
+
+            // open (and close) the reader so that the summary file is created
+            SSTableReader reader = SSTableReader.open(descriptor);
+            reader.selfRef().release();
+
+            // then check the boundaries are reasonable, and corrupt them
+            checkThenCorruptBoundaries(descriptor, rows * rowSize < Integer.MAX_VALUE);
+
+            // then check that reopening corrects the corruption
+            reader = SSTableReader.open(descriptor);
+            reader.selfRef().release();
+            checkThenCorruptBoundaries(descriptor, rows * rowSize < Integer.MAX_VALUE);
+        }
+        finally
+        {
+            FileUtils.deleteRecursive(tempdir);
+        }
+    }
+
+    private static void checkThenCorruptBoundaries(Descriptor descriptor, boolean expectDataMmappable) throws IOException
+    {
+        File summaryFile = new File(descriptor.filenameFor(Component.SUMMARY));
+        DataInputStream iStream = new DataInputStream(new FileInputStream(summaryFile));
+        IndexSummary indexSummary = IndexSummary.serializer.deserialize(iStream, StorageService.getPartitioner(), true, CFMetaData.DEFAULT_MIN_INDEX_INTERVAL, CFMetaData.DEFAULT_MAX_INDEX_INTERVAL);
+        ByteBuffer first = ByteBufferUtil.readWithLength(iStream);
+        ByteBuffer last = ByteBufferUtil.readWithLength(iStream);
+        MmappedSegmentedFile.Builder ibuilder = new MmappedSegmentedFile.Builder();
+        MmappedSegmentedFile.Builder dbuilder = new MmappedSegmentedFile.Builder();
+        ibuilder.deserializeBounds(iStream);
+        dbuilder.deserializeBounds(iStream);
+        iStream.close();
+        // index file cannot generally be non-mmappable, as index entries cannot be larger than MAX_SEGMENT_SIZE (due to promotedSize being encoded as an int)
+        assertBoundaries(descriptor.filenameFor(Component.PRIMARY_INDEX), true, ibuilder.boundaries());
+        assertBoundaries(descriptor.filenameFor(Component.DATA), expectDataMmappable, dbuilder.boundaries());
+
+        DataOutputStreamPlus oStream = new DataOutputStreamPlus(new FileOutputStream(summaryFile));
+        IndexSummary.serializer.serialize(indexSummary, oStream, true);
+        ByteBufferUtil.writeWithLength(first, oStream);
+        ByteBufferUtil.writeWithLength(last, oStream);
+        oStream.writeInt(1);
+        oStream.writeLong(new File(descriptor.filenameFor(Component.PRIMARY_INDEX)).length());
+        oStream.writeLong(new File(descriptor.filenameFor(Component.DATA)).length());
+        oStream.close();
+    }
+
+    private static void assertBoundaries(String path, boolean expectMmappable, long[] boundaries)
+    {
+        long length = new File(path).length();
+        long prev = boundaries[0];
+        for (int i = 1 ; i <= boundaries.length && prev < length ; i++)
+        {
+            long boundary = i == boundaries.length ? length : boundaries[i];
+            Assert.assertEquals(String.format("[%d, %d), %d of %d", boundary, prev, i, boundaries.length),
+                                expectMmappable, boundary - prev <= Integer.MAX_VALUE);
+            prev = boundary;
+        }
+    }
+
+}


[14/16] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by be...@apache.org.
Merge branch 'cassandra-2.2' into cassandra-3.0


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

Branch: refs/heads/cassandra-3.0
Commit: e3d58448b6cadef7be060bb35996cd86d9ec9d59
Parents: a039b7d 25de92e
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Wed Sep 30 19:47:39 2015 +0100
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Wed Sep 30 19:47:39 2015 +0100

----------------------------------------------------------------------

----------------------------------------------------------------------



[02/16] cassandra git commit: Fix Mmapped File Boundaries

Posted by be...@apache.org.
Fix Mmapped File Boundaries

This patch fixes two bugs with mmap segment boundary
tracking, and introduces automated correction of
this bug on startup

patch by benedict; reviewed by tjake for CASSANDRA-10357


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

Branch: refs/heads/cassandra-2.2
Commit: c37562e345c24720c55428a8644191df68319812
Parents: f6cab37
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Wed Sep 16 18:09:32 2015 +0100
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Wed Sep 30 19:45:49 2015 +0100

----------------------------------------------------------------------
 .../io/sstable/AbstractSSTableSimpleWriter.java |   2 +
 .../cassandra/io/sstable/CQLSSTableWriter.java  |  15 +
 .../cassandra/io/sstable/SSTableReader.java     |  34 +-
 .../io/sstable/SSTableSimpleUnsortedWriter.java |   6 +
 .../io/sstable/SSTableSimpleWriter.java         |  12 +
 .../cassandra/io/util/MappedFileDataInput.java  |   8 +-
 .../cassandra/io/util/MmappedSegmentedFile.java | 270 +++++++++++++---
 .../apache/cassandra/io/util/SegmentedFile.java |   1 +
 .../sstable/LongSegmentedFileBoundaryTest.java  | 322 +++++++++++++++++++
 9 files changed, 601 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
index 165a4b2..557c3de 100644
--- a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
@@ -200,4 +200,6 @@ public abstract class AbstractSSTableSimpleWriter implements Closeable
     protected abstract void writeRow(DecoratedKey key, ColumnFamily columnFamily) throws IOException;
 
     protected abstract ColumnFamily getColumnFamily() throws IOException;
+
+    public abstract Descriptor getCurrentDescriptor();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
index b211a90..c364171 100644
--- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
@@ -271,6 +271,16 @@ public class CQLSSTableWriter implements Closeable
         writer.close();
     }
 
+    public Descriptor getCurrentDescriptor()
+    {
+        return writer.getCurrentDescriptor();
+    }
+
+    public CFMetaData getCFMetaData()
+    {
+        return writer.metadata;
+    }
+
     /**
      * A Builder for a CQLSSTableWriter object.
      */
@@ -366,6 +376,11 @@ public class CQLSSTableWriter implements Closeable
             }
         }
 
+        CFMetaData metadata()
+        {
+            return schema;
+        }
+
         /**
          * Adds the specified column family to the specified keyspace.
          *

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/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 0f307b0..84add6f 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
@@ -17,13 +17,7 @@
  */
 package org.apache.cassandra.io.sstable;
 
-import java.io.BufferedInputStream;
-import java.io.DataInputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.IOException;
+import java.io.*;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.*;
@@ -70,20 +64,14 @@ import org.apache.cassandra.dht.LocalPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.FSError;
+import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.io.compress.CompressionMetadata;
 import org.apache.cassandra.io.sstable.metadata.CompactionMetadata;
 import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
 import org.apache.cassandra.io.sstable.metadata.MetadataType;
 import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
 import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
-import org.apache.cassandra.io.util.BufferedSegmentedFile;
-import org.apache.cassandra.io.util.CompressedSegmentedFile;
-import org.apache.cassandra.io.util.DataOutputStreamAndChannel;
-import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.io.util.ICompressedFile;
-import org.apache.cassandra.io.util.RandomAccessReader;
-import org.apache.cassandra.io.util.SegmentedFile;
+import org.apache.cassandra.io.util.*;
 import org.apache.cassandra.metrics.RestorableMeter;
 import org.apache.cassandra.metrics.StorageMetrics;
 import org.apache.cassandra.service.ActiveRepairService;
@@ -162,6 +150,7 @@ import static org.apache.cassandra.db.Directories.SECONDARY_INDEX_NAME_SEPARATOR
 public class SSTableReader extends SSTable implements SelfRefCounted<SSTableReader>
 {
     private static final Logger logger = LoggerFactory.getLogger(SSTableReader.class);
+    private static final int ACCURATE_BOUNDARIES_MAGIC_NUMBER = 248923458;
 
     private static final ScheduledThreadPoolExecutor syncExecutor = new ScheduledThreadPoolExecutor(1);
     static
@@ -892,6 +881,19 @@ public class SSTableReader extends SSTable implements SelfRefCounted<SSTableRead
             last = partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
             ibuilder.deserializeBounds(iStream);
             dbuilder.deserializeBounds(iStream);
+
+            boolean checkForRepair = true;
+            try
+            {
+                int v = iStream.readInt();
+                // check for our magic number, indicating this summary has been sampled correctly
+                checkForRepair = v != ACCURATE_BOUNDARIES_MAGIC_NUMBER;
+            }
+            catch (Throwable t) {}
+
+            // fix CASSANDRA-10357 on-the-fly
+            if (checkForRepair && MmappedSegmentedFile.maybeRepair(metadata, descriptor, indexSummary, ibuilder, dbuilder))
+                saveSummary(ibuilder, dbuilder);
         }
         catch (IOException e)
         {
@@ -992,6 +994,8 @@ public class SSTableReader extends SSTable implements SelfRefCounted<SSTableRead
             ByteBufferUtil.writeWithLength(last.getKey(), oStream);
             ibuilder.serializeBounds(oStream);
             dbuilder.serializeBounds(oStream);
+            // write a magic number, to indicate this summary has been sampled correctly
+            oStream.writeInt(ACCURATE_BOUNDARIES_MAGIC_NUMBER);
         }
         catch (IOException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
index 9ee9ea1..25ec354 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
@@ -140,6 +140,12 @@ public class SSTableSimpleUnsortedWriter extends AbstractSSTableSimpleWriter
         return previous;
     }
 
+    public Descriptor getCurrentDescriptor()
+    {
+        // can be implemented, but isn't necessary
+        throw new UnsupportedOperationException();
+    }
+
     protected ColumnFamily createColumnFamily() throws IOException
     {
         return ArrayBackedSortedColumns.factory.create(metadata);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
index 87c8e33..23da501 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
@@ -65,6 +65,13 @@ public class SSTableSimpleWriter extends AbstractSSTableSimpleWriter
         writer = getWriter();
     }
 
+    SSTableReader closeAndOpenReader()
+    {
+        if (currentKey != null)
+            writeRow(currentKey, columnFamily);
+        return writer.closeAndOpenReader();
+    }
+
     public void close()
     {
         try
@@ -89,4 +96,9 @@ public class SSTableSimpleWriter extends AbstractSSTableSimpleWriter
     {
         return ArrayBackedSortedColumns.factory.create(metadata);
     }
+
+    public Descriptor getCurrentDescriptor()
+    {
+        return writer.descriptor;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java b/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java
index d056240..f93ce72 100644
--- a/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java
+++ b/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java
@@ -51,12 +51,18 @@ public class MappedFileDataInput extends AbstractDataInput implements FileDataIn
     public void seek(long pos) throws IOException
     {
         long inSegmentPos = pos - segmentOffset;
-        if (inSegmentPos < 0 || inSegmentPos > buffer.capacity())
+        if (!contains(pos))
             throw new IOException(String.format("Seek position %d is not within mmap segment (seg offs: %d, length: %d)", pos, segmentOffset, buffer.capacity()));
 
         seekInternal((int) inSegmentPos);
     }
 
+    public boolean contains(long pos)
+    {
+        long inSegmentPos = pos - segmentOffset;
+        return inSegmentPos >= 0 && inSegmentPos < buffer.capacity();
+    }
+
     public long getFilePointer()
     {
         return segmentOffset + (long)position;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
index 1b23343..623f65a 100644
--- a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
@@ -24,11 +24,17 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.RowIndexEntry;
 import org.apache.cassandra.io.FSReadError;
-import org.apache.cassandra.io.sstable.SSTableWriter;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummary;
+import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 
 public class MmappedSegmentedFile extends SegmentedFile
@@ -135,52 +141,220 @@ public class MmappedSegmentedFile extends SegmentedFile
         }
     }
 
+    // see CASSANDRA-10357
+    public static boolean maybeRepair(CFMetaData metadata, Descriptor descriptor, IndexSummary indexSummary, SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
+    {
+        boolean mayNeedRepair = false;
+        if (ibuilder instanceof Builder)
+            mayNeedRepair = ((Builder) ibuilder).mayNeedRepair(descriptor.filenameFor(Component.PRIMARY_INDEX));
+        if (dbuilder instanceof Builder)
+            mayNeedRepair |= ((Builder) dbuilder).mayNeedRepair(descriptor.filenameFor(Component.DATA));
+
+        if (mayNeedRepair)
+            forceRepair(metadata, descriptor, indexSummary, ibuilder, dbuilder);
+        return mayNeedRepair;
+    }
+
+    // if one of the index/data files have boundaries larger than we can mmap, and they were written by a version that did not guarantee correct boundaries were saved,
+    // rebuild the boundaries and save them again
+    private static void forceRepair(CFMetaData metadata, Descriptor descriptor, IndexSummary indexSummary, SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
+    {
+        if (ibuilder instanceof Builder)
+            ((Builder) ibuilder).boundaries.clear();
+        if (dbuilder instanceof Builder)
+            ((Builder) dbuilder).boundaries.clear();
+
+        try (RandomAccessFile raf = new RandomAccessFile(descriptor.filenameFor(Component.PRIMARY_INDEX), "r");)
+        {
+            long iprev = 0, dprev = 0;
+            for (int i = 0; i < indexSummary.size(); i++)
+            {
+                // first read the position in the summary, and read the corresponding position in the data file
+                long icur = indexSummary.getPosition(i);
+                raf.seek(icur);
+                ByteBufferUtil.readWithShortLength(raf);
+                RowIndexEntry rie = metadata.comparator.rowIndexEntrySerializer().deserialize(raf, descriptor.version);
+                long dcur = rie.position;
+
+                // if these positions are small enough to map out a segment from the prior version (i.e. less than 2Gb),
+                // just add these as a boundary and proceed to the next index summary record; most scenarios will be
+                // served by this, keeping the cost of rebuild to a minimum.
+
+                if (Math.max(icur - iprev , dcur - dprev) > MAX_SEGMENT_SIZE)
+                {
+                    // otherwise, loop over its index block, providing each RIE as a potential boundary for both files
+                    raf.seek(iprev);
+                    while (raf.getFilePointer() < icur)
+                    {
+                        // add the position of this record in the index file as an index file boundary
+                        ibuilder.addPotentialBoundary(raf.getFilePointer());
+                        // then read the RIE, and add its data file position as a boundary for the data file
+                        ByteBufferUtil.readWithShortLength(raf);
+                        rie = metadata.comparator.rowIndexEntrySerializer().deserialize(raf, descriptor.version);
+                        dbuilder.addPotentialBoundary(rie.position);
+                    }
+                }
+
+                ibuilder.addPotentialBoundary(icur);
+                dbuilder.addPotentialBoundary(dcur);
+
+                iprev = icur;
+                dprev = dcur;
+            }
+        }
+        catch (IOException e)
+        {
+            logger.error("Failed to recalculate boundaries for {}; mmap access may degrade to buffered for this file", descriptor);
+        }
+    }
+
     /**
      * Overrides the default behaviour to create segments of a maximum size.
      */
-    static class Builder extends SegmentedFile.Builder
+    public static class Builder extends SegmentedFile.Builder
     {
-        // planned segment boundaries
-        private List<Long> boundaries;
+        @VisibleForTesting
+        public static class Boundaries
+        {
+            private long[] boundaries;
+
+            // number of boundaries we have "fixed" (i.e. have determined the final value of)
+            private int fixedCount;
+
+            public Boundaries()
+            {
+                // we always have a boundary of zero, so we start with a fixedCount of 1
+                this(new long[8], 1);
+            }
+
+            public Boundaries(long[] boundaries, int fixedCount)
+            {
+                init(boundaries, fixedCount);
+            }
+
+            void init(long[] boundaries, int fixedCount)
+            {
+                this.boundaries = boundaries;
+                this.fixedCount = fixedCount;
+            }
+
+            public void addCandidate(long candidate)
+            {
+                // we make sure we have room before adding another element, so that we can share the addCandidate logic statically
+                boundaries = ensureCapacity(boundaries, fixedCount);
+                fixedCount = addCandidate(boundaries, fixedCount, candidate);
+            }
+
+            private static int addCandidate(long[] boundaries, int fixedCount, long candidate)
+            {
+                // check how far we are from the last fixed boundary
+                long delta = candidate - boundaries[fixedCount - 1];
+                assert delta >= 0;
+                if (delta != 0)
+                {
+                    if (delta <= MAX_SEGMENT_SIZE)
+                        // overwrite the unfixed (potential) boundary if the resultant segment would still be mmappable
+                        boundaries[fixedCount] = candidate;
+                    else if (boundaries[fixedCount] == 0)
+                        // or, if it is not initialised, we cannot make an mmapped segment here, so this is the fixed boundary
+                        boundaries[fixedCount++] = candidate;
+                    else
+                        // otherwise, fix the prior boundary and initialise our unfixed boundary
+                        boundaries[++fixedCount] = candidate;
+                }
+                return fixedCount;
+            }
+
+            // ensures there is room for another fixed boundary AND an unfixed candidate boundary, i.e. fixedCount + 2 items
+            private static long[] ensureCapacity(long[] boundaries, int fixedCount)
+            {
+                if (fixedCount + 1 >= boundaries.length)
+                    return Arrays.copyOf(boundaries, boundaries.length * 2);
+                return boundaries;
+            }
+
+            void clear()
+            {
+                fixedCount = 1;
+                Arrays.fill(boundaries, 0);
+            }
+
+            // returns the fixed boundaries, truncated to a correctly sized long[]
+            public long[] truncate()
+            {
+                return Arrays.copyOf(boundaries, fixedCount);
+            }
 
-        // offset of the open segment (first segment begins at 0).
-        private long currentStart = 0;
+            // returns the finished boundaries for the provided length, truncated to a correctly sized long[]
+            public long[] finish(long length, boolean isFinal)
+            {
+                assert length > 0;
+                // ensure there's room for the length to be added
+                boundaries = ensureCapacity(boundaries, fixedCount);
+
+                // clone our current contents, so we don't corrupt them
+                int fixedCount = this.fixedCount;
+                long[] boundaries = this.boundaries.clone();
+
+                // if we're finishing early, our length may be before some of our boundaries,
+                // so walk backwards until our boundaries are <= length
+                while (boundaries[fixedCount - 1] >= length)
+                    boundaries[fixedCount--] = 0;
+                if (boundaries[fixedCount] >= length)
+                    boundaries[fixedCount] = 0;
+
+                // add our length as a boundary
+                fixedCount = addCandidate(boundaries, fixedCount, length);
+
+                // if we have any unfixed boundary at the end, it's now fixed, since we're done
+                if (boundaries[fixedCount] != 0)
+                    fixedCount++;
+
+                boundaries = Arrays.copyOf(boundaries, fixedCount);
+                if (isFinal)
+                {
+                    // if this is the final one, save it
+                    this.boundaries = boundaries;
+                    this.fixedCount = fixedCount;
+                }
+                return boundaries;
+            }
+        }
 
-        // current length of the open segment.
-        // used to allow merging multiple too-large-to-mmap segments, into a single buffered segment.
-        private long currentSize = 0;
+        private final Boundaries boundaries = new Boundaries();
 
         public Builder()
         {
             super();
-            boundaries = new ArrayList<>();
-            boundaries.add(0L);
         }
 
-        public void addPotentialBoundary(long boundary)
+        public long[] boundaries()
         {
-            if (boundary - currentStart <= MAX_SEGMENT_SIZE)
-            {
-                // boundary fits into current segment: expand it
-                currentSize = boundary - currentStart;
-                return;
-            }
+            return boundaries.truncate();
+        }
 
-            // close the current segment to try and make room for the boundary
-            if (currentSize > 0)
-            {
-                currentStart += currentSize;
-                boundaries.add(currentStart);
-            }
-            currentSize = boundary - currentStart;
+        // indicates if we may need to repair the mmapped file boundaries. this is a cheap check to see if there
+        // are any spans larger than an mmap segment size, which should be rare to occur in practice.
+        boolean mayNeedRepair(String path)
+        {
+            // old boundaries were created without the length, so add it as a candidate
+            long length = new File(path).length();
+            boundaries.addCandidate(length);
+            long[] boundaries = this.boundaries.truncate();
 
-            // if we couldn't make room, the boundary needs its own segment
-            if (currentSize > MAX_SEGMENT_SIZE)
+            long prev = 0;
+            for (long boundary : boundaries)
             {
-                currentStart = boundary;
-                boundaries.add(currentStart);
-                currentSize = 0;
+                if (boundary - prev > MAX_SEGMENT_SIZE)
+                    return true;
+                prev = boundary;
             }
+            return false;
+        }
+
+        public void addPotentialBoundary(long boundary)
+        {
+            boundaries.addCandidate(boundary);
         }
 
         public SegmentedFile complete(String path, long overrideLength, boolean isFinal)
@@ -188,10 +362,10 @@ public class MmappedSegmentedFile extends SegmentedFile
             assert !isFinal || overrideLength <= 0;
             long length = overrideLength > 0 ? overrideLength : new File(path).length();
             // create the segments
-            return new MmappedSegmentedFile(path, length, createSegments(path, length));
+            return new MmappedSegmentedFile(path, length, createSegments(path, length, isFinal));
         }
 
-        private Segment[] createSegments(String path, long length)
+        private Segment[] createSegments(String path, long length, boolean isFinal)
         {
             RandomAccessFile raf;
             try
@@ -203,27 +377,17 @@ public class MmappedSegmentedFile extends SegmentedFile
                 throw new RuntimeException(e);
             }
 
-            // if we're early finishing a range that doesn't span multiple segments, but the finished file now does,
-            // we remove these from the end (we loop incase somehow this spans multiple segments, but that would
-            // be a loco dataset
-            while (length < boundaries.get(boundaries.size() - 1))
-                boundaries.remove(boundaries.size() -1);
-
-            // add a sentinel value == length
-            List<Long> boundaries = new ArrayList<>(this.boundaries);
-            if (length != boundaries.get(boundaries.size() - 1))
-                boundaries.add(length);
-
+            long[] boundaries = this.boundaries.finish(length, isFinal);
 
-            int segcount = boundaries.size() - 1;
+            int segcount = boundaries.length - 1;
             Segment[] segments = new Segment[segcount];
 
             try
             {
                 for (int i = 0; i < segcount; i++)
                 {
-                    long start = boundaries.get(i);
-                    long size = boundaries.get(i + 1) - start;
+                    long start = boundaries[i];
+                    long size = boundaries[i + 1] - start;
                     MappedByteBuffer segment = size <= MAX_SEGMENT_SIZE
                                                ? raf.getChannel().map(FileChannel.MapMode.READ_ONLY, start, size)
                                                : null;
@@ -245,9 +409,10 @@ public class MmappedSegmentedFile extends SegmentedFile
         public void serializeBounds(DataOutput out) throws IOException
         {
             super.serializeBounds(out);
-            out.writeInt(boundaries.size());
-            for (long position: boundaries)
-                out.writeLong(position);
+            long[] boundaries = this.boundaries.truncate();
+            out.writeInt(boundaries.length);
+            for (long boundary : boundaries)
+                out.writeLong(boundary);
         }
 
         @Override
@@ -256,12 +421,11 @@ public class MmappedSegmentedFile extends SegmentedFile
             super.deserializeBounds(in);
 
             int size = in.readInt();
-            List<Long> temp = new ArrayList<>(size);
-            
+            long[] boundaries = new long[size];
             for (int i = 0; i < size; i++)
-                temp.add(in.readLong());
+                boundaries[i] = in.readLong();
 
-            boundaries = temp;
+            this.boundaries.init(boundaries, size);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/util/SegmentedFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/SegmentedFile.java b/src/java/org/apache/cassandra/io/util/SegmentedFile.java
index c65ecbf..23454bc 100644
--- a/src/java/org/apache/cassandra/io/util/SegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/SegmentedFile.java
@@ -23,6 +23,7 @@ import java.io.File;
 import java.io.IOException;
 import java.nio.MappedByteBuffer;
 import java.util.Iterator;
+import java.util.List;
 import java.util.NoSuchElementException;
 
 import com.google.common.util.concurrent.RateLimiter;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
----------------------------------------------------------------------
diff --git a/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java b/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
new file mode 100644
index 0000000..e17c6a7
--- /dev/null
+++ b/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
@@ -0,0 +1,322 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied.  See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+package org.apache.cassandra.io.sstable;
+
+import java.io.*;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Random;
+
+import com.google.common.io.Files;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.MmappedSegmentedFile;
+import org.apache.cassandra.io.util.MmappedSegmentedFile.Builder.Boundaries;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class LongSegmentedFileBoundaryTest
+{
+    @BeforeClass
+    public static void setup() throws Exception
+    {
+        SchemaLoader.cleanupAndLeaveDirs();
+        Keyspace.setInitialized();
+        StorageService.instance.initServer();
+    }
+
+    @AfterClass
+    public static void tearDown()
+    {
+        Config.setClientMode(false);
+    }
+
+    @Test
+    public void testRandomBoundaries()
+    {
+        long[] candidates = new long[1 + (1 << 16)];
+        int[] indexesToCheck = new int[1 << 8];
+        Random random = new Random();
+
+        for (int run = 0; run < 100; run++)
+        {
+
+            long seed = random.nextLong();
+            random.setSeed(seed);
+            System.out.println("Seed: " + seed);
+
+            // at least 1Ki, and as many as 256Ki, boundaries
+            int candidateCount = (1 + random.nextInt(candidates.length >> 10)) << 10;
+            generateBoundaries(random, candidateCount, candidates, indexesToCheck);
+
+            Boundaries builder = new Boundaries();
+            int nextIndexToCheck = indexesToCheck[0];
+            int checkCount = 0;
+            System.out.printf("[0..%d)", candidateCount);
+            for (int i = 1; i < candidateCount - 1; i++)
+            {
+                if (i == nextIndexToCheck)
+                {
+                    if (checkCount % 20 == 0)
+                        System.out.printf(" %d", i);
+                    // grow number of samples logarithmically; work will still increase superlinearly, as size of dataset grows linearly
+                    int sampleCount = 1 << (31 - Integer.numberOfLeadingZeros(++checkCount));
+                    checkBoundarySample(random, candidates, i, sampleCount, builder);
+                    // select out next index to check (there may be dups, so skip them)
+                    while ((nextIndexToCheck = checkCount == indexesToCheck.length ? candidateCount : indexesToCheck[checkCount]) == i)
+                        checkCount++;
+                }
+
+                builder.addCandidate(candidates[i]);
+            }
+            System.out.println();
+            checkBoundaries(candidates, candidateCount - 1, builder, candidates[candidateCount - 1]);
+            Assert.assertEquals(candidateCount, nextIndexToCheck);
+        }
+    }
+
+    private static void generateBoundaries(Random random, int candidateCount, long[] candidates, int[] indexesToCheck)
+    {
+        // average averageBoundarySize is 4MiB, max 4GiB, min 4KiB
+        long averageBoundarySize = (4L << 10) * random.nextInt(1 << 20);
+        long prev = 0;
+        for (int i = 1 ; i < candidateCount ; i++)
+            candidates[i] = prev += Math.max(1, averageBoundarySize + (random.nextGaussian() * averageBoundarySize));
+
+        // generate indexes we will corroborate our behaviour on
+        for (int i = 0 ; i < indexesToCheck.length ; i++)
+            indexesToCheck[i] = 1 + random.nextInt(candidateCount - 2);
+        Arrays.sort(indexesToCheck);
+    }
+
+    private static void checkBoundarySample(Random random, long[] candidates, int candidateCount, int sampleCount, Boundaries builder)
+    {
+        for (int i = 0 ; i < sampleCount ; i++)
+        {
+            // pick a number exponentially less likely to be near the beginning, since we test that area earlier
+            int position = 0 ;
+            while (position <= 0)
+                position = candidateCount / (Integer.lowestOneBit(random.nextInt()));
+            long upperBound = candidates[position];
+            long lowerBound = random.nextBoolean() ? (rand(random, 0, upperBound) / (Integer.lowestOneBit(random.nextInt())))
+                                                   : candidates[Math.max(0, position - random.nextInt(64))];
+            long length = rand(random, lowerBound, upperBound);
+            checkBoundaries(candidates, candidateCount, builder, length);
+        }
+        checkBoundaries(candidates, candidateCount, builder, candidates[candidateCount]);
+    }
+
+    private static long rand(Random random, long lowerBound, long upperBound)
+    {
+        if (upperBound == lowerBound)
+            return upperBound;
+        return lowerBound + ((random.nextLong() & Long.MAX_VALUE) % (upperBound - lowerBound));
+    }
+
+    private static void checkBoundaries(long[] candidates, int candidateCount, Boundaries builder, long length)
+    {
+        if (length == 0)
+            return;
+
+        long[] boundaries = new long[(int) (10 + 2 * (length / Integer.MAX_VALUE))];
+        int count = 1;
+        int prev = 0;
+        while (true)
+        {
+            int p = candidates[prev + 1] - boundaries[count - 1] >= Integer.MAX_VALUE
+                    ? prev + 1
+                    : Arrays.binarySearch(candidates, prev, candidateCount, boundaries[count - 1] + Integer.MAX_VALUE);
+            if (p < 0) p = -2 -p;
+            if (p >= candidateCount - 1 || candidates[p] >= length)
+                break;
+            boundaries[count++] = candidates[p];
+            if (candidates[p + 1] >= length)
+                break;
+            prev = p;
+        }
+        if (candidates[candidateCount - 1] < length && length - boundaries[count - 1] >= Integer.MAX_VALUE)
+            boundaries[count++] = candidates[candidateCount - 1];
+        boundaries[count++] = length;
+        final long[] canon = Arrays.copyOf(boundaries, count);
+        final long[] check = builder.finish(length, false);
+        if (!Arrays.equals(canon, check))
+            Assert.assertTrue("\n" + Arrays.toString(canon) + "\n" + Arrays.toString(check), Arrays.equals(canon, check));
+    }
+
+    @Test
+    public void testBoundariesAndRepairSmall() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1, 1 << 16);
+    }
+
+    @Test
+    public void testBoundariesAndRepairMedium() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1, 1 << 20);
+    }
+
+    @Test
+    public void testBoundariesAndRepairLarge() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1, 100 << 20);
+    }
+
+    @Test
+    public void testBoundariesAndRepairHuge() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1, Integer.MAX_VALUE - 1024);
+    }
+
+    @Test
+    public void testBoundariesAndRepairTooHuge() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1, Integer.MAX_VALUE);
+    }
+
+    @Test
+    public void testBoundariesAndRepairHugeIndex() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1 << 7, 1 << 15);
+    }
+
+    @Test
+    public void testBoundariesAndRepairReallyHugeIndex() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1 << 14, 1 << 15);
+    }
+
+    private void testBoundariesAndRepair(int rows, int rowSize) throws InvalidRequestException, IOException
+    {
+        String KS = "cql_keyspace";
+        String TABLE = "table1";
+
+        File tempdir = Files.createTempDir();
+        try
+        {
+            Assert.assertTrue(DatabaseDescriptor.getColumnIndexSize() < rowSize);
+            Assert.assertTrue(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap);
+            Assert.assertTrue(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap);
+            Assert.assertTrue(StorageService.getPartitioner() instanceof ByteOrderedPartitioner);
+            File dataDir = new File(tempdir.getAbsolutePath() + File.separator + KS + File.separator + TABLE);
+            Assert.assertTrue(dataDir.mkdirs());
+
+            String schema = "CREATE TABLE cql_keyspace.table" + (rows > 1 ? "2" : "1") + " (k bigint, v1 blob, v2 blob, v3 blob, v4 blob, v5 blob, PRIMARY KEY (k" + (rows > 1 ? ", v1" : "") + ")) WITH compression = { 'sstable_compression':'' };";
+            String insert = "INSERT INTO cql_keyspace.table" + (rows > 1 ? "2" : "1") + " (k, v1, v2, v3, v4, v5) VALUES (?, ?, ?, ?, ?, ?)";
+
+            CQLSSTableWriter.Builder builder = CQLSSTableWriter.builder()
+                                                      .inDirectory(dataDir)
+                                                      .forTable(schema)
+                                                      .withPartitioner(StorageService.getPartitioner())
+                                                      .using(insert)
+                                                      .sorted();
+            CQLSSTableWriter writer = builder.build();
+
+            // write 8Gb of decorated keys
+            ByteBuffer[] value = new ByteBuffer[rows];
+            for (int row = 0 ; row < rows ; row++)
+            {
+                // if we're using clustering columns, the clustering key is replicated across every other column
+                value[row] = ByteBuffer.allocate(rowSize / (rows > 1 ? 8 : 5));
+                value[row].putInt(0, row);
+            }
+            long targetSize = 8L << 30;
+            long dk = 0;
+            long size = 0;
+            long dkSize = rowSize * rows;
+            while (size < targetSize)
+            {
+                for (int row = 0 ; row < rows ; row++)
+                    writer.addRow(dk, value[row], value[row], value[row], value[row], value[row]);
+                size += dkSize;
+                dk++;
+            }
+
+            Descriptor descriptor = writer.getCurrentDescriptor().asType(Descriptor.Type.FINAL);
+            writer.close();
+
+            // open (and close) the reader so that the summary file is created
+            SSTableReader reader = SSTableReader.open(descriptor);
+            reader.selfRef().release();
+
+            // then check the boundaries are reasonable, and corrupt them
+            checkThenCorruptBoundaries(descriptor, rows * rowSize < Integer.MAX_VALUE);
+
+            // then check that reopening corrects the corruption
+            reader = SSTableReader.open(descriptor);
+            reader.selfRef().release();
+            checkThenCorruptBoundaries(descriptor, rows * rowSize < Integer.MAX_VALUE);
+        }
+        finally
+        {
+            FileUtils.deleteRecursive(tempdir);
+        }
+    }
+
+    private static void checkThenCorruptBoundaries(Descriptor descriptor, boolean expectDataMmappable) throws IOException
+    {
+        File summaryFile = new File(descriptor.filenameFor(Component.SUMMARY));
+        DataInputStream iStream = new DataInputStream(new FileInputStream(summaryFile));
+        IndexSummary indexSummary = IndexSummary.serializer.deserialize(iStream, StorageService.getPartitioner(), true, CFMetaData.DEFAULT_MIN_INDEX_INTERVAL, CFMetaData.DEFAULT_MAX_INDEX_INTERVAL);
+        ByteBuffer first = ByteBufferUtil.readWithLength(iStream);
+        ByteBuffer last = ByteBufferUtil.readWithLength(iStream);
+        MmappedSegmentedFile.Builder ibuilder = new MmappedSegmentedFile.Builder();
+        MmappedSegmentedFile.Builder dbuilder = new MmappedSegmentedFile.Builder();
+        ibuilder.deserializeBounds(iStream);
+        dbuilder.deserializeBounds(iStream);
+        iStream.close();
+        // index file cannot generally be non-mmappable, as index entries cannot be larger than MAX_SEGMENT_SIZE (due to promotedSize being encoded as an int)
+        assertBoundaries(descriptor.filenameFor(Component.PRIMARY_INDEX), true, ibuilder.boundaries());
+        assertBoundaries(descriptor.filenameFor(Component.DATA), expectDataMmappable, dbuilder.boundaries());
+
+        DataOutputStreamPlus oStream = new DataOutputStreamPlus(new FileOutputStream(summaryFile));
+        IndexSummary.serializer.serialize(indexSummary, oStream, true);
+        ByteBufferUtil.writeWithLength(first, oStream);
+        ByteBufferUtil.writeWithLength(last, oStream);
+        oStream.writeInt(1);
+        oStream.writeLong(new File(descriptor.filenameFor(Component.PRIMARY_INDEX)).length());
+        oStream.writeLong(new File(descriptor.filenameFor(Component.DATA)).length());
+        oStream.close();
+    }
+
+    private static void assertBoundaries(String path, boolean expectMmappable, long[] boundaries)
+    {
+        long length = new File(path).length();
+        long prev = boundaries[0];
+        for (int i = 1 ; i <= boundaries.length && prev < length ; i++)
+        {
+            long boundary = i == boundaries.length ? length : boundaries[i];
+            Assert.assertEquals(String.format("[%d, %d), %d of %d", boundary, prev, i, boundaries.length),
+                                expectMmappable, boundary - prev <= Integer.MAX_VALUE);
+            prev = boundary;
+        }
+    }
+
+}


[07/16] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by be...@apache.org.
Merge branch 'cassandra-2.1' into cassandra-2.2

Conflicts:
	src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
	src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
	src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java


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

Branch: refs/heads/cassandra-3.0
Commit: 25de92e321604626d6c098233082904832c07814
Parents: 7452b20 c37562e
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Wed Sep 30 19:47:14 2015 +0100
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Wed Sep 30 19:47:14 2015 +0100

----------------------------------------------------------------------
 .../io/sstable/AbstractSSTableSimpleWriter.java |   2 +
 .../cassandra/io/sstable/CQLSSTableWriter.java  |  15 +
 .../io/sstable/SSTableSimpleUnsortedWriter.java |   6 +
 .../io/sstable/SSTableSimpleWriter.java         |  13 +
 .../io/sstable/format/SSTableReader.java        |  17 +
 .../cassandra/io/util/ByteBufferDataInput.java  |   8 +-
 .../cassandra/io/util/MmappedSegmentedFile.java | 275 ++++++++++++----
 .../apache/cassandra/io/util/SegmentedFile.java |   1 +
 .../sstable/LongSegmentedFileBoundaryTest.java  | 324 +++++++++++++++++++
 9 files changed, 605 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
index 4181ed0,c364171..8873f88
--- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
@@@ -370,20 -376,22 +380,25 @@@ public class CQLSSTableWriter implement
              }
          }
  
+         CFMetaData metadata()
+         {
+             return schema;
+         }
+ 
          /**
 -         * Adds the specified column family to the specified keyspace.
 +         * Creates the keyspace with the specified table.
           *
 -         * @param ksm the keyspace meta data
 -         * @param cfm the column family meta data
 +         * @param the table the table that must be created.
           */
 -        private static void addTableToKeyspace(KSMetaData ksm, CFMetaData cfm)
 +        private static void createKeyspaceWithTable(CFMetaData table)
          {
 -            ksm = KSMetaData.cloneWith(ksm, Iterables.concat(ksm.cfMetaData().values(), Collections.singleton(cfm)));
 -            Schema.instance.load(cfm);
 -            Schema.instance.setKeyspaceDefinition(ksm);
 +            KSMetaData ksm;
 +            ksm = KSMetaData.newKeyspace(table.ksName,
 +                                         AbstractReplicationStrategy.getClass("org.apache.cassandra.locator.SimpleStrategy"),
 +                                         ImmutableMap.of("replication_factor", "1"),
 +                                         true,
 +                                         Collections.singleton(table));
 +            Schema.instance.load(ksm);
          }
  
          /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
index 4bb75bc,25ec354..534e77b
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
@@@ -143,7 -140,13 +143,13 @@@ public class SSTableSimpleUnsortedWrite
          return previous;
      }
  
+     public Descriptor getCurrentDescriptor()
+     {
+         // can be implemented, but isn't necessary
+         throw new UnsupportedOperationException();
+     }
+ 
 -    protected ColumnFamily createColumnFamily() throws IOException
 +    protected ColumnFamily createColumnFamily()
      {
          return ArrayBackedSortedColumns.factory.create(metadata);
      }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
index 2601d6d,23da501..f81e57d
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
@@@ -26,7 -24,6 +26,8 @@@ import org.apache.cassandra.db.*
  import org.apache.cassandra.db.marshal.AbstractType;
  import org.apache.cassandra.dht.IPartitioner;
  import org.apache.cassandra.io.FSError;
++import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.io.sstable.format.SSTableWriter;
  
  /**
   * A SSTable writer that assumes rows are in (partitioner) sorted order.
@@@ -71,6 -65,13 +72,13 @@@ public class SSTableSimpleWriter extend
          writer = getWriter();
      }
  
+     SSTableReader closeAndOpenReader()
+     {
+         if (currentKey != null)
+             writeRow(currentKey, columnFamily);
 -        return writer.closeAndOpenReader();
++        return writer.finish(true);
+     }
+ 
      public void close()
      {
          try


[10/16] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by be...@apache.org.
Merge branch 'cassandra-2.1' into cassandra-2.2

Conflicts:
	src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
	src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
	src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java


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

Branch: refs/heads/cassandra-2.2
Commit: 25de92e321604626d6c098233082904832c07814
Parents: 7452b20 c37562e
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Wed Sep 30 19:47:14 2015 +0100
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Wed Sep 30 19:47:14 2015 +0100

----------------------------------------------------------------------
 .../io/sstable/AbstractSSTableSimpleWriter.java |   2 +
 .../cassandra/io/sstable/CQLSSTableWriter.java  |  15 +
 .../io/sstable/SSTableSimpleUnsortedWriter.java |   6 +
 .../io/sstable/SSTableSimpleWriter.java         |  13 +
 .../io/sstable/format/SSTableReader.java        |  17 +
 .../cassandra/io/util/ByteBufferDataInput.java  |   8 +-
 .../cassandra/io/util/MmappedSegmentedFile.java | 275 ++++++++++++----
 .../apache/cassandra/io/util/SegmentedFile.java |   1 +
 .../sstable/LongSegmentedFileBoundaryTest.java  | 324 +++++++++++++++++++
 9 files changed, 605 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
index 4181ed0,c364171..8873f88
--- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
@@@ -370,20 -376,22 +380,25 @@@ public class CQLSSTableWriter implement
              }
          }
  
+         CFMetaData metadata()
+         {
+             return schema;
+         }
+ 
          /**
 -         * Adds the specified column family to the specified keyspace.
 +         * Creates the keyspace with the specified table.
           *
 -         * @param ksm the keyspace meta data
 -         * @param cfm the column family meta data
 +         * @param the table the table that must be created.
           */
 -        private static void addTableToKeyspace(KSMetaData ksm, CFMetaData cfm)
 +        private static void createKeyspaceWithTable(CFMetaData table)
          {
 -            ksm = KSMetaData.cloneWith(ksm, Iterables.concat(ksm.cfMetaData().values(), Collections.singleton(cfm)));
 -            Schema.instance.load(cfm);
 -            Schema.instance.setKeyspaceDefinition(ksm);
 +            KSMetaData ksm;
 +            ksm = KSMetaData.newKeyspace(table.ksName,
 +                                         AbstractReplicationStrategy.getClass("org.apache.cassandra.locator.SimpleStrategy"),
 +                                         ImmutableMap.of("replication_factor", "1"),
 +                                         true,
 +                                         Collections.singleton(table));
 +            Schema.instance.load(ksm);
          }
  
          /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
index 4bb75bc,25ec354..534e77b
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
@@@ -143,7 -140,13 +143,13 @@@ public class SSTableSimpleUnsortedWrite
          return previous;
      }
  
+     public Descriptor getCurrentDescriptor()
+     {
+         // can be implemented, but isn't necessary
+         throw new UnsupportedOperationException();
+     }
+ 
 -    protected ColumnFamily createColumnFamily() throws IOException
 +    protected ColumnFamily createColumnFamily()
      {
          return ArrayBackedSortedColumns.factory.create(metadata);
      }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
index 2601d6d,23da501..f81e57d
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
@@@ -26,7 -24,6 +26,8 @@@ import org.apache.cassandra.db.*
  import org.apache.cassandra.db.marshal.AbstractType;
  import org.apache.cassandra.dht.IPartitioner;
  import org.apache.cassandra.io.FSError;
++import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.io.sstable.format.SSTableWriter;
  
  /**
   * A SSTable writer that assumes rows are in (partitioner) sorted order.
@@@ -71,6 -65,13 +72,13 @@@ public class SSTableSimpleWriter extend
          writer = getWriter();
      }
  
+     SSTableReader closeAndOpenReader()
+     {
+         if (currentKey != null)
+             writeRow(currentKey, columnFamily);
 -        return writer.closeAndOpenReader();
++        return writer.finish(true);
+     }
+ 
      public void close()
      {
          try


[11/16] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/util/ByteBufferDataInput.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/util/ByteBufferDataInput.java
index bf926e9,0000000..2f00687
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/io/util/ByteBufferDataInput.java
+++ b/src/java/org/apache/cassandra/io/util/ByteBufferDataInput.java
@@@ -1,171 -1,0 +1,177 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.io.util;
 +
 +import java.io.*;
 +import java.nio.ByteBuffer;
 +
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +
 +public class ByteBufferDataInput extends AbstractDataInput implements FileDataInput, DataInput
 +{
 +    private final ByteBuffer buffer;
 +    private final String filename;
 +    private final long segmentOffset;
 +    private int position;
 +
 +    public ByteBufferDataInput(ByteBuffer buffer, String filename, long segmentOffset, int position)
 +    {
 +        assert buffer != null;
 +        this.buffer = buffer;
 +        this.filename = filename;
 +        this.segmentOffset = segmentOffset;
 +        this.position = position;
 +    }
 +
 +    // Only use when we know the seek in within the mapped segment. Throws an
 +    // IOException otherwise.
 +    public void seek(long pos) throws IOException
 +    {
 +        long inSegmentPos = pos - segmentOffset;
-         if (inSegmentPos < 0 || inSegmentPos > buffer.capacity())
++        if (!contains(pos))
 +            throw new IOException(String.format("Seek position %d is not within mmap segment (seg offs: %d, length: %d)", pos, segmentOffset, buffer.capacity()));
 +
 +        position = (int) inSegmentPos;
 +    }
 +
++    public boolean contains(long pos)
++    {
++        long inSegmentPos = pos - segmentOffset;
++        return inSegmentPos >= 0 && inSegmentPos < buffer.capacity();
++    }
++
 +    public long getFilePointer()
 +    {
 +        return segmentOffset + position;
 +    }
 +
 +    public long getPosition()
 +    {
 +        return segmentOffset + position;
 +    }
 +
 +    public long getPositionLimit()
 +    {
 +        return segmentOffset + buffer.capacity();
 +    }
 +
 +    @Override
 +    public boolean markSupported()
 +    {
 +        return false;
 +    }
 +
 +    public void reset(FileMark mark) throws IOException
 +    {
 +        assert mark instanceof MappedFileDataInputMark;
 +        position = ((MappedFileDataInputMark) mark).position;
 +    }
 +
 +    public FileMark mark()
 +    {
 +        return new MappedFileDataInputMark(position);
 +    }
 +
 +    public long bytesPastMark(FileMark mark)
 +    {
 +        assert mark instanceof MappedFileDataInputMark;
 +        assert position >= ((MappedFileDataInputMark) mark).position;
 +        return position - ((MappedFileDataInputMark) mark).position;
 +    }
 +
 +    public boolean isEOF() throws IOException
 +    {
 +        return position == buffer.capacity();
 +    }
 +
 +    public long bytesRemaining() throws IOException
 +    {
 +        return buffer.capacity() - position;
 +    }
 +
 +    public String getPath()
 +    {
 +        return filename;
 +    }
 +
 +    public int read() throws IOException
 +    {
 +        if (isEOF())
 +            return -1;
 +        return buffer.get(position++) & 0xFF;
 +    }
 +
 +    /**
 +     * Does the same thing as <code>readFully</code> do but without copying data (thread safe)
 +     * @param length length of the bytes to read
 +     * @return buffer with portion of file content
 +     * @throws IOException on any fail of I/O operation
 +     */
 +    public ByteBuffer readBytes(int length) throws IOException
 +    {
 +        int remaining = buffer.remaining() - position;
 +        if (length > remaining)
 +            throw new IOException(String.format("mmap segment underflow; remaining is %d but %d requested",
 +                                                remaining, length));
 +
 +        if (length == 0)
 +            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
 +
 +        ByteBuffer bytes = buffer.duplicate();
 +        bytes.position(buffer.position() + position).limit(buffer.position() + position + length);
 +        position += length;
 +
 +        // we have to copy the data in case we unreference the underlying sstable.  See CASSANDRA-3179
 +        ByteBuffer clone = ByteBuffer.allocate(bytes.remaining());
 +        clone.put(bytes);
 +        clone.flip();
 +        return clone;
 +    }
 +
 +    @Override
 +    public final void readFully(byte[] bytes) throws IOException
 +    {
 +        ByteBufferUtil.arrayCopy(buffer, buffer.position() + position, bytes, 0, bytes.length);
 +        position += bytes.length;
 +    }
 +
 +    @Override
 +    public final void readFully(byte[] bytes, int offset, int count) throws IOException
 +    {
 +        ByteBufferUtil.arrayCopy(buffer, buffer.position() + position, bytes, offset, count);
 +        position += count;
 +    }
 +
 +    private static class MappedFileDataInputMark implements FileMark
 +    {
 +        int position;
 +
 +        MappedFileDataInputMark(int position)
 +        {
 +            this.position = position;
 +        }
 +    }
 +
 +    @Override
 +    public String toString() {
 +        return getClass().getSimpleName() + "(" +
 +               "filename='" + filename + "'" +
 +               ", position=" + position +
 +               ")";
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
index a8fae9f,623f65a..808b5ad
--- a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
@@@ -27,6 -28,13 +28,12 @@@ import com.google.common.annotations.Vi
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
+ import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.db.RowIndexEntry;
 -import org.apache.cassandra.io.FSReadError;
+ import org.apache.cassandra.io.sstable.Component;
+ import org.apache.cassandra.io.sstable.Descriptor;
+ import org.apache.cassandra.io.sstable.IndexSummary;
+ import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.utils.JVMStabilityInspector;
  
  public class MmappedSegmentedFile extends SegmentedFile
@@@ -135,6 -141,73 +142,74 @@@
          }
      }
  
+     // see CASSANDRA-10357
+     public static boolean maybeRepair(CFMetaData metadata, Descriptor descriptor, IndexSummary indexSummary, SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
+     {
+         boolean mayNeedRepair = false;
+         if (ibuilder instanceof Builder)
+             mayNeedRepair = ((Builder) ibuilder).mayNeedRepair(descriptor.filenameFor(Component.PRIMARY_INDEX));
+         if (dbuilder instanceof Builder)
+             mayNeedRepair |= ((Builder) dbuilder).mayNeedRepair(descriptor.filenameFor(Component.DATA));
+ 
+         if (mayNeedRepair)
+             forceRepair(metadata, descriptor, indexSummary, ibuilder, dbuilder);
+         return mayNeedRepair;
+     }
+ 
+     // if one of the index/data files have boundaries larger than we can mmap, and they were written by a version that did not guarantee correct boundaries were saved,
+     // rebuild the boundaries and save them again
+     private static void forceRepair(CFMetaData metadata, Descriptor descriptor, IndexSummary indexSummary, SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
+     {
+         if (ibuilder instanceof Builder)
+             ((Builder) ibuilder).boundaries.clear();
+         if (dbuilder instanceof Builder)
+             ((Builder) dbuilder).boundaries.clear();
+ 
++        RowIndexEntry.IndexSerializer rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata);
+         try (RandomAccessFile raf = new RandomAccessFile(descriptor.filenameFor(Component.PRIMARY_INDEX), "r");)
+         {
+             long iprev = 0, dprev = 0;
+             for (int i = 0; i < indexSummary.size(); i++)
+             {
+                 // first read the position in the summary, and read the corresponding position in the data file
+                 long icur = indexSummary.getPosition(i);
+                 raf.seek(icur);
+                 ByteBufferUtil.readWithShortLength(raf);
 -                RowIndexEntry rie = metadata.comparator.rowIndexEntrySerializer().deserialize(raf, descriptor.version);
++                RowIndexEntry rie = rowIndexEntrySerializer.deserialize(raf, descriptor.version);
+                 long dcur = rie.position;
+ 
+                 // if these positions are small enough to map out a segment from the prior version (i.e. less than 2Gb),
+                 // just add these as a boundary and proceed to the next index summary record; most scenarios will be
+                 // served by this, keeping the cost of rebuild to a minimum.
+ 
+                 if (Math.max(icur - iprev , dcur - dprev) > MAX_SEGMENT_SIZE)
+                 {
+                     // otherwise, loop over its index block, providing each RIE as a potential boundary for both files
+                     raf.seek(iprev);
+                     while (raf.getFilePointer() < icur)
+                     {
+                         // add the position of this record in the index file as an index file boundary
+                         ibuilder.addPotentialBoundary(raf.getFilePointer());
+                         // then read the RIE, and add its data file position as a boundary for the data file
+                         ByteBufferUtil.readWithShortLength(raf);
 -                        rie = metadata.comparator.rowIndexEntrySerializer().deserialize(raf, descriptor.version);
++                        rie = rowIndexEntrySerializer.deserialize(raf, descriptor.version);
+                         dbuilder.addPotentialBoundary(rie.position);
+                     }
+                 }
+ 
+                 ibuilder.addPotentialBoundary(icur);
+                 dbuilder.addPotentialBoundary(dcur);
+ 
+                 iprev = icur;
+                 dprev = dcur;
+             }
+         }
+         catch (IOException e)
+         {
+             logger.error("Failed to recalculate boundaries for {}; mmap access may degrade to buffered for this file", descriptor);
+         }
+     }
+ 
      /**
       * Overrides the default behaviour to create segments of a maximum size.
       */
@@@ -153,68 -326,83 +328,58 @@@
          public Builder()
          {
              super();
-             boundaries = new ArrayList<>();
-             boundaries.add(0L);
          }
  
-         public void addPotentialBoundary(long boundary)
+         public long[] boundaries()
          {
-             if (boundary - currentStart <= MAX_SEGMENT_SIZE)
-             {
-                 // boundary fits into current segment: expand it
-                 currentSize = boundary - currentStart;
-                 return;
-             }
+             return boundaries.truncate();
+         }
  
-             // close the current segment to try and make room for the boundary
-             if (currentSize > 0)
-             {
-                 currentStart += currentSize;
-                 boundaries.add(currentStart);
-             }
-             currentSize = boundary - currentStart;
+         // indicates if we may need to repair the mmapped file boundaries. this is a cheap check to see if there
+         // are any spans larger than an mmap segment size, which should be rare to occur in practice.
+         boolean mayNeedRepair(String path)
+         {
+             // old boundaries were created without the length, so add it as a candidate
+             long length = new File(path).length();
+             boundaries.addCandidate(length);
+             long[] boundaries = this.boundaries.truncate();
  
-             // if we couldn't make room, the boundary needs its own segment
-             if (currentSize > MAX_SEGMENT_SIZE)
+             long prev = 0;
+             for (long boundary : boundaries)
              {
-                 currentStart = boundary;
-                 boundaries.add(currentStart);
-                 currentSize = 0;
+                 if (boundary - prev > MAX_SEGMENT_SIZE)
+                     return true;
+                 prev = boundary;
              }
+             return false;
+         }
+ 
+         public void addPotentialBoundary(long boundary)
+         {
+             boundaries.addCandidate(boundary);
          }
  
 -        public SegmentedFile complete(String path, long overrideLength, boolean isFinal)
 +        public SegmentedFile complete(ChannelProxy channel, long overrideLength)
          {
 -            assert !isFinal || overrideLength <= 0;
 -            long length = overrideLength > 0 ? overrideLength : new File(path).length();
 +            long length = overrideLength > 0 ? overrideLength : channel.size();
              // create the segments
-             return new MmappedSegmentedFile(channel, length, createSegments(channel, length));
 -            return new MmappedSegmentedFile(path, length, createSegments(path, length, isFinal));
--        }
 -
 -        private Segment[] createSegments(String path, long length, boolean isFinal)
 -        {
 -            RandomAccessFile raf;
 -            try
 -            {
 -                raf = new RandomAccessFile(path, "r");
 -            }
 -            catch (IOException e)
 -            {
 -                throw new RuntimeException(e);
 -            }
  
-         private Segment[] createSegments(ChannelProxy channel, long length)
-         {
-             // if we're early finishing a range that doesn't span multiple segments, but the finished file now does,
-             // we remove these from the end (we loop incase somehow this spans multiple segments, but that would
-             // be a loco dataset
-             while (length < boundaries.get(boundaries.size() - 1))
-                 boundaries.remove(boundaries.size() -1);
- 
-             // add a sentinel value == length
-             List<Long> boundaries = new ArrayList<>(this.boundaries);
-             if (length != boundaries.get(boundaries.size() - 1))
-                 boundaries.add(length);
- 
-             int segcount = boundaries.size() - 1;
 -            long[] boundaries = this.boundaries.finish(length, isFinal);
++            long[] boundaries = this.boundaries.finish(length, overrideLength <= 0);
+ 
+             int segcount = boundaries.length - 1;
              Segment[] segments = new Segment[segcount];
+ 
 -            try
 -            {
 -                for (int i = 0; i < segcount; i++)
 -                {
 -                    long start = boundaries[i];
 -                    long size = boundaries[i + 1] - start;
 -                    MappedByteBuffer segment = size <= MAX_SEGMENT_SIZE
 -                                               ? raf.getChannel().map(FileChannel.MapMode.READ_ONLY, start, size)
 -                                               : null;
 -                    segments[i] = new Segment(start, segment);
 -                }
 -            }
 -            catch (IOException e)
 -            {
 -                throw new FSReadError(e, path);
 -            }
 -            finally
 +            for (int i = 0; i < segcount; i++)
              {
-                 long start = boundaries.get(i);
-                 long size = boundaries.get(i + 1) - start;
 -                FileUtils.closeQuietly(raf);
++                long start = boundaries[i];
++                long size = boundaries[i + 1] - start;
 +                MappedByteBuffer segment = size <= MAX_SEGMENT_SIZE
 +                                           ? channel.map(FileChannel.MapMode.READ_ONLY, start, size)
 +                                           : null;
 +                segments[i] = new Segment(start, segment);
              }
--            return segments;
++
++            return new MmappedSegmentedFile(channel, length, segments);
          }
  
          @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/util/SegmentedFile.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/util/SegmentedFile.java
index 66898c6,23454bc..30707d8
--- a/src/java/org/apache/cassandra/io/util/SegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/SegmentedFile.java
@@@ -23,9 -23,9 +23,10 @@@ import java.io.File
  import java.io.IOException;
  import java.nio.MappedByteBuffer;
  import java.util.Iterator;
+ import java.util.List;
  import java.util.NoSuchElementException;
  
 +import com.google.common.base.Throwables;
  import com.google.common.util.concurrent.RateLimiter;
  
  import org.apache.cassandra.config.Config;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
----------------------------------------------------------------------
diff --cc test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
index 0000000,e17c6a7..4913b32
mode 000000,100644..100644
--- a/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
+++ b/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
@@@ -1,0 -1,322 +1,324 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ package org.apache.cassandra.io.sstable;
+ 
+ import java.io.*;
+ import java.nio.ByteBuffer;
+ import java.util.Arrays;
+ import java.util.Random;
+ 
+ import com.google.common.io.Files;
+ import org.junit.AfterClass;
+ import org.junit.BeforeClass;
+ import org.junit.Test;
+ 
+ import junit.framework.Assert;
+ import org.apache.cassandra.SchemaLoader;
+ import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.config.Config;
+ import org.apache.cassandra.config.DatabaseDescriptor;
+ import org.apache.cassandra.db.Keyspace;
+ import org.apache.cassandra.dht.ByteOrderedPartitioner;
+ import org.apache.cassandra.exceptions.InvalidRequestException;
++import org.apache.cassandra.io.sstable.format.SSTableReader;
+ import org.apache.cassandra.io.util.DataOutputStreamPlus;
+ import org.apache.cassandra.io.util.FileUtils;
+ import org.apache.cassandra.io.util.MmappedSegmentedFile;
+ import org.apache.cassandra.io.util.MmappedSegmentedFile.Builder.Boundaries;
++import org.apache.cassandra.io.util.WrappedDataOutputStreamPlus;
+ import org.apache.cassandra.service.StorageService;
+ import org.apache.cassandra.utils.ByteBufferUtil;
+ 
+ public class LongSegmentedFileBoundaryTest
+ {
+     @BeforeClass
+     public static void setup() throws Exception
+     {
+         SchemaLoader.cleanupAndLeaveDirs();
+         Keyspace.setInitialized();
+         StorageService.instance.initServer();
+     }
+ 
+     @AfterClass
+     public static void tearDown()
+     {
+         Config.setClientMode(false);
+     }
+ 
+     @Test
+     public void testRandomBoundaries()
+     {
+         long[] candidates = new long[1 + (1 << 16)];
+         int[] indexesToCheck = new int[1 << 8];
+         Random random = new Random();
+ 
+         for (int run = 0; run < 100; run++)
+         {
+ 
+             long seed = random.nextLong();
+             random.setSeed(seed);
+             System.out.println("Seed: " + seed);
+ 
+             // at least 1Ki, and as many as 256Ki, boundaries
+             int candidateCount = (1 + random.nextInt(candidates.length >> 10)) << 10;
+             generateBoundaries(random, candidateCount, candidates, indexesToCheck);
+ 
+             Boundaries builder = new Boundaries();
+             int nextIndexToCheck = indexesToCheck[0];
+             int checkCount = 0;
+             System.out.printf("[0..%d)", candidateCount);
+             for (int i = 1; i < candidateCount - 1; i++)
+             {
+                 if (i == nextIndexToCheck)
+                 {
+                     if (checkCount % 20 == 0)
+                         System.out.printf(" %d", i);
+                     // grow number of samples logarithmically; work will still increase superlinearly, as size of dataset grows linearly
+                     int sampleCount = 1 << (31 - Integer.numberOfLeadingZeros(++checkCount));
+                     checkBoundarySample(random, candidates, i, sampleCount, builder);
+                     // select out next index to check (there may be dups, so skip them)
+                     while ((nextIndexToCheck = checkCount == indexesToCheck.length ? candidateCount : indexesToCheck[checkCount]) == i)
+                         checkCount++;
+                 }
+ 
+                 builder.addCandidate(candidates[i]);
+             }
+             System.out.println();
+             checkBoundaries(candidates, candidateCount - 1, builder, candidates[candidateCount - 1]);
+             Assert.assertEquals(candidateCount, nextIndexToCheck);
+         }
+     }
+ 
+     private static void generateBoundaries(Random random, int candidateCount, long[] candidates, int[] indexesToCheck)
+     {
+         // average averageBoundarySize is 4MiB, max 4GiB, min 4KiB
+         long averageBoundarySize = (4L << 10) * random.nextInt(1 << 20);
+         long prev = 0;
+         for (int i = 1 ; i < candidateCount ; i++)
+             candidates[i] = prev += Math.max(1, averageBoundarySize + (random.nextGaussian() * averageBoundarySize));
+ 
+         // generate indexes we will corroborate our behaviour on
+         for (int i = 0 ; i < indexesToCheck.length ; i++)
+             indexesToCheck[i] = 1 + random.nextInt(candidateCount - 2);
+         Arrays.sort(indexesToCheck);
+     }
+ 
+     private static void checkBoundarySample(Random random, long[] candidates, int candidateCount, int sampleCount, Boundaries builder)
+     {
+         for (int i = 0 ; i < sampleCount ; i++)
+         {
+             // pick a number exponentially less likely to be near the beginning, since we test that area earlier
+             int position = 0 ;
+             while (position <= 0)
+                 position = candidateCount / (Integer.lowestOneBit(random.nextInt()));
+             long upperBound = candidates[position];
+             long lowerBound = random.nextBoolean() ? (rand(random, 0, upperBound) / (Integer.lowestOneBit(random.nextInt())))
+                                                    : candidates[Math.max(0, position - random.nextInt(64))];
+             long length = rand(random, lowerBound, upperBound);
+             checkBoundaries(candidates, candidateCount, builder, length);
+         }
+         checkBoundaries(candidates, candidateCount, builder, candidates[candidateCount]);
+     }
+ 
+     private static long rand(Random random, long lowerBound, long upperBound)
+     {
+         if (upperBound == lowerBound)
+             return upperBound;
+         return lowerBound + ((random.nextLong() & Long.MAX_VALUE) % (upperBound - lowerBound));
+     }
+ 
+     private static void checkBoundaries(long[] candidates, int candidateCount, Boundaries builder, long length)
+     {
+         if (length == 0)
+             return;
+ 
+         long[] boundaries = new long[(int) (10 + 2 * (length / Integer.MAX_VALUE))];
+         int count = 1;
+         int prev = 0;
+         while (true)
+         {
+             int p = candidates[prev + 1] - boundaries[count - 1] >= Integer.MAX_VALUE
+                     ? prev + 1
+                     : Arrays.binarySearch(candidates, prev, candidateCount, boundaries[count - 1] + Integer.MAX_VALUE);
+             if (p < 0) p = -2 -p;
+             if (p >= candidateCount - 1 || candidates[p] >= length)
+                 break;
+             boundaries[count++] = candidates[p];
+             if (candidates[p + 1] >= length)
+                 break;
+             prev = p;
+         }
+         if (candidates[candidateCount - 1] < length && length - boundaries[count - 1] >= Integer.MAX_VALUE)
+             boundaries[count++] = candidates[candidateCount - 1];
+         boundaries[count++] = length;
+         final long[] canon = Arrays.copyOf(boundaries, count);
+         final long[] check = builder.finish(length, false);
+         if (!Arrays.equals(canon, check))
+             Assert.assertTrue("\n" + Arrays.toString(canon) + "\n" + Arrays.toString(check), Arrays.equals(canon, check));
+     }
+ 
+     @Test
+     public void testBoundariesAndRepairSmall() throws InvalidRequestException, IOException
+     {
+         testBoundariesAndRepair(1, 1 << 16);
+     }
+ 
+     @Test
+     public void testBoundariesAndRepairMedium() throws InvalidRequestException, IOException
+     {
+         testBoundariesAndRepair(1, 1 << 20);
+     }
+ 
+     @Test
+     public void testBoundariesAndRepairLarge() throws InvalidRequestException, IOException
+     {
+         testBoundariesAndRepair(1, 100 << 20);
+     }
+ 
+     @Test
+     public void testBoundariesAndRepairHuge() throws InvalidRequestException, IOException
+     {
+         testBoundariesAndRepair(1, Integer.MAX_VALUE - 1024);
+     }
+ 
+     @Test
+     public void testBoundariesAndRepairTooHuge() throws InvalidRequestException, IOException
+     {
+         testBoundariesAndRepair(1, Integer.MAX_VALUE);
+     }
+ 
+     @Test
+     public void testBoundariesAndRepairHugeIndex() throws InvalidRequestException, IOException
+     {
+         testBoundariesAndRepair(1 << 7, 1 << 15);
+     }
+ 
+     @Test
+     public void testBoundariesAndRepairReallyHugeIndex() throws InvalidRequestException, IOException
+     {
+         testBoundariesAndRepair(1 << 14, 1 << 15);
+     }
+ 
+     private void testBoundariesAndRepair(int rows, int rowSize) throws InvalidRequestException, IOException
+     {
+         String KS = "cql_keyspace";
+         String TABLE = "table1";
+ 
+         File tempdir = Files.createTempDir();
+         try
+         {
+             Assert.assertTrue(DatabaseDescriptor.getColumnIndexSize() < rowSize);
+             Assert.assertTrue(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap);
+             Assert.assertTrue(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap);
+             Assert.assertTrue(StorageService.getPartitioner() instanceof ByteOrderedPartitioner);
+             File dataDir = new File(tempdir.getAbsolutePath() + File.separator + KS + File.separator + TABLE);
+             Assert.assertTrue(dataDir.mkdirs());
+ 
+             String schema = "CREATE TABLE cql_keyspace.table" + (rows > 1 ? "2" : "1") + " (k bigint, v1 blob, v2 blob, v3 blob, v4 blob, v5 blob, PRIMARY KEY (k" + (rows > 1 ? ", v1" : "") + ")) WITH compression = { 'sstable_compression':'' };";
+             String insert = "INSERT INTO cql_keyspace.table" + (rows > 1 ? "2" : "1") + " (k, v1, v2, v3, v4, v5) VALUES (?, ?, ?, ?, ?, ?)";
+ 
+             CQLSSTableWriter.Builder builder = CQLSSTableWriter.builder()
+                                                       .inDirectory(dataDir)
+                                                       .forTable(schema)
+                                                       .withPartitioner(StorageService.getPartitioner())
+                                                       .using(insert)
+                                                       .sorted();
+             CQLSSTableWriter writer = builder.build();
+ 
+             // write 8Gb of decorated keys
+             ByteBuffer[] value = new ByteBuffer[rows];
+             for (int row = 0 ; row < rows ; row++)
+             {
+                 // if we're using clustering columns, the clustering key is replicated across every other column
+                 value[row] = ByteBuffer.allocate(rowSize / (rows > 1 ? 8 : 5));
+                 value[row].putInt(0, row);
+             }
+             long targetSize = 8L << 30;
+             long dk = 0;
+             long size = 0;
+             long dkSize = rowSize * rows;
+             while (size < targetSize)
+             {
+                 for (int row = 0 ; row < rows ; row++)
+                     writer.addRow(dk, value[row], value[row], value[row], value[row], value[row]);
+                 size += dkSize;
+                 dk++;
+             }
+ 
+             Descriptor descriptor = writer.getCurrentDescriptor().asType(Descriptor.Type.FINAL);
+             writer.close();
+ 
+             // open (and close) the reader so that the summary file is created
+             SSTableReader reader = SSTableReader.open(descriptor);
+             reader.selfRef().release();
+ 
+             // then check the boundaries are reasonable, and corrupt them
+             checkThenCorruptBoundaries(descriptor, rows * rowSize < Integer.MAX_VALUE);
+ 
+             // then check that reopening corrects the corruption
+             reader = SSTableReader.open(descriptor);
+             reader.selfRef().release();
+             checkThenCorruptBoundaries(descriptor, rows * rowSize < Integer.MAX_VALUE);
+         }
+         finally
+         {
+             FileUtils.deleteRecursive(tempdir);
+         }
+     }
+ 
+     private static void checkThenCorruptBoundaries(Descriptor descriptor, boolean expectDataMmappable) throws IOException
+     {
+         File summaryFile = new File(descriptor.filenameFor(Component.SUMMARY));
+         DataInputStream iStream = new DataInputStream(new FileInputStream(summaryFile));
+         IndexSummary indexSummary = IndexSummary.serializer.deserialize(iStream, StorageService.getPartitioner(), true, CFMetaData.DEFAULT_MIN_INDEX_INTERVAL, CFMetaData.DEFAULT_MAX_INDEX_INTERVAL);
+         ByteBuffer first = ByteBufferUtil.readWithLength(iStream);
+         ByteBuffer last = ByteBufferUtil.readWithLength(iStream);
+         MmappedSegmentedFile.Builder ibuilder = new MmappedSegmentedFile.Builder();
+         MmappedSegmentedFile.Builder dbuilder = new MmappedSegmentedFile.Builder();
+         ibuilder.deserializeBounds(iStream);
+         dbuilder.deserializeBounds(iStream);
+         iStream.close();
+         // index file cannot generally be non-mmappable, as index entries cannot be larger than MAX_SEGMENT_SIZE (due to promotedSize being encoded as an int)
+         assertBoundaries(descriptor.filenameFor(Component.PRIMARY_INDEX), true, ibuilder.boundaries());
+         assertBoundaries(descriptor.filenameFor(Component.DATA), expectDataMmappable, dbuilder.boundaries());
+ 
 -        DataOutputStreamPlus oStream = new DataOutputStreamPlus(new FileOutputStream(summaryFile));
++        DataOutputStreamPlus oStream = new WrappedDataOutputStreamPlus(new FileOutputStream(summaryFile));
+         IndexSummary.serializer.serialize(indexSummary, oStream, true);
+         ByteBufferUtil.writeWithLength(first, oStream);
+         ByteBufferUtil.writeWithLength(last, oStream);
+         oStream.writeInt(1);
+         oStream.writeLong(new File(descriptor.filenameFor(Component.PRIMARY_INDEX)).length());
+         oStream.writeLong(new File(descriptor.filenameFor(Component.DATA)).length());
+         oStream.close();
+     }
+ 
+     private static void assertBoundaries(String path, boolean expectMmappable, long[] boundaries)
+     {
+         long length = new File(path).length();
+         long prev = boundaries[0];
+         for (int i = 1 ; i <= boundaries.length && prev < length ; i++)
+         {
+             long boundary = i == boundaries.length ? length : boundaries[i];
+             Assert.assertEquals(String.format("[%d, %d), %d of %d", boundary, prev, i, boundaries.length),
+                                 expectMmappable, boundary - prev <= Integer.MAX_VALUE);
+             prev = boundary;
+         }
+     }
+ 
+ }


[04/16] cassandra git commit: Fix Mmapped File Boundaries

Posted by be...@apache.org.
Fix Mmapped File Boundaries

This patch fixes two bugs with mmap segment boundary
tracking, and introduces automated correction of
this bug on startup

patch by benedict; reviewed by tjake for CASSANDRA-10357


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

Branch: refs/heads/trunk
Commit: c37562e345c24720c55428a8644191df68319812
Parents: f6cab37
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Wed Sep 16 18:09:32 2015 +0100
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Wed Sep 30 19:45:49 2015 +0100

----------------------------------------------------------------------
 .../io/sstable/AbstractSSTableSimpleWriter.java |   2 +
 .../cassandra/io/sstable/CQLSSTableWriter.java  |  15 +
 .../cassandra/io/sstable/SSTableReader.java     |  34 +-
 .../io/sstable/SSTableSimpleUnsortedWriter.java |   6 +
 .../io/sstable/SSTableSimpleWriter.java         |  12 +
 .../cassandra/io/util/MappedFileDataInput.java  |   8 +-
 .../cassandra/io/util/MmappedSegmentedFile.java | 270 +++++++++++++---
 .../apache/cassandra/io/util/SegmentedFile.java |   1 +
 .../sstable/LongSegmentedFileBoundaryTest.java  | 322 +++++++++++++++++++
 9 files changed, 601 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
index 165a4b2..557c3de 100644
--- a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
@@ -200,4 +200,6 @@ public abstract class AbstractSSTableSimpleWriter implements Closeable
     protected abstract void writeRow(DecoratedKey key, ColumnFamily columnFamily) throws IOException;
 
     protected abstract ColumnFamily getColumnFamily() throws IOException;
+
+    public abstract Descriptor getCurrentDescriptor();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
index b211a90..c364171 100644
--- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
@@ -271,6 +271,16 @@ public class CQLSSTableWriter implements Closeable
         writer.close();
     }
 
+    public Descriptor getCurrentDescriptor()
+    {
+        return writer.getCurrentDescriptor();
+    }
+
+    public CFMetaData getCFMetaData()
+    {
+        return writer.metadata;
+    }
+
     /**
      * A Builder for a CQLSSTableWriter object.
      */
@@ -366,6 +376,11 @@ public class CQLSSTableWriter implements Closeable
             }
         }
 
+        CFMetaData metadata()
+        {
+            return schema;
+        }
+
         /**
          * Adds the specified column family to the specified keyspace.
          *

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/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 0f307b0..84add6f 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
@@ -17,13 +17,7 @@
  */
 package org.apache.cassandra.io.sstable;
 
-import java.io.BufferedInputStream;
-import java.io.DataInputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.IOException;
+import java.io.*;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.*;
@@ -70,20 +64,14 @@ import org.apache.cassandra.dht.LocalPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.FSError;
+import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.io.compress.CompressionMetadata;
 import org.apache.cassandra.io.sstable.metadata.CompactionMetadata;
 import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
 import org.apache.cassandra.io.sstable.metadata.MetadataType;
 import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
 import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
-import org.apache.cassandra.io.util.BufferedSegmentedFile;
-import org.apache.cassandra.io.util.CompressedSegmentedFile;
-import org.apache.cassandra.io.util.DataOutputStreamAndChannel;
-import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.io.util.ICompressedFile;
-import org.apache.cassandra.io.util.RandomAccessReader;
-import org.apache.cassandra.io.util.SegmentedFile;
+import org.apache.cassandra.io.util.*;
 import org.apache.cassandra.metrics.RestorableMeter;
 import org.apache.cassandra.metrics.StorageMetrics;
 import org.apache.cassandra.service.ActiveRepairService;
@@ -162,6 +150,7 @@ import static org.apache.cassandra.db.Directories.SECONDARY_INDEX_NAME_SEPARATOR
 public class SSTableReader extends SSTable implements SelfRefCounted<SSTableReader>
 {
     private static final Logger logger = LoggerFactory.getLogger(SSTableReader.class);
+    private static final int ACCURATE_BOUNDARIES_MAGIC_NUMBER = 248923458;
 
     private static final ScheduledThreadPoolExecutor syncExecutor = new ScheduledThreadPoolExecutor(1);
     static
@@ -892,6 +881,19 @@ public class SSTableReader extends SSTable implements SelfRefCounted<SSTableRead
             last = partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
             ibuilder.deserializeBounds(iStream);
             dbuilder.deserializeBounds(iStream);
+
+            boolean checkForRepair = true;
+            try
+            {
+                int v = iStream.readInt();
+                // check for our magic number, indicating this summary has been sampled correctly
+                checkForRepair = v != ACCURATE_BOUNDARIES_MAGIC_NUMBER;
+            }
+            catch (Throwable t) {}
+
+            // fix CASSANDRA-10357 on-the-fly
+            if (checkForRepair && MmappedSegmentedFile.maybeRepair(metadata, descriptor, indexSummary, ibuilder, dbuilder))
+                saveSummary(ibuilder, dbuilder);
         }
         catch (IOException e)
         {
@@ -992,6 +994,8 @@ public class SSTableReader extends SSTable implements SelfRefCounted<SSTableRead
             ByteBufferUtil.writeWithLength(last.getKey(), oStream);
             ibuilder.serializeBounds(oStream);
             dbuilder.serializeBounds(oStream);
+            // write a magic number, to indicate this summary has been sampled correctly
+            oStream.writeInt(ACCURATE_BOUNDARIES_MAGIC_NUMBER);
         }
         catch (IOException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
index 9ee9ea1..25ec354 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
@@ -140,6 +140,12 @@ public class SSTableSimpleUnsortedWriter extends AbstractSSTableSimpleWriter
         return previous;
     }
 
+    public Descriptor getCurrentDescriptor()
+    {
+        // can be implemented, but isn't necessary
+        throw new UnsupportedOperationException();
+    }
+
     protected ColumnFamily createColumnFamily() throws IOException
     {
         return ArrayBackedSortedColumns.factory.create(metadata);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
index 87c8e33..23da501 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
@@ -65,6 +65,13 @@ public class SSTableSimpleWriter extends AbstractSSTableSimpleWriter
         writer = getWriter();
     }
 
+    SSTableReader closeAndOpenReader()
+    {
+        if (currentKey != null)
+            writeRow(currentKey, columnFamily);
+        return writer.closeAndOpenReader();
+    }
+
     public void close()
     {
         try
@@ -89,4 +96,9 @@ public class SSTableSimpleWriter extends AbstractSSTableSimpleWriter
     {
         return ArrayBackedSortedColumns.factory.create(metadata);
     }
+
+    public Descriptor getCurrentDescriptor()
+    {
+        return writer.descriptor;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java b/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java
index d056240..f93ce72 100644
--- a/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java
+++ b/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java
@@ -51,12 +51,18 @@ public class MappedFileDataInput extends AbstractDataInput implements FileDataIn
     public void seek(long pos) throws IOException
     {
         long inSegmentPos = pos - segmentOffset;
-        if (inSegmentPos < 0 || inSegmentPos > buffer.capacity())
+        if (!contains(pos))
             throw new IOException(String.format("Seek position %d is not within mmap segment (seg offs: %d, length: %d)", pos, segmentOffset, buffer.capacity()));
 
         seekInternal((int) inSegmentPos);
     }
 
+    public boolean contains(long pos)
+    {
+        long inSegmentPos = pos - segmentOffset;
+        return inSegmentPos >= 0 && inSegmentPos < buffer.capacity();
+    }
+
     public long getFilePointer()
     {
         return segmentOffset + (long)position;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
index 1b23343..623f65a 100644
--- a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
@@ -24,11 +24,17 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.RowIndexEntry;
 import org.apache.cassandra.io.FSReadError;
-import org.apache.cassandra.io.sstable.SSTableWriter;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummary;
+import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 
 public class MmappedSegmentedFile extends SegmentedFile
@@ -135,52 +141,220 @@ public class MmappedSegmentedFile extends SegmentedFile
         }
     }
 
+    // see CASSANDRA-10357
+    public static boolean maybeRepair(CFMetaData metadata, Descriptor descriptor, IndexSummary indexSummary, SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
+    {
+        boolean mayNeedRepair = false;
+        if (ibuilder instanceof Builder)
+            mayNeedRepair = ((Builder) ibuilder).mayNeedRepair(descriptor.filenameFor(Component.PRIMARY_INDEX));
+        if (dbuilder instanceof Builder)
+            mayNeedRepair |= ((Builder) dbuilder).mayNeedRepair(descriptor.filenameFor(Component.DATA));
+
+        if (mayNeedRepair)
+            forceRepair(metadata, descriptor, indexSummary, ibuilder, dbuilder);
+        return mayNeedRepair;
+    }
+
+    // if one of the index/data files have boundaries larger than we can mmap, and they were written by a version that did not guarantee correct boundaries were saved,
+    // rebuild the boundaries and save them again
+    private static void forceRepair(CFMetaData metadata, Descriptor descriptor, IndexSummary indexSummary, SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
+    {
+        if (ibuilder instanceof Builder)
+            ((Builder) ibuilder).boundaries.clear();
+        if (dbuilder instanceof Builder)
+            ((Builder) dbuilder).boundaries.clear();
+
+        try (RandomAccessFile raf = new RandomAccessFile(descriptor.filenameFor(Component.PRIMARY_INDEX), "r");)
+        {
+            long iprev = 0, dprev = 0;
+            for (int i = 0; i < indexSummary.size(); i++)
+            {
+                // first read the position in the summary, and read the corresponding position in the data file
+                long icur = indexSummary.getPosition(i);
+                raf.seek(icur);
+                ByteBufferUtil.readWithShortLength(raf);
+                RowIndexEntry rie = metadata.comparator.rowIndexEntrySerializer().deserialize(raf, descriptor.version);
+                long dcur = rie.position;
+
+                // if these positions are small enough to map out a segment from the prior version (i.e. less than 2Gb),
+                // just add these as a boundary and proceed to the next index summary record; most scenarios will be
+                // served by this, keeping the cost of rebuild to a minimum.
+
+                if (Math.max(icur - iprev , dcur - dprev) > MAX_SEGMENT_SIZE)
+                {
+                    // otherwise, loop over its index block, providing each RIE as a potential boundary for both files
+                    raf.seek(iprev);
+                    while (raf.getFilePointer() < icur)
+                    {
+                        // add the position of this record in the index file as an index file boundary
+                        ibuilder.addPotentialBoundary(raf.getFilePointer());
+                        // then read the RIE, and add its data file position as a boundary for the data file
+                        ByteBufferUtil.readWithShortLength(raf);
+                        rie = metadata.comparator.rowIndexEntrySerializer().deserialize(raf, descriptor.version);
+                        dbuilder.addPotentialBoundary(rie.position);
+                    }
+                }
+
+                ibuilder.addPotentialBoundary(icur);
+                dbuilder.addPotentialBoundary(dcur);
+
+                iprev = icur;
+                dprev = dcur;
+            }
+        }
+        catch (IOException e)
+        {
+            logger.error("Failed to recalculate boundaries for {}; mmap access may degrade to buffered for this file", descriptor);
+        }
+    }
+
     /**
      * Overrides the default behaviour to create segments of a maximum size.
      */
-    static class Builder extends SegmentedFile.Builder
+    public static class Builder extends SegmentedFile.Builder
     {
-        // planned segment boundaries
-        private List<Long> boundaries;
+        @VisibleForTesting
+        public static class Boundaries
+        {
+            private long[] boundaries;
+
+            // number of boundaries we have "fixed" (i.e. have determined the final value of)
+            private int fixedCount;
+
+            public Boundaries()
+            {
+                // we always have a boundary of zero, so we start with a fixedCount of 1
+                this(new long[8], 1);
+            }
+
+            public Boundaries(long[] boundaries, int fixedCount)
+            {
+                init(boundaries, fixedCount);
+            }
+
+            void init(long[] boundaries, int fixedCount)
+            {
+                this.boundaries = boundaries;
+                this.fixedCount = fixedCount;
+            }
+
+            public void addCandidate(long candidate)
+            {
+                // we make sure we have room before adding another element, so that we can share the addCandidate logic statically
+                boundaries = ensureCapacity(boundaries, fixedCount);
+                fixedCount = addCandidate(boundaries, fixedCount, candidate);
+            }
+
+            private static int addCandidate(long[] boundaries, int fixedCount, long candidate)
+            {
+                // check how far we are from the last fixed boundary
+                long delta = candidate - boundaries[fixedCount - 1];
+                assert delta >= 0;
+                if (delta != 0)
+                {
+                    if (delta <= MAX_SEGMENT_SIZE)
+                        // overwrite the unfixed (potential) boundary if the resultant segment would still be mmappable
+                        boundaries[fixedCount] = candidate;
+                    else if (boundaries[fixedCount] == 0)
+                        // or, if it is not initialised, we cannot make an mmapped segment here, so this is the fixed boundary
+                        boundaries[fixedCount++] = candidate;
+                    else
+                        // otherwise, fix the prior boundary and initialise our unfixed boundary
+                        boundaries[++fixedCount] = candidate;
+                }
+                return fixedCount;
+            }
+
+            // ensures there is room for another fixed boundary AND an unfixed candidate boundary, i.e. fixedCount + 2 items
+            private static long[] ensureCapacity(long[] boundaries, int fixedCount)
+            {
+                if (fixedCount + 1 >= boundaries.length)
+                    return Arrays.copyOf(boundaries, boundaries.length * 2);
+                return boundaries;
+            }
+
+            void clear()
+            {
+                fixedCount = 1;
+                Arrays.fill(boundaries, 0);
+            }
+
+            // returns the fixed boundaries, truncated to a correctly sized long[]
+            public long[] truncate()
+            {
+                return Arrays.copyOf(boundaries, fixedCount);
+            }
 
-        // offset of the open segment (first segment begins at 0).
-        private long currentStart = 0;
+            // returns the finished boundaries for the provided length, truncated to a correctly sized long[]
+            public long[] finish(long length, boolean isFinal)
+            {
+                assert length > 0;
+                // ensure there's room for the length to be added
+                boundaries = ensureCapacity(boundaries, fixedCount);
+
+                // clone our current contents, so we don't corrupt them
+                int fixedCount = this.fixedCount;
+                long[] boundaries = this.boundaries.clone();
+
+                // if we're finishing early, our length may be before some of our boundaries,
+                // so walk backwards until our boundaries are <= length
+                while (boundaries[fixedCount - 1] >= length)
+                    boundaries[fixedCount--] = 0;
+                if (boundaries[fixedCount] >= length)
+                    boundaries[fixedCount] = 0;
+
+                // add our length as a boundary
+                fixedCount = addCandidate(boundaries, fixedCount, length);
+
+                // if we have any unfixed boundary at the end, it's now fixed, since we're done
+                if (boundaries[fixedCount] != 0)
+                    fixedCount++;
+
+                boundaries = Arrays.copyOf(boundaries, fixedCount);
+                if (isFinal)
+                {
+                    // if this is the final one, save it
+                    this.boundaries = boundaries;
+                    this.fixedCount = fixedCount;
+                }
+                return boundaries;
+            }
+        }
 
-        // current length of the open segment.
-        // used to allow merging multiple too-large-to-mmap segments, into a single buffered segment.
-        private long currentSize = 0;
+        private final Boundaries boundaries = new Boundaries();
 
         public Builder()
         {
             super();
-            boundaries = new ArrayList<>();
-            boundaries.add(0L);
         }
 
-        public void addPotentialBoundary(long boundary)
+        public long[] boundaries()
         {
-            if (boundary - currentStart <= MAX_SEGMENT_SIZE)
-            {
-                // boundary fits into current segment: expand it
-                currentSize = boundary - currentStart;
-                return;
-            }
+            return boundaries.truncate();
+        }
 
-            // close the current segment to try and make room for the boundary
-            if (currentSize > 0)
-            {
-                currentStart += currentSize;
-                boundaries.add(currentStart);
-            }
-            currentSize = boundary - currentStart;
+        // indicates if we may need to repair the mmapped file boundaries. this is a cheap check to see if there
+        // are any spans larger than an mmap segment size, which should be rare to occur in practice.
+        boolean mayNeedRepair(String path)
+        {
+            // old boundaries were created without the length, so add it as a candidate
+            long length = new File(path).length();
+            boundaries.addCandidate(length);
+            long[] boundaries = this.boundaries.truncate();
 
-            // if we couldn't make room, the boundary needs its own segment
-            if (currentSize > MAX_SEGMENT_SIZE)
+            long prev = 0;
+            for (long boundary : boundaries)
             {
-                currentStart = boundary;
-                boundaries.add(currentStart);
-                currentSize = 0;
+                if (boundary - prev > MAX_SEGMENT_SIZE)
+                    return true;
+                prev = boundary;
             }
+            return false;
+        }
+
+        public void addPotentialBoundary(long boundary)
+        {
+            boundaries.addCandidate(boundary);
         }
 
         public SegmentedFile complete(String path, long overrideLength, boolean isFinal)
@@ -188,10 +362,10 @@ public class MmappedSegmentedFile extends SegmentedFile
             assert !isFinal || overrideLength <= 0;
             long length = overrideLength > 0 ? overrideLength : new File(path).length();
             // create the segments
-            return new MmappedSegmentedFile(path, length, createSegments(path, length));
+            return new MmappedSegmentedFile(path, length, createSegments(path, length, isFinal));
         }
 
-        private Segment[] createSegments(String path, long length)
+        private Segment[] createSegments(String path, long length, boolean isFinal)
         {
             RandomAccessFile raf;
             try
@@ -203,27 +377,17 @@ public class MmappedSegmentedFile extends SegmentedFile
                 throw new RuntimeException(e);
             }
 
-            // if we're early finishing a range that doesn't span multiple segments, but the finished file now does,
-            // we remove these from the end (we loop incase somehow this spans multiple segments, but that would
-            // be a loco dataset
-            while (length < boundaries.get(boundaries.size() - 1))
-                boundaries.remove(boundaries.size() -1);
-
-            // add a sentinel value == length
-            List<Long> boundaries = new ArrayList<>(this.boundaries);
-            if (length != boundaries.get(boundaries.size() - 1))
-                boundaries.add(length);
-
+            long[] boundaries = this.boundaries.finish(length, isFinal);
 
-            int segcount = boundaries.size() - 1;
+            int segcount = boundaries.length - 1;
             Segment[] segments = new Segment[segcount];
 
             try
             {
                 for (int i = 0; i < segcount; i++)
                 {
-                    long start = boundaries.get(i);
-                    long size = boundaries.get(i + 1) - start;
+                    long start = boundaries[i];
+                    long size = boundaries[i + 1] - start;
                     MappedByteBuffer segment = size <= MAX_SEGMENT_SIZE
                                                ? raf.getChannel().map(FileChannel.MapMode.READ_ONLY, start, size)
                                                : null;
@@ -245,9 +409,10 @@ public class MmappedSegmentedFile extends SegmentedFile
         public void serializeBounds(DataOutput out) throws IOException
         {
             super.serializeBounds(out);
-            out.writeInt(boundaries.size());
-            for (long position: boundaries)
-                out.writeLong(position);
+            long[] boundaries = this.boundaries.truncate();
+            out.writeInt(boundaries.length);
+            for (long boundary : boundaries)
+                out.writeLong(boundary);
         }
 
         @Override
@@ -256,12 +421,11 @@ public class MmappedSegmentedFile extends SegmentedFile
             super.deserializeBounds(in);
 
             int size = in.readInt();
-            List<Long> temp = new ArrayList<>(size);
-            
+            long[] boundaries = new long[size];
             for (int i = 0; i < size; i++)
-                temp.add(in.readLong());
+                boundaries[i] = in.readLong();
 
-            boundaries = temp;
+            this.boundaries.init(boundaries, size);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/util/SegmentedFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/SegmentedFile.java b/src/java/org/apache/cassandra/io/util/SegmentedFile.java
index c65ecbf..23454bc 100644
--- a/src/java/org/apache/cassandra/io/util/SegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/SegmentedFile.java
@@ -23,6 +23,7 @@ import java.io.File;
 import java.io.IOException;
 import java.nio.MappedByteBuffer;
 import java.util.Iterator;
+import java.util.List;
 import java.util.NoSuchElementException;
 
 import com.google.common.util.concurrent.RateLimiter;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
----------------------------------------------------------------------
diff --git a/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java b/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
new file mode 100644
index 0000000..e17c6a7
--- /dev/null
+++ b/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
@@ -0,0 +1,322 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied.  See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+package org.apache.cassandra.io.sstable;
+
+import java.io.*;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Random;
+
+import com.google.common.io.Files;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.MmappedSegmentedFile;
+import org.apache.cassandra.io.util.MmappedSegmentedFile.Builder.Boundaries;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class LongSegmentedFileBoundaryTest
+{
+    @BeforeClass
+    public static void setup() throws Exception
+    {
+        SchemaLoader.cleanupAndLeaveDirs();
+        Keyspace.setInitialized();
+        StorageService.instance.initServer();
+    }
+
+    @AfterClass
+    public static void tearDown()
+    {
+        Config.setClientMode(false);
+    }
+
+    @Test
+    public void testRandomBoundaries()
+    {
+        long[] candidates = new long[1 + (1 << 16)];
+        int[] indexesToCheck = new int[1 << 8];
+        Random random = new Random();
+
+        for (int run = 0; run < 100; run++)
+        {
+
+            long seed = random.nextLong();
+            random.setSeed(seed);
+            System.out.println("Seed: " + seed);
+
+            // at least 1Ki, and as many as 256Ki, boundaries
+            int candidateCount = (1 + random.nextInt(candidates.length >> 10)) << 10;
+            generateBoundaries(random, candidateCount, candidates, indexesToCheck);
+
+            Boundaries builder = new Boundaries();
+            int nextIndexToCheck = indexesToCheck[0];
+            int checkCount = 0;
+            System.out.printf("[0..%d)", candidateCount);
+            for (int i = 1; i < candidateCount - 1; i++)
+            {
+                if (i == nextIndexToCheck)
+                {
+                    if (checkCount % 20 == 0)
+                        System.out.printf(" %d", i);
+                    // grow number of samples logarithmically; work will still increase superlinearly, as size of dataset grows linearly
+                    int sampleCount = 1 << (31 - Integer.numberOfLeadingZeros(++checkCount));
+                    checkBoundarySample(random, candidates, i, sampleCount, builder);
+                    // select out next index to check (there may be dups, so skip them)
+                    while ((nextIndexToCheck = checkCount == indexesToCheck.length ? candidateCount : indexesToCheck[checkCount]) == i)
+                        checkCount++;
+                }
+
+                builder.addCandidate(candidates[i]);
+            }
+            System.out.println();
+            checkBoundaries(candidates, candidateCount - 1, builder, candidates[candidateCount - 1]);
+            Assert.assertEquals(candidateCount, nextIndexToCheck);
+        }
+    }
+
+    private static void generateBoundaries(Random random, int candidateCount, long[] candidates, int[] indexesToCheck)
+    {
+        // average averageBoundarySize is 4MiB, max 4GiB, min 4KiB
+        long averageBoundarySize = (4L << 10) * random.nextInt(1 << 20);
+        long prev = 0;
+        for (int i = 1 ; i < candidateCount ; i++)
+            candidates[i] = prev += Math.max(1, averageBoundarySize + (random.nextGaussian() * averageBoundarySize));
+
+        // generate indexes we will corroborate our behaviour on
+        for (int i = 0 ; i < indexesToCheck.length ; i++)
+            indexesToCheck[i] = 1 + random.nextInt(candidateCount - 2);
+        Arrays.sort(indexesToCheck);
+    }
+
+    private static void checkBoundarySample(Random random, long[] candidates, int candidateCount, int sampleCount, Boundaries builder)
+    {
+        for (int i = 0 ; i < sampleCount ; i++)
+        {
+            // pick a number exponentially less likely to be near the beginning, since we test that area earlier
+            int position = 0 ;
+            while (position <= 0)
+                position = candidateCount / (Integer.lowestOneBit(random.nextInt()));
+            long upperBound = candidates[position];
+            long lowerBound = random.nextBoolean() ? (rand(random, 0, upperBound) / (Integer.lowestOneBit(random.nextInt())))
+                                                   : candidates[Math.max(0, position - random.nextInt(64))];
+            long length = rand(random, lowerBound, upperBound);
+            checkBoundaries(candidates, candidateCount, builder, length);
+        }
+        checkBoundaries(candidates, candidateCount, builder, candidates[candidateCount]);
+    }
+
+    private static long rand(Random random, long lowerBound, long upperBound)
+    {
+        if (upperBound == lowerBound)
+            return upperBound;
+        return lowerBound + ((random.nextLong() & Long.MAX_VALUE) % (upperBound - lowerBound));
+    }
+
+    private static void checkBoundaries(long[] candidates, int candidateCount, Boundaries builder, long length)
+    {
+        if (length == 0)
+            return;
+
+        long[] boundaries = new long[(int) (10 + 2 * (length / Integer.MAX_VALUE))];
+        int count = 1;
+        int prev = 0;
+        while (true)
+        {
+            int p = candidates[prev + 1] - boundaries[count - 1] >= Integer.MAX_VALUE
+                    ? prev + 1
+                    : Arrays.binarySearch(candidates, prev, candidateCount, boundaries[count - 1] + Integer.MAX_VALUE);
+            if (p < 0) p = -2 -p;
+            if (p >= candidateCount - 1 || candidates[p] >= length)
+                break;
+            boundaries[count++] = candidates[p];
+            if (candidates[p + 1] >= length)
+                break;
+            prev = p;
+        }
+        if (candidates[candidateCount - 1] < length && length - boundaries[count - 1] >= Integer.MAX_VALUE)
+            boundaries[count++] = candidates[candidateCount - 1];
+        boundaries[count++] = length;
+        final long[] canon = Arrays.copyOf(boundaries, count);
+        final long[] check = builder.finish(length, false);
+        if (!Arrays.equals(canon, check))
+            Assert.assertTrue("\n" + Arrays.toString(canon) + "\n" + Arrays.toString(check), Arrays.equals(canon, check));
+    }
+
+    @Test
+    public void testBoundariesAndRepairSmall() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1, 1 << 16);
+    }
+
+    @Test
+    public void testBoundariesAndRepairMedium() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1, 1 << 20);
+    }
+
+    @Test
+    public void testBoundariesAndRepairLarge() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1, 100 << 20);
+    }
+
+    @Test
+    public void testBoundariesAndRepairHuge() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1, Integer.MAX_VALUE - 1024);
+    }
+
+    @Test
+    public void testBoundariesAndRepairTooHuge() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1, Integer.MAX_VALUE);
+    }
+
+    @Test
+    public void testBoundariesAndRepairHugeIndex() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1 << 7, 1 << 15);
+    }
+
+    @Test
+    public void testBoundariesAndRepairReallyHugeIndex() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1 << 14, 1 << 15);
+    }
+
+    private void testBoundariesAndRepair(int rows, int rowSize) throws InvalidRequestException, IOException
+    {
+        String KS = "cql_keyspace";
+        String TABLE = "table1";
+
+        File tempdir = Files.createTempDir();
+        try
+        {
+            Assert.assertTrue(DatabaseDescriptor.getColumnIndexSize() < rowSize);
+            Assert.assertTrue(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap);
+            Assert.assertTrue(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap);
+            Assert.assertTrue(StorageService.getPartitioner() instanceof ByteOrderedPartitioner);
+            File dataDir = new File(tempdir.getAbsolutePath() + File.separator + KS + File.separator + TABLE);
+            Assert.assertTrue(dataDir.mkdirs());
+
+            String schema = "CREATE TABLE cql_keyspace.table" + (rows > 1 ? "2" : "1") + " (k bigint, v1 blob, v2 blob, v3 blob, v4 blob, v5 blob, PRIMARY KEY (k" + (rows > 1 ? ", v1" : "") + ")) WITH compression = { 'sstable_compression':'' };";
+            String insert = "INSERT INTO cql_keyspace.table" + (rows > 1 ? "2" : "1") + " (k, v1, v2, v3, v4, v5) VALUES (?, ?, ?, ?, ?, ?)";
+
+            CQLSSTableWriter.Builder builder = CQLSSTableWriter.builder()
+                                                      .inDirectory(dataDir)
+                                                      .forTable(schema)
+                                                      .withPartitioner(StorageService.getPartitioner())
+                                                      .using(insert)
+                                                      .sorted();
+            CQLSSTableWriter writer = builder.build();
+
+            // write 8Gb of decorated keys
+            ByteBuffer[] value = new ByteBuffer[rows];
+            for (int row = 0 ; row < rows ; row++)
+            {
+                // if we're using clustering columns, the clustering key is replicated across every other column
+                value[row] = ByteBuffer.allocate(rowSize / (rows > 1 ? 8 : 5));
+                value[row].putInt(0, row);
+            }
+            long targetSize = 8L << 30;
+            long dk = 0;
+            long size = 0;
+            long dkSize = rowSize * rows;
+            while (size < targetSize)
+            {
+                for (int row = 0 ; row < rows ; row++)
+                    writer.addRow(dk, value[row], value[row], value[row], value[row], value[row]);
+                size += dkSize;
+                dk++;
+            }
+
+            Descriptor descriptor = writer.getCurrentDescriptor().asType(Descriptor.Type.FINAL);
+            writer.close();
+
+            // open (and close) the reader so that the summary file is created
+            SSTableReader reader = SSTableReader.open(descriptor);
+            reader.selfRef().release();
+
+            // then check the boundaries are reasonable, and corrupt them
+            checkThenCorruptBoundaries(descriptor, rows * rowSize < Integer.MAX_VALUE);
+
+            // then check that reopening corrects the corruption
+            reader = SSTableReader.open(descriptor);
+            reader.selfRef().release();
+            checkThenCorruptBoundaries(descriptor, rows * rowSize < Integer.MAX_VALUE);
+        }
+        finally
+        {
+            FileUtils.deleteRecursive(tempdir);
+        }
+    }
+
+    private static void checkThenCorruptBoundaries(Descriptor descriptor, boolean expectDataMmappable) throws IOException
+    {
+        File summaryFile = new File(descriptor.filenameFor(Component.SUMMARY));
+        DataInputStream iStream = new DataInputStream(new FileInputStream(summaryFile));
+        IndexSummary indexSummary = IndexSummary.serializer.deserialize(iStream, StorageService.getPartitioner(), true, CFMetaData.DEFAULT_MIN_INDEX_INTERVAL, CFMetaData.DEFAULT_MAX_INDEX_INTERVAL);
+        ByteBuffer first = ByteBufferUtil.readWithLength(iStream);
+        ByteBuffer last = ByteBufferUtil.readWithLength(iStream);
+        MmappedSegmentedFile.Builder ibuilder = new MmappedSegmentedFile.Builder();
+        MmappedSegmentedFile.Builder dbuilder = new MmappedSegmentedFile.Builder();
+        ibuilder.deserializeBounds(iStream);
+        dbuilder.deserializeBounds(iStream);
+        iStream.close();
+        // index file cannot generally be non-mmappable, as index entries cannot be larger than MAX_SEGMENT_SIZE (due to promotedSize being encoded as an int)
+        assertBoundaries(descriptor.filenameFor(Component.PRIMARY_INDEX), true, ibuilder.boundaries());
+        assertBoundaries(descriptor.filenameFor(Component.DATA), expectDataMmappable, dbuilder.boundaries());
+
+        DataOutputStreamPlus oStream = new DataOutputStreamPlus(new FileOutputStream(summaryFile));
+        IndexSummary.serializer.serialize(indexSummary, oStream, true);
+        ByteBufferUtil.writeWithLength(first, oStream);
+        ByteBufferUtil.writeWithLength(last, oStream);
+        oStream.writeInt(1);
+        oStream.writeLong(new File(descriptor.filenameFor(Component.PRIMARY_INDEX)).length());
+        oStream.writeLong(new File(descriptor.filenameFor(Component.DATA)).length());
+        oStream.close();
+    }
+
+    private static void assertBoundaries(String path, boolean expectMmappable, long[] boundaries)
+    {
+        long length = new File(path).length();
+        long prev = boundaries[0];
+        for (int i = 1 ; i <= boundaries.length && prev < length ; i++)
+        {
+            long boundary = i == boundaries.length ? length : boundaries[i];
+            Assert.assertEquals(String.format("[%d, %d), %d of %d", boundary, prev, i, boundaries.length),
+                                expectMmappable, boundary - prev <= Integer.MAX_VALUE);
+            prev = boundary;
+        }
+    }
+
+}


[12/16] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index 87891ae,0000000..2b65792
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@@ -1,2287 -1,0 +1,2304 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.io.sstable.format;
 +
 +import java.io.*;
 +import java.nio.ByteBuffer;
 +import java.util.*;
 +import java.util.concurrent.*;
 +import java.util.concurrent.atomic.AtomicBoolean;
 +import java.util.concurrent.atomic.AtomicLong;
 +
 +import com.google.common.annotations.VisibleForTesting;
 +import com.google.common.base.Predicate;
 +import com.google.common.collect.Iterables;
 +import com.google.common.collect.Iterators;
 +import com.google.common.collect.Ordering;
 +import com.google.common.primitives.Longs;
 +import com.google.common.util.concurrent.RateLimiter;
 +
 +import com.clearspring.analytics.stream.cardinality.CardinalityMergeException;
 +import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus;
 +import com.clearspring.analytics.stream.cardinality.ICardinality;
 +import com.codahale.metrics.Counter;
 +import org.apache.cassandra.cache.CachingOptions;
 +import org.apache.cassandra.cache.InstrumentingCache;
 +import org.apache.cassandra.cache.KeyCacheKey;
 +import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
 +import org.apache.cassandra.concurrent.ScheduledExecutors;
 +import org.apache.cassandra.config.*;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 +import org.apache.cassandra.db.commitlog.ReplayPosition;
 +import org.apache.cassandra.db.composites.CellName;
 +import org.apache.cassandra.db.filter.ColumnSlice;
 +import org.apache.cassandra.db.index.SecondaryIndex;
 +import org.apache.cassandra.db.lifecycle.Tracker;
 +import org.apache.cassandra.dht.*;
 +import org.apache.cassandra.io.FSError;
++import org.apache.cassandra.io.FSReadError;
 +import org.apache.cassandra.io.compress.CompressionMetadata;
 +import org.apache.cassandra.io.sstable.*;
 +import org.apache.cassandra.io.sstable.metadata.*;
 +import org.apache.cassandra.io.util.*;
 +import org.apache.cassandra.metrics.RestorableMeter;
 +import org.apache.cassandra.metrics.StorageMetrics;
 +import org.apache.cassandra.service.ActiveRepairService;
 +import org.apache.cassandra.service.CacheService;
 +import org.apache.cassandra.service.StorageService;
 +import org.apache.cassandra.utils.*;
 +import org.apache.cassandra.utils.concurrent.OpOrder;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +import org.apache.cassandra.utils.concurrent.Ref;
 +import org.apache.cassandra.utils.concurrent.SelfRefCounted;
 +
 +import static org.apache.cassandra.db.Directories.SECONDARY_INDEX_NAME_SEPARATOR;
 +
 +/**
 + * An SSTableReader can be constructed in a number of places, but typically is either
 + * read from disk at startup, or constructed from a flushed memtable, or after compaction
 + * to replace some existing sstables. However once created, an sstablereader may also be modified.
 + *
 + * A reader's OpenReason describes its current stage in its lifecycle, as follows:
 + *
 + *
 + * <pre> {@code
 + * NORMAL
 + * From:       None        => Reader has been read from disk, either at startup or from a flushed memtable
 + *             EARLY       => Reader is the final result of a compaction
 + *             MOVED_START => Reader WAS being compacted, but this failed and it has been restored to NORMAL status
 + *
 + * EARLY
 + * From:       None        => Reader is a compaction replacement that is either incomplete and has been opened
 + *                            to represent its partial result status, or has been finished but the compaction
 + *                            it is a part of has not yet completed fully
 + *             EARLY       => Same as from None, only it is not the first time it has been
 + *
 + * MOVED_START
 + * From:       NORMAL      => Reader is being compacted. This compaction has not finished, but the compaction result
 + *                            is either partially or fully opened, to either partially or fully replace this reader.
 + *                            This reader's start key has been updated to represent this, so that reads only hit
 + *                            one or the other reader.
 + *
 + * METADATA_CHANGE
 + * From:       NORMAL      => Reader has seen low traffic and the amount of memory available for index summaries is
 + *                            constrained, so its index summary has been downsampled.
 + *         METADATA_CHANGE => Same
 + * } </pre>
 + *
 + * Note that in parallel to this, there are two different Descriptor types; TMPLINK and FINAL; the latter corresponds
 + * to NORMAL state readers and all readers that replace a NORMAL one. TMPLINK is used for EARLY state readers and
 + * no others.
 + *
 + * When a reader is being compacted, if the result is large its replacement may be opened as EARLY before compaction
 + * completes in order to present the result to consumers earlier. In this case the reader will itself be changed to
 + * a MOVED_START state, where its start no longer represents its on-disk minimum key. This is to permit reads to be
 + * directed to only one reader when the two represent the same data. The EARLY file can represent a compaction result
 + * that is either partially complete and still in-progress, or a complete and immutable sstable that is part of a larger
 + * macro compaction action that has not yet fully completed.
 + *
 + * Currently ALL compaction results at least briefly go through an EARLY open state prior to completion, regardless
 + * of if early opening is enabled.
 + *
 + * Since a reader can be created multiple times over the same shared underlying resources, and the exact resources
 + * it shares between each instance differ subtly, we track the lifetime of any underlying resource with its own
 + * reference count, which each instance takes a Ref to. Each instance then tracks references to itself, and once these
 + * all expire it releases its Refs to these underlying resources.
 + *
 + * There is some shared cleanup behaviour needed only once all sstablereaders in a certain stage of their lifecycle
 + * (i.e. EARLY or NORMAL opening), and some that must only occur once all readers of any kind over a single logical
 + * sstable have expired. These are managed by the TypeTidy and GlobalTidy classes at the bottom, and are effectively
 + * managed as another resource each instance tracks its own Ref instance to, to ensure all of these resources are
 + * cleaned up safely and can be debugged otherwise.
 + *
 + * TODO: fill in details about Tracker and lifecycle interactions for tools, and for compaction strategies
 + */
 +public abstract class SSTableReader extends SSTable implements SelfRefCounted<SSTableReader>
 +{
 +    private static final Logger logger = LoggerFactory.getLogger(SSTableReader.class);
++    private static final int ACCURATE_BOUNDARIES_MAGIC_NUMBER = 248923458;
 +
 +    private static final ScheduledThreadPoolExecutor syncExecutor = new ScheduledThreadPoolExecutor(1);
 +    static
 +    {
 +        // Immediately remove readMeter sync task when cancelled.
 +        syncExecutor.setRemoveOnCancelPolicy(true);
 +    }
 +    private static final RateLimiter meterSyncThrottle = RateLimiter.create(100.0);
 +
 +    public static final Comparator<SSTableReader> maxTimestampComparator = new Comparator<SSTableReader>()
 +    {
 +        public int compare(SSTableReader o1, SSTableReader o2)
 +        {
 +            long ts1 = o1.getMaxTimestamp();
 +            long ts2 = o2.getMaxTimestamp();
 +            return (ts1 > ts2 ? -1 : (ts1 == ts2 ? 0 : 1));
 +        }
 +    };
 +
 +    // it's just an object, which we use regular Object equality on; we introduce a special class just for easy recognition
 +    public static final class UniqueIdentifier {}
 +
 +    public static final Comparator<SSTableReader> sstableComparator = new Comparator<SSTableReader>()
 +    {
 +        public int compare(SSTableReader o1, SSTableReader o2)
 +        {
 +            return o1.first.compareTo(o2.first);
 +        }
 +    };
 +
 +    public static final Ordering<SSTableReader> sstableOrdering = Ordering.from(sstableComparator);
 +
 +    /**
 +     * maxDataAge is a timestamp in local server time (e.g. System.currentTimeMilli) which represents an upper bound
 +     * to the newest piece of data stored in the sstable. In other words, this sstable does not contain items created
 +     * later than maxDataAge.
 +     *
 +     * The field is not serialized to disk, so relying on it for more than what truncate does is not advised.
 +     *
 +     * When a new sstable is flushed, maxDataAge is set to the time of creation.
 +     * When a sstable is created from compaction, maxDataAge is set to max of all merged sstables.
 +     *
 +     * The age is in milliseconds since epoc and is local to this host.
 +     */
 +    public final long maxDataAge;
 +
 +    public enum OpenReason
 +    {
 +        NORMAL,
 +        EARLY,
 +        METADATA_CHANGE,
 +        MOVED_START
 +    }
 +
 +    public final OpenReason openReason;
 +    public final UniqueIdentifier instanceId = new UniqueIdentifier();
 +
 +    // indexfile and datafile: might be null before a call to load()
 +    protected SegmentedFile ifile;
 +    protected SegmentedFile dfile;
 +    protected IndexSummary indexSummary;
 +    protected IFilter bf;
 +
 +    protected final RowIndexEntry.IndexSerializer rowIndexEntrySerializer;
 +
 +    protected InstrumentingCache<KeyCacheKey, RowIndexEntry> keyCache;
 +
 +    protected final BloomFilterTracker bloomFilterTracker = new BloomFilterTracker();
 +
 +    // technically isCompacted is not necessary since it should never be unreferenced unless it is also compacted,
 +    // but it seems like a good extra layer of protection against reference counting bugs to not delete data based on that alone
 +    protected final AtomicBoolean isSuspect = new AtomicBoolean(false);
 +
 +    // not final since we need to be able to change level on a file.
 +    protected volatile StatsMetadata sstableMetadata;
 +
 +    protected final AtomicLong keyCacheHit = new AtomicLong(0);
 +    protected final AtomicLong keyCacheRequest = new AtomicLong(0);
 +
 +    private final InstanceTidier tidy = new InstanceTidier(descriptor, metadata);
 +    private final Ref<SSTableReader> selfRef = new Ref<>(this, tidy);
 +
 +    private RestorableMeter readMeter;
 +
 +    /**
 +     * Calculate approximate key count.
 +     * If cardinality estimator is available on all given sstables, then this method use them to estimate
 +     * key count.
 +     * If not, then this uses index summaries.
 +     *
 +     * @param sstables SSTables to calculate key count
 +     * @return estimated key count
 +     */
 +    public static long getApproximateKeyCount(Collection<SSTableReader> sstables)
 +    {
 +        long count = -1;
 +
 +        // check if cardinality estimator is available for all SSTables
 +        boolean cardinalityAvailable = !sstables.isEmpty() && Iterators.all(sstables.iterator(), new Predicate<SSTableReader>()
 +        {
 +            public boolean apply(SSTableReader sstable)
 +            {
 +                return sstable.descriptor.version.hasNewStatsFile();
 +            }
 +        });
 +
 +        // if it is, load them to estimate key count
 +        if (cardinalityAvailable)
 +        {
 +            boolean failed = false;
 +            ICardinality cardinality = null;
 +            for (SSTableReader sstable : sstables)
 +            {
 +                if (sstable.openReason == OpenReason.EARLY)
 +                    continue;
 +
 +                try
 +                {
 +                    CompactionMetadata metadata = (CompactionMetadata) sstable.descriptor.getMetadataSerializer().deserialize(sstable.descriptor, MetadataType.COMPACTION);
 +                    assert metadata != null : sstable.getFilename();
 +                    if (cardinality == null)
 +                        cardinality = metadata.cardinalityEstimator;
 +                    else
 +                        cardinality = cardinality.merge(metadata.cardinalityEstimator);
 +                }
 +                catch (IOException e)
 +                {
 +                    logger.warn("Reading cardinality from Statistics.db failed.", e);
 +                    failed = true;
 +                    break;
 +                }
 +                catch (CardinalityMergeException e)
 +                {
 +                    logger.warn("Cardinality merge failed.", e);
 +                    failed = true;
 +                    break;
 +                }
 +            }
 +            if (cardinality != null && !failed)
 +                count = cardinality.cardinality();
 +        }
 +
 +        // if something went wrong above or cardinality is not available, calculate using index summary
 +        if (count < 0)
 +        {
 +            for (SSTableReader sstable : sstables)
 +                count += sstable.estimatedKeys();
 +        }
 +        return count;
 +    }
 +
 +    /**
 +     * Estimates how much of the keys we would keep if the sstables were compacted together
 +     */
 +    public static double estimateCompactionGain(Set<SSTableReader> overlapping)
 +    {
 +        Set<ICardinality> cardinalities = new HashSet<>(overlapping.size());
 +        for (SSTableReader sstable : overlapping)
 +        {
 +            try
 +            {
 +                ICardinality cardinality = ((CompactionMetadata) sstable.descriptor.getMetadataSerializer().deserialize(sstable.descriptor, MetadataType.COMPACTION)).cardinalityEstimator;
 +                if (cardinality != null)
 +                    cardinalities.add(cardinality);
 +                else
 +                    logger.trace("Got a null cardinality estimator in: {}", sstable.getFilename());
 +            }
 +            catch (IOException e)
 +            {
 +                logger.warn("Could not read up compaction metadata for {}", sstable, e);
 +            }
 +        }
 +        long totalKeyCountBefore = 0;
 +        for (ICardinality cardinality : cardinalities)
 +        {
 +            totalKeyCountBefore += cardinality.cardinality();
 +        }
 +        if (totalKeyCountBefore == 0)
 +            return 1;
 +
 +        long totalKeyCountAfter = mergeCardinalities(cardinalities).cardinality();
 +        logger.trace("Estimated compaction gain: {}/{}={}", totalKeyCountAfter, totalKeyCountBefore, ((double)totalKeyCountAfter)/totalKeyCountBefore);
 +        return ((double)totalKeyCountAfter)/totalKeyCountBefore;
 +    }
 +
 +    private static ICardinality mergeCardinalities(Collection<ICardinality> cardinalities)
 +    {
 +        ICardinality base = new HyperLogLogPlus(13, 25); // see MetadataCollector.cardinality
 +        try
 +        {
 +            base = base.merge(cardinalities.toArray(new ICardinality[cardinalities.size()]));
 +        }
 +        catch (CardinalityMergeException e)
 +        {
 +            logger.warn("Could not merge cardinalities", e);
 +        }
 +        return base;
 +    }
 +
 +    public static SSTableReader open(Descriptor descriptor) throws IOException
 +    {
 +        CFMetaData metadata;
 +        if (descriptor.cfname.contains(SECONDARY_INDEX_NAME_SEPARATOR))
 +        {
 +            int i = descriptor.cfname.indexOf(SECONDARY_INDEX_NAME_SEPARATOR);
 +            String parentName = descriptor.cfname.substring(0, i);
 +            CFMetaData parent = Schema.instance.getCFMetaData(descriptor.ksname, parentName);
 +            ColumnDefinition def = parent.getColumnDefinitionForIndex(descriptor.cfname.substring(i + 1));
 +            metadata = CFMetaData.newIndexMetadata(parent, def, SecondaryIndex.getIndexComparator(parent, def));
 +        }
 +        else
 +        {
 +            metadata = Schema.instance.getCFMetaData(descriptor.ksname, descriptor.cfname);
 +        }
 +        return open(descriptor, metadata);
 +    }
 +
 +    public static SSTableReader open(Descriptor desc, CFMetaData metadata) throws IOException
 +    {
 +        IPartitioner p = desc.cfname.contains(SECONDARY_INDEX_NAME_SEPARATOR)
 +                ? new LocalPartitioner(metadata.getKeyValidator())
 +                : StorageService.getPartitioner();
 +        return open(desc, componentsFor(desc), metadata, p);
 +    }
 +
 +    public static SSTableReader open(Descriptor descriptor, Set<Component> components, CFMetaData metadata, IPartitioner partitioner) throws IOException
 +    {
 +        return open(descriptor, components, metadata, partitioner, true, true);
 +    }
 +
 +    // use only for offline or "Standalone" operations
 +    public static SSTableReader openNoValidation(Descriptor descriptor, Set<Component> components, ColumnFamilyStore cfs) throws IOException
 +    {
 +        return open(descriptor, components, cfs.metadata, cfs.partitioner, false, false); // do not track hotness
 +    }
 +
 +    /**
 +     * Open SSTable reader to be used in batch mode(such as sstableloader).
 +     *
 +     * @param descriptor
 +     * @param components
 +     * @param metadata
 +     * @param partitioner
 +     * @return opened SSTableReader
 +     * @throws IOException
 +     */
 +    public static SSTableReader openForBatch(Descriptor descriptor, Set<Component> components, CFMetaData metadata, IPartitioner partitioner) throws IOException
 +    {
 +        // Minimum components without which we can't do anything
 +        assert components.contains(Component.DATA) : "Data component is missing for sstable " + descriptor;
 +        assert components.contains(Component.PRIMARY_INDEX) : "Primary index component is missing for sstable " + descriptor;
 +
 +        Map<MetadataType, MetadataComponent> sstableMetadata = descriptor.getMetadataSerializer().deserialize(descriptor,
 +                EnumSet.of(MetadataType.VALIDATION, MetadataType.STATS));
 +        ValidationMetadata validationMetadata = (ValidationMetadata) sstableMetadata.get(MetadataType.VALIDATION);
 +        StatsMetadata statsMetadata = (StatsMetadata) sstableMetadata.get(MetadataType.STATS);
 +
 +        // Check if sstable is created using same partitioner.
 +        // Partitioner can be null, which indicates older version of sstable or no stats available.
 +        // In that case, we skip the check.
 +        String partitionerName = partitioner.getClass().getCanonicalName();
 +        if (validationMetadata != null && !partitionerName.equals(validationMetadata.partitioner))
 +        {
 +            logger.error(String.format("Cannot open %s; partitioner %s does not match system partitioner %s.  Note that the default partitioner starting with Cassandra 1.2 is Murmur3Partitioner, so you will need to edit that to match your old partitioner if upgrading.",
 +                    descriptor, validationMetadata.partitioner, partitionerName));
 +            System.exit(1);
 +        }
 +
 +        logger.debug("Opening {} ({} bytes)", descriptor, new File(descriptor.filenameFor(Component.DATA)).length());
 +        SSTableReader sstable = internalOpen(descriptor, components, metadata, partitioner, System.currentTimeMillis(),
 +                statsMetadata, OpenReason.NORMAL);
 +
 +        // special implementation of load to use non-pooled SegmentedFile builders
 +        try(SegmentedFile.Builder ibuilder = new BufferedSegmentedFile.Builder();
 +            SegmentedFile.Builder dbuilder = sstable.compression
 +                ? new CompressedSegmentedFile.Builder(null)
 +                : new BufferedSegmentedFile.Builder())
 +        {
 +            if (!sstable.loadSummary(ibuilder, dbuilder))
 +                sstable.buildSummary(false, ibuilder, dbuilder, false, Downsampling.BASE_SAMPLING_LEVEL);
 +            sstable.ifile = ibuilder.complete(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX));
 +            sstable.dfile = dbuilder.complete(sstable.descriptor.filenameFor(Component.DATA));
 +            sstable.bf = FilterFactory.AlwaysPresent;
 +            sstable.setup(false);
 +            return sstable;
 +        }
 +    }
 +
 +    public static SSTableReader open(Descriptor descriptor,
 +                                      Set<Component> components,
 +                                      CFMetaData metadata,
 +                                      IPartitioner partitioner,
 +                                      boolean validate,
 +                                      boolean trackHotness) throws IOException
 +    {
 +        // Minimum components without which we can't do anything
 +        assert components.contains(Component.DATA) : "Data component is missing for sstable " + descriptor;
 +        assert !validate || components.contains(Component.PRIMARY_INDEX) : "Primary index component is missing for sstable " + descriptor;
 +
 +        Map<MetadataType, MetadataComponent> sstableMetadata = descriptor.getMetadataSerializer().deserialize(descriptor,
 +                                                                                                               EnumSet.of(MetadataType.VALIDATION, MetadataType.STATS));
 +        ValidationMetadata validationMetadata = (ValidationMetadata) sstableMetadata.get(MetadataType.VALIDATION);
 +        StatsMetadata statsMetadata = (StatsMetadata) sstableMetadata.get(MetadataType.STATS);
 +
 +        // Check if sstable is created using same partitioner.
 +        // Partitioner can be null, which indicates older version of sstable or no stats available.
 +        // In that case, we skip the check.
 +        String partitionerName = partitioner.getClass().getCanonicalName();
 +        if (validationMetadata != null && !partitionerName.equals(validationMetadata.partitioner))
 +        {
 +            logger.error(String.format("Cannot open %s; partitioner %s does not match system partitioner %s.  Note that the default partitioner starting with Cassandra 1.2 is Murmur3Partitioner, so you will need to edit that to match your old partitioner if upgrading.",
 +                    descriptor, validationMetadata.partitioner, partitionerName));
 +            System.exit(1);
 +        }
 +
 +        logger.debug("Opening {} ({} bytes)", descriptor, new File(descriptor.filenameFor(Component.DATA)).length());
 +        SSTableReader sstable = internalOpen(descriptor, components, metadata, partitioner, System.currentTimeMillis(),
 +                                             statsMetadata, OpenReason.NORMAL);
 +        try
 +        {
 +            // load index and filter
 +            long start = System.nanoTime();
 +            sstable.load(validationMetadata);
 +            logger.trace("INDEX LOAD TIME for {}: {} ms.", descriptor, TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
 +
 +            sstable.setup(trackHotness);
 +            if (validate)
 +                sstable.validate();
 +
 +            if (sstable.getKeyCache() != null)
 +                logger.trace("key cache contains {}/{} keys", sstable.getKeyCache().size(), sstable.getKeyCache().getCapacity());
 +
 +            return sstable;
 +        }
 +        catch (Throwable t)
 +        {
 +            sstable.selfRef().release();
 +            throw t;
 +        }
 +    }
 +
 +    public static void logOpenException(Descriptor descriptor, IOException e)
 +    {
 +        if (e instanceof FileNotFoundException)
 +            logger.error("Missing sstable component in {}; skipped because of {}", descriptor, e.getMessage());
 +        else
 +            logger.error("Corrupt sstable {}; skipped", descriptor, e);
 +    }
 +
 +    public static Collection<SSTableReader> openAll(Set<Map.Entry<Descriptor, Set<Component>>> entries,
 +                                                    final CFMetaData metadata,
 +                                                    final IPartitioner partitioner)
 +    {
 +        final Collection<SSTableReader> sstables = new LinkedBlockingQueue<>();
 +
 +        ExecutorService executor = DebuggableThreadPoolExecutor.createWithFixedPoolSize("SSTableBatchOpen", FBUtilities.getAvailableProcessors());
 +        for (final Map.Entry<Descriptor, Set<Component>> entry : entries)
 +        {
 +            Runnable runnable = new Runnable()
 +            {
 +                public void run()
 +                {
 +                    SSTableReader sstable;
 +                    try
 +                    {
 +                        sstable = open(entry.getKey(), entry.getValue(), metadata, partitioner);
 +                    }
 +                    catch (CorruptSSTableException ex)
 +                    {
 +                        FileUtils.handleCorruptSSTable(ex);
 +                        logger.error("Corrupt sstable {}; skipping table", entry, ex);
 +                        return;
 +                    }
 +                    catch (FSError ex)
 +                    {
 +                        FileUtils.handleFSError(ex);
 +                        logger.error("Cannot read sstable {}; file system error, skipping table", entry, ex);
 +                        return;
 +                    }
 +                    catch (IOException ex)
 +                    {
 +                        logger.error("Cannot read sstable {}; other IO error, skipping table", entry, ex);
 +                        return;
 +                    }
 +                    sstables.add(sstable);
 +                }
 +            };
 +            executor.submit(runnable);
 +        }
 +
 +        executor.shutdown();
 +        try
 +        {
 +            executor.awaitTermination(7, TimeUnit.DAYS);
 +        }
 +        catch (InterruptedException e)
 +        {
 +            throw new AssertionError(e);
 +        }
 +
 +        return sstables;
 +
 +    }
 +
 +    /**
 +     * Open a RowIndexedReader which already has its state initialized (by SSTableWriter).
 +     */
 +    public static SSTableReader internalOpen(Descriptor desc,
 +                                      Set<Component> components,
 +                                      CFMetaData metadata,
 +                                      IPartitioner partitioner,
 +                                      SegmentedFile ifile,
 +                                      SegmentedFile dfile,
 +                                      IndexSummary isummary,
 +                                      IFilter bf,
 +                                      long maxDataAge,
 +                                      StatsMetadata sstableMetadata,
 +                                      OpenReason openReason)
 +    {
 +        assert desc != null && partitioner != null && ifile != null && dfile != null && isummary != null && bf != null && sstableMetadata != null;
 +
 +        SSTableReader reader = internalOpen(desc, components, metadata, partitioner, maxDataAge, sstableMetadata, openReason);
 +
 +        reader.bf = bf;
 +        reader.ifile = ifile;
 +        reader.dfile = dfile;
 +        reader.indexSummary = isummary;
 +        reader.setup(true);
 +
 +        return reader;
 +    }
 +
 +
 +    private static SSTableReader internalOpen(final Descriptor descriptor,
 +                                            Set<Component> components,
 +                                            CFMetaData metadata,
 +                                            IPartitioner partitioner,
 +                                            Long maxDataAge,
 +                                            StatsMetadata sstableMetadata,
 +                                            OpenReason openReason)
 +    {
 +        Factory readerFactory = descriptor.getFormat().getReaderFactory();
 +
 +        return readerFactory.open(descriptor, components, metadata, partitioner, maxDataAge, sstableMetadata, openReason);
 +    }
 +
 +    protected SSTableReader(final Descriptor desc,
 +                            Set<Component> components,
 +                            CFMetaData metadata,
 +                            IPartitioner partitioner,
 +                            long maxDataAge,
 +                            StatsMetadata sstableMetadata,
 +                            OpenReason openReason)
 +    {
 +        super(desc, components, metadata, partitioner);
 +        this.sstableMetadata = sstableMetadata;
 +        this.maxDataAge = maxDataAge;
 +        this.openReason = openReason;
 +        this.rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata);
 +    }
 +
 +    public static long getTotalBytes(Iterable<SSTableReader> sstables)
 +    {
 +        long sum = 0;
 +        for (SSTableReader sstable : sstables)
 +            sum += sstable.onDiskLength();
 +        return sum;
 +    }
 +
 +    public static long getTotalUncompressedBytes(Iterable<SSTableReader> sstables)
 +    {
 +        long sum = 0;
 +        for (SSTableReader sstable : sstables)
 +            sum += sstable.uncompressedLength();
 +
 +        return sum;
 +    }
 +
 +    public boolean equals(Object that)
 +    {
 +        return that instanceof SSTableReader && ((SSTableReader) that).descriptor.equals(this.descriptor);
 +    }
 +
 +    public int hashCode()
 +    {
 +        return this.descriptor.hashCode();
 +    }
 +
 +    public String getFilename()
 +    {
 +        return dfile.path();
 +    }
 +
 +    public void setupKeyCache()
 +    {
 +        // under normal operation we can do this at any time, but SSTR is also used outside C* proper,
 +        // e.g. by BulkLoader, which does not initialize the cache.  As a kludge, we set up the cache
 +        // here when we know we're being wired into the rest of the server infrastructure.
 +        keyCache = CacheService.instance.keyCache;
 +    }
 +
 +    private void load(ValidationMetadata validation) throws IOException
 +    {
 +        if (metadata.getBloomFilterFpChance() == 1.0)
 +        {
 +            // bf is disabled.
 +            load(false, true);
 +            bf = FilterFactory.AlwaysPresent;
 +        }
 +        else if (!components.contains(Component.PRIMARY_INDEX))
 +        {
 +            // avoid any reading of the missing primary index component.
 +            // this should only happen during StandaloneScrubber
 +            load(false, false);
 +        }
 +        else if (!components.contains(Component.FILTER) || validation == null)
 +        {
 +            // bf is enabled, but filter component is missing.
 +            load(true, true);
 +        }
 +        else if (validation.bloomFilterFPChance != metadata.getBloomFilterFpChance())
 +        {
 +            // bf fp chance in sstable metadata and it has changed since compaction.
 +            load(true, true);
 +        }
 +        else
 +        {
 +            // bf is enabled and fp chance matches the currently configured value.
 +            load(false, true);
 +            loadBloomFilter();
 +        }
 +    }
 +
 +    /**
 +     * Load bloom filter from Filter.db file.
 +     *
 +     * @throws IOException
 +     */
 +    private void loadBloomFilter() throws IOException
 +    {
 +        try (DataInputStream stream = new DataInputStream(new BufferedInputStream(new FileInputStream(descriptor.filenameFor(Component.FILTER)))))
 +        {
 +            bf = FilterFactory.deserialize(stream, true);
 +        }
 +    }
 +
 +    /**
 +     * Loads ifile, dfile and indexSummary, and optionally recreates the bloom filter.
 +     * @param saveSummaryIfCreated for bulk loading purposes, if the summary was absent and needed to be built, you can
 +     *                             avoid persisting it to disk by setting this to false
 +     */
 +    private void load(boolean recreateBloomFilter, boolean saveSummaryIfCreated) throws IOException
 +    {
 +        try(SegmentedFile.Builder ibuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode(), false);
 +            SegmentedFile.Builder dbuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode(), compression))
 +        {
 +            boolean summaryLoaded = loadSummary(ibuilder, dbuilder);
 +            boolean builtSummary = false;
 +            if (recreateBloomFilter || !summaryLoaded)
 +            {
 +                buildSummary(recreateBloomFilter, ibuilder, dbuilder, summaryLoaded, Downsampling.BASE_SAMPLING_LEVEL);
 +                builtSummary = true;
 +            }
 +
 +            if (components.contains(Component.PRIMARY_INDEX))
 +                ifile = ibuilder.complete(descriptor.filenameFor(Component.PRIMARY_INDEX));
 +
 +            dfile = dbuilder.complete(descriptor.filenameFor(Component.DATA));
 +
 +            // Check for an index summary that was downsampled even though the serialization format doesn't support
 +            // that.  If it was downsampled, rebuild it.  See CASSANDRA-8993 for details.
 +        if (!descriptor.version.hasSamplingLevel() && !builtSummary && !validateSummarySamplingLevel() && ifile != null)
 +            {
 +                indexSummary.close();
 +                ifile.close();
 +                dfile.close();
 +
 +                logger.info("Detected erroneously downsampled index summary; will rebuild summary at full sampling");
 +                FileUtils.deleteWithConfirm(new File(descriptor.filenameFor(Component.SUMMARY)));
 +
 +                try(SegmentedFile.Builder ibuilderRebuild = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode(), false);
 +                    SegmentedFile.Builder dbuilderRebuild = SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode(), compression))
 +                {
 +                    buildSummary(false, ibuilderRebuild, dbuilderRebuild, false, Downsampling.BASE_SAMPLING_LEVEL);
 +                    ifile = ibuilderRebuild.complete(descriptor.filenameFor(Component.PRIMARY_INDEX));
 +                    dfile = dbuilderRebuild.complete(descriptor.filenameFor(Component.DATA));
 +                    saveSummary(ibuilderRebuild, dbuilderRebuild);
 +                }
 +            }
 +            else if (saveSummaryIfCreated && builtSummary)
 +            {
 +                saveSummary(ibuilder, dbuilder);
 +            }
 +        }
 +        catch (Throwable t)
 +        { // Because the tidier has not been set-up yet in SSTableReader.open(), we must release the files in case of error
 +            if (ifile != null)
 +            {
 +                ifile.close();
 +                ifile = null;
 +            }
 +
 +            if (dfile != null)
 +            {
 +                dfile.close();
 +                dfile = null;
 +            }
 +
 +            if (indexSummary != null)
 +            {
 +                indexSummary.close();
 +                indexSummary = null;
 +            }
 +
 +            throw t;
 +        }
 +    }
 +
 +    /**
 +     * Build index summary(and optionally bloom filter) by reading through Index.db file.
 +     *
 +     * @param recreateBloomFilter true if recreate bloom filter
 +     * @param ibuilder
 +     * @param dbuilder
 +     * @param summaryLoaded true if index summary is already loaded and not need to build again
 +     * @throws IOException
 +     */
 +    private void buildSummary(boolean recreateBloomFilter, SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, boolean summaryLoaded, int samplingLevel) throws IOException
 +    {
 +         if (!components.contains(Component.PRIMARY_INDEX))
 +             return;
 +
 +        // we read the positions in a BRAF so we don't have to worry about an entry spanning a mmap boundary.
 +        try (RandomAccessReader primaryIndex = RandomAccessReader.open(new File(descriptor.filenameFor(Component.PRIMARY_INDEX))))
 +        {
 +            long indexSize = primaryIndex.length();
 +            long histogramCount = sstableMetadata.estimatedRowSize.count();
 +            long estimatedKeys = histogramCount > 0 && !sstableMetadata.estimatedRowSize.isOverflowed()
 +                    ? histogramCount
 +                    : estimateRowsFromIndex(primaryIndex); // statistics is supposed to be optional
 +
 +            if (recreateBloomFilter)
 +                bf = FilterFactory.getFilter(estimatedKeys, metadata.getBloomFilterFpChance(), true);
 +
 +            try (IndexSummaryBuilder summaryBuilder = summaryLoaded ? null : new IndexSummaryBuilder(estimatedKeys, metadata.getMinIndexInterval(), samplingLevel))
 +            {
 +                long indexPosition;
 +                RowIndexEntry.IndexSerializer rowIndexSerializer = descriptor.getFormat().getIndexSerializer(metadata);
 +
 +                while ((indexPosition = primaryIndex.getFilePointer()) != indexSize)
 +                {
 +                    ByteBuffer key = ByteBufferUtil.readWithShortLength(primaryIndex);
 +                    RowIndexEntry indexEntry = rowIndexSerializer.deserialize(primaryIndex, descriptor.version);
 +                    DecoratedKey decoratedKey = partitioner.decorateKey(key);
 +                    if (first == null)
 +                        first = decoratedKey;
 +                    last = decoratedKey;
 +
 +                    if (recreateBloomFilter)
 +                        bf.add(decoratedKey);
 +
 +                    // if summary was already read from disk we don't want to re-populate it using primary index
 +                    if (!summaryLoaded)
 +                    {
 +                        summaryBuilder.maybeAddEntry(decoratedKey, indexPosition);
 +                        ibuilder.addPotentialBoundary(indexPosition);
 +                        dbuilder.addPotentialBoundary(indexEntry.position);
 +                    }
 +                }
 +
 +                if (!summaryLoaded)
 +                    indexSummary = summaryBuilder.build(partitioner);
 +            }
 +        }
 +
 +        first = getMinimalKey(first);
 +        last = getMinimalKey(last);
 +    }
 +
 +    /**
 +     * Load index summary from Summary.db file if it exists.
 +     *
 +     * if loaded index summary has different index interval from current value stored in schema,
 +     * then Summary.db file will be deleted and this returns false to rebuild summary.
 +     *
 +     * @param ibuilder
 +     * @param dbuilder
 +     * @return true if index summary is loaded successfully from Summary.db file.
 +     */
 +    @SuppressWarnings("resource")
 +    public boolean loadSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
 +    {
 +        File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
 +        if (!summariesFile.exists())
 +            return false;
 +
 +        DataInputStream iStream = null;
 +        try
 +        {
 +            iStream = new DataInputStream(new FileInputStream(summariesFile));
 +            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);
 +            dbuilder.deserializeBounds(iStream);
++
++            boolean checkForRepair = true;
++            try
++            {
++                int v = iStream.readInt();
++                // check for our magic number, indicating this summary has been sampled correctly
++                checkForRepair = v != ACCURATE_BOUNDARIES_MAGIC_NUMBER;
++            }
++            catch (Throwable t) {}
++
++            // fix CASSANDRA-10357 on-the-fly
++            if (checkForRepair && MmappedSegmentedFile.maybeRepair(metadata, descriptor, indexSummary, ibuilder, dbuilder))
++                saveSummary(ibuilder, dbuilder);
 +        }
 +        catch (IOException e)
 +        {
 +            if (indexSummary != null)
 +                indexSummary.close();
 +            logger.trace("Cannot deserialize SSTable Summary File {}: {}", summariesFile.getPath(), e.getMessage());
 +            // corrupted; delete it and fall back to creating a new summary
 +            FileUtils.closeQuietly(iStream);
 +            // delete it and fall back to creating a new summary
 +            FileUtils.deleteWithConfirm(summariesFile);
 +            return false;
 +        }
 +        finally
 +        {
 +            FileUtils.closeQuietly(iStream);
 +        }
 +
 +        return true;
 +    }
 +
 +    /**
 +     * Validates that an index summary has full sampling, as expected when the serialization format does not support
 +     * persisting the sampling level.
 +     * @return true if the summary has full sampling, false otherwise
 +     */
 +    private boolean validateSummarySamplingLevel()
 +    {
 +        // We need to check index summary entries against the index to verify that none of them were dropped due to
 +        // downsampling.  Downsampling can drop any of the first BASE_SAMPLING_LEVEL entries (repeating that drop pattern
 +        // for the remainder of the summary).  Unfortunately, the first entry to be dropped is the entry at
 +        // index (BASE_SAMPLING_LEVEL - 1), so we need to check a full set of BASE_SAMPLING_LEVEL entries.
 +        if (ifile == null)
 +            return false;
 +
 +        Iterator<FileDataInput> segments = ifile.iterator(0);
 +        int i = 0;
 +        int summaryEntriesChecked = 0;
 +        int expectedIndexInterval = getMinIndexInterval();
 +        while (segments.hasNext())
 +        {
 +            String path = null;
 +            try (FileDataInput in = segments.next())
 +            {
 +                path = in.getPath();
 +                while (!in.isEOF())
 +                {
 +                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
 +                    if (i % expectedIndexInterval == 0)
 +                    {
 +                        ByteBuffer summaryKey = ByteBuffer.wrap(indexSummary.getKey(i / expectedIndexInterval));
 +                        if (!summaryKey.equals(indexKey))
 +                            return false;
 +                        summaryEntriesChecked++;
 +
 +                        if (summaryEntriesChecked == Downsampling.BASE_SAMPLING_LEVEL)
 +                            return true;
 +                    }
 +                    RowIndexEntry.Serializer.skip(in);
 +                    i++;
 +                }
 +            }
 +            catch (IOException e)
 +            {
 +                markSuspect();
 +                throw new CorruptSSTableException(e, path);
 +            }
 +        }
 +
 +        return true;
 +    }
 +
 +    /**
 +     * Save index summary to Summary.db file.
 +     *
 +     * @param ibuilder
 +     * @param dbuilder
 +     */
 +
 +    public void saveSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
 +    {
 +        saveSummary(this.descriptor, this.first, this.last, ibuilder, dbuilder, indexSummary);
 +    }
 +
 +    private void saveSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, IndexSummary newSummary)
 +    {
 +        saveSummary(this.descriptor, this.first, this.last, ibuilder, dbuilder, newSummary);
 +    }
 +    /**
 +     * Save index summary to Summary.db file.
 +     */
 +    public static void saveSummary(Descriptor descriptor, DecoratedKey first, DecoratedKey last,
 +                                   SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, IndexSummary summary)
 +    {
 +        File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
 +        if (summariesFile.exists())
 +            FileUtils.deleteWithConfirm(summariesFile);
 +
 +        try (DataOutputStreamPlus oStream = new BufferedDataOutputStreamPlus(new FileOutputStream(summariesFile));)
 +        {
 +            IndexSummary.serializer.serialize(summary, oStream, descriptor.version.hasSamplingLevel());
 +            ByteBufferUtil.writeWithLength(first.getKey(), oStream);
 +            ByteBufferUtil.writeWithLength(last.getKey(), oStream);
 +            ibuilder.serializeBounds(oStream);
 +            dbuilder.serializeBounds(oStream);
++            // write a magic number, to indicate this summary has been sampled correctly
++            oStream.writeInt(ACCURATE_BOUNDARIES_MAGIC_NUMBER);
 +        }
 +        catch (IOException e)
 +        {
 +            logger.trace("Cannot save SSTable Summary: ", e);
 +
 +            // corrupted hence delete it and let it load it now.
 +            if (summariesFile.exists())
 +                FileUtils.deleteWithConfirm(summariesFile);
 +        }
 +    }
 +
 +    public void setReplaced()
 +    {
 +        synchronized (tidy.global)
 +        {
 +            assert !tidy.isReplaced;
 +            tidy.isReplaced = true;
 +        }
 +    }
 +
 +    public boolean isReplaced()
 +    {
 +        synchronized (tidy.global)
 +        {
 +            return tidy.isReplaced;
 +        }
 +    }
 +
 +    // runOnClose must NOT be an anonymous or non-static inner class, nor must it retain a reference chain to this reader
 +    public void runOnClose(final Runnable runOnClose)
 +    {
 +        synchronized (tidy.global)
 +        {
 +            final Runnable existing = tidy.runOnClose;
 +            tidy.runOnClose = AndThen.get(existing, runOnClose);
 +        }
 +    }
 +
 +    private static class AndThen implements Runnable
 +    {
 +        final Runnable runFirst;
 +        final Runnable runSecond;
 +
 +        private AndThen(Runnable runFirst, Runnable runSecond)
 +        {
 +            this.runFirst = runFirst;
 +            this.runSecond = runSecond;
 +        }
 +
 +        public void run()
 +        {
 +            runFirst.run();
 +            runSecond.run();
 +        }
 +
 +        static Runnable get(Runnable runFirst, Runnable runSecond)
 +        {
 +            if (runFirst == null)
 +                return runSecond;
 +            return new AndThen(runFirst, runSecond);
 +        }
 +    }
 +
 +    /**
 +     * Clone this reader with the provided start and open reason, and set the clone as replacement.
 +     *
 +     * @param newFirst the first key for the replacement (which can be different from the original due to the pre-emptive
 +     * opening of compaction results).
 +     * @param reason the {@code OpenReason} for the replacement.
 +     *
 +     * @return the cloned reader. That reader is set as a replacement by the method.
 +     */
 +    private SSTableReader cloneAndReplace(DecoratedKey newFirst, OpenReason reason)
 +    {
 +        return cloneAndReplace(newFirst, reason, indexSummary.sharedCopy());
 +    }
 +
 +    /**
 +     * Clone this reader with the new values and set the clone as replacement.
 +     *
 +     * @param newFirst the first key for the replacement (which can be different from the original due to the pre-emptive
 +     * opening of compaction results).
 +     * @param reason the {@code OpenReason} for the replacement.
 +     * @param newSummary the index summary for the replacement.
 +     *
 +     * @return the cloned reader. That reader is set as a replacement by the method.
 +     */
 +    private SSTableReader cloneAndReplace(DecoratedKey newFirst, OpenReason reason, IndexSummary newSummary)
 +    {
 +        SSTableReader replacement = internalOpen(descriptor,
 +                                                 components,
 +                                                 metadata,
 +                                                 partitioner,
 +                                                 ifile != null ? ifile.sharedCopy() : null,
 +                                                 dfile.sharedCopy(),
 +                                                 newSummary,
 +                                                 bf.sharedCopy(),
 +                                                 maxDataAge,
 +                                                 sstableMetadata,
 +                                                 reason);
 +        replacement.first = newFirst;
 +        replacement.last = last;
 +        replacement.isSuspect.set(isSuspect.get());
 +        return replacement;
 +    }
 +
 +    // runOnClose must NOT be an anonymous or non-static inner class, nor must it retain a reference chain to this reader
 +    public SSTableReader cloneWithNewStart(DecoratedKey newStart, final Runnable runOnClose)
 +    {
 +        synchronized (tidy.global)
 +        {
 +            assert openReason != OpenReason.EARLY;
 +            // TODO: merge with caller's firstKeyBeyond() work,to save time
 +            if (newStart.compareTo(first) > 0)
 +            {
 +                final long dataStart = getPosition(newStart, Operator.EQ).position;
 +                final long indexStart = getIndexScanPosition(newStart);
 +                this.tidy.runOnClose = new DropPageCache(dfile, dataStart, ifile, indexStart, runOnClose);
 +            }
 +
 +            return cloneAndReplace(newStart, OpenReason.MOVED_START);
 +        }
 +    }
 +
 +    private static class DropPageCache implements Runnable
 +    {
 +        final SegmentedFile dfile;
 +        final long dfilePosition;
 +        final SegmentedFile ifile;
 +        final long ifilePosition;
 +        final Runnable andThen;
 +
 +        private DropPageCache(SegmentedFile dfile, long dfilePosition, SegmentedFile ifile, long ifilePosition, Runnable andThen)
 +        {
 +            this.dfile = dfile;
 +            this.dfilePosition = dfilePosition;
 +            this.ifile = ifile;
 +            this.ifilePosition = ifilePosition;
 +            this.andThen = andThen;
 +        }
 +
 +        public void run()
 +        {
 +            dfile.dropPageCache(dfilePosition);
 +
 +            if (ifile != null)
 +                ifile.dropPageCache(ifilePosition);
 +            andThen.run();
 +        }
 +    }
 +
 +    /**
 +     * Returns a new SSTableReader with the same properties as this SSTableReader except that a new IndexSummary will
 +     * be built at the target samplingLevel.  This (original) SSTableReader instance will be marked as replaced, have
 +     * its DeletingTask removed, and have its periodic read-meter sync task cancelled.
 +     * @param samplingLevel the desired sampling level for the index summary on the new SSTableReader
 +     * @return a new SSTableReader
 +     * @throws IOException
 +     */
 +    @SuppressWarnings("resource")
 +    public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
 +    {
 +        assert descriptor.version.hasSamplingLevel();
 +
 +        synchronized (tidy.global)
 +        {
 +            assert openReason != OpenReason.EARLY;
 +
 +            int minIndexInterval = metadata.getMinIndexInterval();
 +            int maxIndexInterval = metadata.getMaxIndexInterval();
 +            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
 +
 +            IndexSummary newSummary;
 +            long oldSize = bytesOnDisk();
 +
 +            // 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 = 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);
 +
 +                try(SegmentedFile.Builder ibuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode(), false);
 +                    SegmentedFile.Builder dbuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode(), compression))
 +                {
 +                    saveSummary(ibuilder, dbuilder, newSummary);
 +                }
 +            }
 +            else
 +            {
 +                throw new AssertionError("Attempted to clone SSTableReader with the same index summary sampling level and " +
 +                        "no adjustments to min/max_index_interval");
 +            }
 +
 +            long newSize = bytesOnDisk();
 +            StorageMetrics.load.inc(newSize - oldSize);
 +            parent.metric.liveDiskSpaceUsed.inc(newSize - oldSize);
 +
 +            return cloneAndReplace(first, OpenReason.METADATA_CHANGE, newSummary);
 +        }
 +    }
 +
 +    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();
 +            try (IndexSummaryBuilder summaryBuilder = new IndexSummaryBuilder(estimatedKeys(), metadata.getMinIndexInterval(), newSamplingLevel))
 +            {
 +                long indexPosition;
 +                while ((indexPosition = primaryIndex.getFilePointer()) != indexSize)
 +                {
 +                    summaryBuilder.maybeAddEntry(partitioner.decorateKey(ByteBufferUtil.readWithShortLength(primaryIndex)), indexPosition);
 +                    RowIndexEntry.Serializer.skip(primaryIndex);
 +                }
 +
 +                return summaryBuilder.build(partitioner);
 +            }
 +        }
 +        finally
 +        {
 +            FileUtils.closeQuietly(primaryIndex);
 +        }
 +    }
 +
 +    public RestorableMeter getReadMeter()
 +    {
 +        return readMeter;
 +    }
 +
 +    public int getIndexSummarySamplingLevel()
 +    {
 +        return indexSummary.getSamplingLevel();
 +    }
 +
 +    public long getIndexSummaryOffHeapSize()
 +    {
 +        return indexSummary.getOffHeapSize();
 +    }
 +
 +    public int getMinIndexInterval()
 +    {
 +        return indexSummary.getMinIndexInterval();
 +    }
 +
 +    public double getEffectiveIndexInterval()
 +    {
 +        return indexSummary.getEffectiveIndexInterval();
 +    }
 +
 +    public void releaseSummary()
 +    {
 +        tidy.releaseSummary();
 +        indexSummary = null;
 +    }
 +
 +    private void validate()
 +    {
 +        if (this.first.compareTo(this.last) > 0)
 +        {
 +            selfRef().release();
 +            throw new IllegalStateException(String.format("SSTable first key %s > last key %s", this.first, this.last));
 +        }
 +    }
 +
 +    /**
 +     * Gets the position in the index file to start scanning to find the given key (at most indexInterval keys away,
 +     * modulo downsampling of the index summary). Always returns a value >= 0
 +     */
 +    public long getIndexScanPosition(RowPosition key)
 +    {
 +        if (openReason == OpenReason.MOVED_START && key.compareTo(first) < 0)
 +            key = first;
 +
 +        return getIndexScanPositionFromBinarySearchResult(indexSummary.binarySearch(key), indexSummary);
 +    }
 +
 +    @VisibleForTesting
 +    public static long getIndexScanPositionFromBinarySearchResult(int binarySearchResult, IndexSummary referencedIndexSummary)
 +    {
 +        if (binarySearchResult == -1)
 +            return 0;
 +        else
 +            return referencedIndexSummary.getPosition(getIndexSummaryIndexFromBinarySearchResult(binarySearchResult));
 +    }
 +
 +    public static int getIndexSummaryIndexFromBinarySearchResult(int binarySearchResult)
 +    {
 +        if (binarySearchResult < 0)
 +        {
 +            // binary search gives us the first index _greater_ than the key searched for,
 +            // i.e., its insertion position
 +            int greaterThan = (binarySearchResult + 1) * -1;
 +            if (greaterThan == 0)
 +                return -1;
 +            return greaterThan - 1;
 +        }
 +        else
 +        {
 +            return binarySearchResult;
 +        }
 +    }
 +
 +    /**
 +     * Returns the compression metadata for this sstable.
 +     * @throws IllegalStateException if the sstable is not compressed
 +     */
 +    public CompressionMetadata getCompressionMetadata()
 +    {
 +        if (!compression)
 +            throw new IllegalStateException(this + " is not compressed");
 +
 +        CompressionMetadata cmd = ((ICompressedFile) dfile).getMetadata();
 +
 +        //We need the parent cf metadata
 +        String cfName = metadata.isSecondaryIndex() ? metadata.getParentColumnFamilyName() : metadata.cfName;
 +        cmd.parameters.setLiveMetadata(Schema.instance.getCFMetaData(metadata.ksName, cfName));
 +
 +        return cmd;
 +    }
 +
 +    /**
 +     * Returns the amount of memory in bytes used off heap by the compression meta-data.
 +     * @return the amount of memory in bytes used off heap by the compression meta-data
 +     */
 +    public long getCompressionMetadataOffHeapSize()
 +    {
 +        if (!compression)
 +            return 0;
 +
 +        return getCompressionMetadata().offHeapSize();
 +    }
 +
 +    /**
 +     * For testing purposes only.
 +     */
 +    public void forceFilterFailures()
 +    {
 +        bf = FilterFactory.AlwaysPresent;
 +    }
 +
 +    public IFilter getBloomFilter()
 +    {
 +        return bf;
 +    }
 +
 +    public long getBloomFilterSerializedSize()
 +    {
 +        return bf.serializedSize();
 +    }
 +
 +    /**
 +     * Returns the amount of memory in bytes used off heap by the bloom filter.
 +     * @return the amount of memory in bytes used off heap by the bloom filter
 +     */
 +    public long getBloomFilterOffHeapSize()
 +    {
 +        return bf.offHeapSize();
 +    }
 +
 +    /**
 +     * @return An estimate of the number of keys in this SSTable based on the index summary.
 +     */
 +    public long estimatedKeys()
 +    {
 +        return indexSummary.getEstimatedKeyCount();
 +    }
 +
 +    /**
 +     * @param ranges
 +     * @return An estimate of the number of keys for given ranges in this SSTable.
 +     */
 +    public long estimatedKeysForRanges(Collection<Range<Token>> ranges)
 +    {
 +        long sampleKeyCount = 0;
 +        List<Pair<Integer, Integer>> sampleIndexes = getSampleIndexesForRanges(indexSummary, ranges);
 +        for (Pair<Integer, Integer> sampleIndexRange : sampleIndexes)
 +            sampleKeyCount += (sampleIndexRange.right - sampleIndexRange.left + 1);
 +
 +        // adjust for the current sampling level: (BSL / SL) * index_interval_at_full_sampling
 +        long estimatedKeys = sampleKeyCount * ((long) Downsampling.BASE_SAMPLING_LEVEL * indexSummary.getMinIndexInterval()) / indexSummary.getSamplingLevel();
 +        return Math.max(1, estimatedKeys);
 +    }
 +
 +    /**
 +     * Returns the number of entries in the IndexSummary.  At full sampling, this is approximately 1/INDEX_INTERVALth of
 +     * the keys in this SSTable.
 +     */
 +    public int getIndexSummarySize()
 +    {
 +        return indexSummary.size();
 +    }
 +
 +    /**
 +     * Returns the approximate number of entries the IndexSummary would contain if it were at full sampling.
 +     */
 +    public int getMaxIndexSummarySize()
 +    {
 +        return indexSummary.getMaxNumberOfEntries();
 +    }
 +
 +    /**
 +     * Returns the key for the index summary entry at `index`.
 +     */
 +    public byte[] getIndexSummaryKey(int index)
 +    {
 +        return indexSummary.getKey(index);
 +    }
 +
 +    private static List<Pair<Integer,Integer>> getSampleIndexesForRanges(IndexSummary summary, Collection<Range<Token>> ranges)
 +    {
 +        // use the index to determine a minimal section for each range
 +        List<Pair<Integer,Integer>> positions = new ArrayList<>();
 +
 +        for (Range<Token> range : Range.normalize(ranges))
 +        {
 +            RowPosition leftPosition = range.left.maxKeyBound();
 +            RowPosition rightPosition = range.right.maxKeyBound();
 +
 +            int left = summary.binarySearch(leftPosition);
 +            if (left < 0)
 +                left = (left + 1) * -1;
 +            else
 +                // left range are start exclusive
 +                left = left + 1;
 +            if (left == summary.size())
 +                // left is past the end of the sampling
 +                continue;
 +
 +            int right = Range.isWrapAround(range.left, range.right)
 +                    ? summary.size() - 1
 +                    : summary.binarySearch(rightPosition);
 +            if (right < 0)
 +            {
 +                // range are end inclusive so we use the previous index from what binarySearch give us
 +                // since that will be the last index we will return
 +                right = (right + 1) * -1;
 +                if (right == 0)
 +                    // Means the first key is already stricly greater that the right bound
 +                    continue;
 +                right--;
 +            }
 +
 +            if (left > right)
 +                // empty range
 +                continue;
 +            positions.add(Pair.create(left, right));
 +        }
 +        return positions;
 +    }
 +
 +    public Iterable<DecoratedKey> getKeySamples(final Range<Token> range)
 +    {
 +        final List<Pair<Integer, Integer>> indexRanges = getSampleIndexesForRanges(indexSummary, Collections.singletonList(range));
 +
 +        if (indexRanges.isEmpty())
 +            return Collections.emptyList();
 +
 +        return new Iterable<DecoratedKey>()
 +        {
 +            public Iterator<DecoratedKey> iterator()
 +            {
 +                return new Iterator<DecoratedKey>()
 +                {
 +                    private Iterator<Pair<Integer, Integer>> rangeIter = indexRanges.iterator();
 +                    private Pair<Integer, Integer> current;
 +                    private int idx;
 +
 +                    public boolean hasNext()
 +                    {
 +                        if (current == null || idx > current.right)
 +                        {
 +                            if (rangeIter.hasNext())
 +                            {
 +                                current = rangeIter.next();
 +                                idx = current.left;
 +                                return true;
 +                            }
 +                            return false;
 +                        }
 +
 +                        return true;
 +                    }
 +
 +                    public DecoratedKey next()
 +                    {
 +                        byte[] bytes = indexSummary.getKey(idx++);
 +                        return partitioner.decorateKey(ByteBuffer.wrap(bytes));
 +                    }
 +
 +                    public void remove()
 +                    {
 +                        throw new UnsupportedOperationException();
 +                    }
 +                };
 +            }
 +        };
 +    }
 +
 +    /**
 +     * Determine the minimal set of sections that can be extracted from this SSTable to cover the given ranges.
 +     * @return A sorted list of (offset,end) pairs that cover the given ranges in the datafile for this SSTable.
 +     */
 +    public List<Pair<Long,Long>> getPositionsForRanges(Collection<Range<Token>> ranges)
 +    {
 +        // use the index to determine a minimal section for each range
 +        List<Pair<Long,Long>> positions = new ArrayList<>();
 +        for (Range<Token> range : Range.normalize(ranges))
 +        {
 +            assert !range.isWrapAround() || range.right.isMinimum();
 +            // truncate the range so it at most covers the sstable
 +            AbstractBounds<RowPosition> bounds = Range.makeRowRange(range);
 +            RowPosition leftBound = bounds.left.compareTo(first) > 0 ? bounds.left : first.getToken().minKeyBound();
 +            RowPosition rightBound = bounds.right.isMinimum() ? last.getToken().maxKeyBound() : bounds.right;
 +
 +            if (leftBound.compareTo(last) > 0 || rightBound.compareTo(first) < 0)
 +                continue;
 +
 +            long left = getPosition(leftBound, Operator.GT).position;
 +            long right = (rightBound.compareTo(last) > 0)
 +                         ? uncompressedLength()
 +                         : getPosition(rightBound, Operator.GT).position;
 +
 +            if (left == right)
 +                // empty range
 +                continue;
 +
 +            assert left < right : String.format("Range=%s openReason=%s first=%s last=%s left=%d right=%d", range, openReason, first, last, left, right);
 +            positions.add(Pair.create(left, right));
 +        }
 +        return positions;
 +    }
 +
 +    public KeyCacheKey getCacheKey(DecoratedKey key)
 +    {
 +        return new KeyCacheKey(metadata.ksAndCFName, descriptor, key.getKey());
 +    }
 +
 +    public void cacheKey(DecoratedKey key, RowIndexEntry info)
 +    {
 +        CachingOptions caching = metadata.getCaching();
 +
 +        if (!caching.keyCache.isEnabled()
 +                || keyCache == null
 +                || keyCache.getCapacity() == 0)
 +        {
 +            return;
 +        }
 +
 +        KeyCacheKey cacheKey = new KeyCacheKey(metadata.ksAndCFName, descriptor, key.getKey());
 +        logger.trace("Adding cache entry for {} -> {}", cacheKey, info);
 +        keyCache.put(cacheKey, info);
 +    }
 +
 +    public RowIndexEntry getCachedPosition(DecoratedKey key, boolean updateStats)
 +    {
 +        return getCachedPosition(new KeyCacheKey(metadata.ksAndCFName, descriptor, key.getKey()), updateStats);
 +    }
 +
 +    protected RowIndexEntry getCachedPosition(KeyCacheKey unifiedKey, boolean updateStats)
 +    {
 +        if (keyCache != null && keyCache.getCapacity() > 0 && metadata.getCaching().keyCache.isEnabled()) {
 +            if (updateStats)
 +            {
 +                RowIndexEntry cachedEntry = keyCache.get(unifiedKey);
 +                keyCacheRequest.incrementAndGet();
 +                if (cachedEntry != null)
 +                {
 +                    keyCacheHit.incrementAndGet();
 +                    bloomFilterTracker.addTruePositive();
 +                }
 +                return cachedEntry;
 +            }
 +            else
 +            {
 +                return keyCache.getInternal(unifiedKey);
 +            }
 +        }
 +        return null;
 +    }
 +
 +    /**
 +     * Get position updating key cache and stats.
 +     * @see #getPosition(org.apache.cassandra.db.RowPosition, SSTableReader.Operator, boolean)
 +     */
 +    public RowIndexEntry getPosition(RowPosition key, Operator op)
 +    {
 +        return getPosition(key, op, true, false);
 +    }
 +
 +    public RowIndexEntry getPosition(RowPosition key, Operator op, boolean updateCacheAndStats)
 +    {
 +        return getPosition(key, op, updateCacheAndStats, false);
 +    }
 +    /**
 +     * @param key The key to apply as the rhs to the given Operator. A 'fake' key is allowed to
 +     * allow key selection by token bounds but only if op != * EQ
 +     * @param op The Operator defining matching keys: the nearest key to the target matching the operator wins.
 +     * @param updateCacheAndStats true if updating stats and cache
 +     * @return The index entry corresponding to the key, or null if the key is not present
 +     */
 +    protected abstract RowIndexEntry getPosition(RowPosition key, Operator op, boolean updateCacheAndStats, boolean permitMatchPastLast);
 +
 +    //Corresponds to a name column
 +    public abstract OnDiskAtomIterator iterator(DecoratedKey key, SortedSet<CellName> columns);
 +    public abstract OnDiskAtomIterator iterator(FileDataInput file, DecoratedKey key, SortedSet<CellName> columns, RowIndexEntry indexEntry);
 +
 +    //Corresponds to a slice query
 +    public abstract OnDiskAtomIterator iterator(DecoratedKey key, ColumnSlice[] slices, boolean reverse);
 +    public abstract OnDiskAtomIterator iterator(FileDataInput file, DecoratedKey key, ColumnSlice[] slices, boolean reversed, RowIndexEntry indexEntry);
 +
 +    /**
 +     * Finds and returns the first key beyond a given token in this SSTable or null if no such key exists.
 +     */
 +    public DecoratedKey firstKeyBeyond(RowPosition token)
 +    {
 +        if (token.compareTo(first) < 0)
 +            return first;
 +
 +        long sampledPosition = getIndexScanPosition(token);
 +
 +        if (ifile == null)
 +            return null;
 +
 +        Iterator<FileDataInput> segments = ifile.iterator(sampledPosition);
 +        while (segments.hasNext())
 +        {
 +            String path = null;
 +            try (FileDataInput in = segments.next();)
 +            {
 +                path = in.getPath();
 +                while (!in.isEOF())
 +                {
 +                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
 +                    DecoratedKey indexDecoratedKey = partitioner.decorateKey(indexKey);
 +                    if (indexDecoratedKey.compareTo(token) > 0)
 +                        return indexDecoratedKey;
 +
 +                    RowIndexEntry.Serializer.skip(in);
 +                }
 +            }
 +            catch (IOException e)
 +            {
 +                markSuspect();
 +                throw new CorruptSSTableException(e, path);
 +            }
 +        }
 +
 +        return null;
 +    }
 +
 +    /**
 +     * @return The length in bytes of the data for this SSTable. For
 +     * compressed files, this is not the same thing as the on disk size (see
 +     * onDiskLength())
 +     */
 +    public long uncompressedLength()
 +    {
 +        return dfile.length;
 +    }
 +
 +    /**
 +     * @return The length in bytes of the on disk size for this SSTable. For
 +     * compressed files, this is not the same thing as the data length (see
 +     * length())
 +     */
 +    public long onDiskLength()
 +    {
 +        return dfile.onDiskLength;
 +    }
 +
 +    /**
 +     * Mark the sstable as obsolete, i.e., compacted into newer sstables.
 +     *
 +     * When calling this function, the caller must ensure that the SSTableReader is not referenced anywhere
 +     * except for threads holding a reference.
 +     *
 +     * @return true if the this is the first time the file was marked obsolete.  Calling this
 +     * multiple times is usually buggy (see exceptions in Tracker.unmarkCompacting and removeOldSSTablesSize).
 +     */
 +    public boolean markObsolete(Tracker tracker)
 +    {
 +        if (logger.isTraceEnabled())
 +            logger.trace("Marking {} compacted", getFilename());
 +
 +        synchronized (tidy.global)
 +        {
 +            assert !tidy.isReplaced;
 +        }
 +        if (!tidy.global.isCompacted.getAndSet(true))
 +        {
 +            tidy.type.markObsolete(this, tracker);
 +            return true;
 +        }
 +        return false;
 +    }
 +
 +    public boolean isMarkedCompacted()
 +    {
 +        return tidy.global.isCompacted.get();
 +    }
 +
 +    public void markSuspect()
 +    {
 +        if (logger.isTraceEnabled())
 +            logger.trace("Marking {} as a suspect for blacklisting.", getFilename());
 +
 +        isSuspect.getAndSet(true);
 +    }
 +
 +    public boolean isMarkedSuspect()
 +    {
 +        return isSuspect.get();
 +    }
 +
 +
 +    /**
 +     * I/O SSTableScanner
 +     * @return A Scanner for seeking over the rows of the SSTable.
 +     */
 +    public ISSTableScanner getScanner()
 +    {
 +        return getScanner((RateLimiter) null);
 +    }
 +
 +    public ISSTableScanner getScanner(RateLimiter limiter)
 +    {
 +        return getScanner(DataRange.allData(partitioner), limiter);
 +    }
 +
 +    /**
 +     *
 +     * @param dataRange filter to use when reading the columns
 +     * @return A Scanner for seeking over the rows of the SSTable.
 +     */
 +    public ISSTableScanner getScanner(DataRange dataRange)
 +    {
 +        return getScanner(dataRange, null);
 +    }
 +
 +    /**
 +     * Direct I/O SSTableScanner over a defined range of tokens.
 +     *
 +     * @param range the range of keys to cover
 +     * @return A Scanner for seeking over the rows of the SSTable.
 +     */
 +    public ISSTableScanner getScanner(Range<Token> range, RateLimiter limiter)
 +    {
 +        if (range == null)
 +            return getScanner(limiter);
 +        return getScanner(Collections.singletonList(range), limiter);
 +    }
 +
 +    /**
 +     * Direct I/O SSTableScanner over a defined collection of ranges of tokens.
 +     *
 +     * @param ranges the range of keys to cover
 +     * @return A Scanner for seeking over the rows of the SSTable.
 +     */
 +    public abstract ISSTableScanner getScanner(Collection<Range<Token>> ranges, RateLimiter limiter);
 +
 +    /**
 +     *
 +     * @param dataRange filter to use when reading the columns
 +     * @return A Scanner for seeking over the rows of the SSTable.
 +     */
 +    public abstract ISSTableScanner getScanner(DataRange dataRange, RateLimiter limiter);
 +
 +
 +
 +    public FileDataInput getFileDataInput(long position)
 +    {
 +        return dfile.getSegment(position);
 +    }
 +
 +    /**
 +     * Tests if the sstable contains data newer than the given age param (in localhost currentMilli time).
 +     * This works in conjunction with maxDataAge which is an upper bound on the create of data in this sstable.
 +     * @param age The age to compare the maxDataAre of this sstable. Measured in millisec since epoc on this host
 +     * @return True iff this sstable contains data that's newer than the given age parameter.
 +     */
 +    public boolean newSince(long age)
 +    {
 +        return maxDataAge > age;
 +    }
 +
 +    public void createLinks(String snapshotDirectoryPath)
 +    {
 +        for (Component component : components)
 +        {
 +            File sourceFile = new File(descriptor.filenameFor(component));
 +            File targetLink = new File(snapshotDirectoryPath, sourceFile.getName());
 +            FileUtils.createHardLink(sourceFile, targetLink);
 +        }
 +    }
 +
 +    public boolean isRepaired()
 +    {
 +        return sstableMetadata.repairedAt != ActiveRepairService.UNREPAIRED_SSTABLE;
 +    }
 +
 +    /**
 +     * TODO: Move someplace reusable
 +     */
 +    public abstract static class Operator
 +    {
 +        public static final Operator EQ = new Equals();
 +        public static final Operator GE = new GreaterThanOrEqualTo();
 +        public static final Operator GT = new GreaterThan();
 +
 +        /**
 +         * @param comparison The result of a call to compare/compareTo, with the desired field on the rhs.
 +         * @return less than 0 if the operator cannot match forward, 0 if it matches, greater than 0 if it might match forward.
 +         */
 +        public abstract int apply(int comparison);
 +
 +        final static class Equals extends Operator
 +        {
 +            public int apply(int comparison) { return -comparison; }
 +        }
 +
 +        final static class GreaterThanOrEqualTo extends Operator
 +        {
 +            public int apply(int comparison) { return comparison >= 0 ? 0 : 1; }
 +        }
 +
 +        final static class GreaterThan extends Operator
 +        {
 +            public int apply(int comparison) { return comparison > 0 ? 0 : 1; }
 +        }
 +    }
 +
 +    public long getBloomFilterFalsePositiveCount()
 +    {
 +        return bloomFilterTracker.getFalsePositiveCount();
 +    }
 +
 +    public long getRecentBloomFilterFalsePositiveCount()
 +    {
 +        return bloomFilterTracker.getRecentFalsePositiveCount();
 +    }
 +
 +    public long getBloomFilterTruePositiveCount()
 +    {
 +        return bloomFilterTracker.getTruePositiveCount();
 +    }
 +
 +    public long getRecentBloomFilterTruePositiveCount()
 +    {
 +        return bloomFilterTracker.getRecentTruePositiveCount();
 +    }
 +
 +    public InstrumentingCache<KeyCacheKey, RowIndexEntry> getKeyCache()
 +    {
 +        return keyCache;
 +    }
 +
 +    public EstimatedHistogram getEstimatedRowSize()
 +    {
 +        return sstableMetadata.estimatedRowSize;
 +    }
 +
 +    public EstimatedHistogram getEstimatedColumnCount()
 +    {
 +        return sstableMetadata.estimatedColumnCount;
 +    }
 +
 +    public double getEstimatedDroppableTombstoneRatio(int gcBefore)
 +    {
 +        return sstableMetadata.getEstimatedDroppableTombstoneRatio(gcBefore);
 +    }
 +
 +    public double getDroppableTombstonesBefore(int gcBefore)
 +    {
 +        return sstableMetadata.getDroppableTombstonesBefore(gcBefore);
 +    }
 +
 +    public double getCompressionRatio()
 +    {
 +        return sstableMetadata.compressionRatio;
 +    }
 +
 +    public ReplayPosition getReplayPosition()
 +    {
 +        return sstableMetadata.replayPosition;
 +    }
 +
 +    public long getMinTimestamp()
 +    {
 +        return sstableMetadata.minTimestamp;
 +    }
 +
 +    public long getMaxTimestamp()
 +    {
 +        return sstableMetadata.maxTimestamp;
 +    }
 +
 +    public Set<Integer> getAncestors()
 +    {
 +        try
 +        {
 +            CompactionMetadata compactionMetadata = (CompactionMetadata) descriptor.getMetadataSerializer().deserialize(descriptor, MetadataType.COMPACTION);
 +            if (compactionMetadata != null)
 +                return compactionMetadata.ancestors;
 +            return Collections.emptySet();
 +        }
 +        catch (IOException e)
 +        {
 +            SSTableReader.logOpenException(descriptor, e);
 +            return Collections.emptySet();
 +        }
 +    }
 +
 +    public int getSSTableLevel()
 +    {
 +        return sstableMetadata.sstableLevel;
 +    }
 +
 +    /**
 +     * Reloads the sstable metadata from disk.
 +     *
 +     * Called after level is changed on sstable, for example if the sstable is dropped to L0
 +     *
 +     * Might be possible to remove in future versions
 +     *
 +     * @throws IOException
 +     */
 +    public void reloadSSTableMetadata() throws IOException
 +    {
 +        this.sstableMetadata = (StatsMetadata) descriptor.getMetadataSerializer().deserialize(descriptor, MetadataType.STATS);
 +    }
 +
 +    public StatsMetadata getSSTableMetadata()
 +    {
 +        return sstableMetadata;
 +    }
 +
 +    public RandomAccessReader openDataReader(RateLimiter limiter)
 +    {
 +        assert limiter != null;
 +        return dfile.createThrottledReader(limiter);
 +    }
 +
 +    public RandomAccessReader openDataReader()
 +    {
 +        return dfile.createReader();
 +    }
 +
 +    public RandomAccessReader openIndexReader()
 +    {
 +        if (ifile != null)
 +            return ifile.createReader();
 +        return null;
 +    }
 +
 +    /**
 +     * @param component component to get timestamp.
 +     * @return last modified time for given component. 0 if given component does not exist or IO error occurs.
 +     */
 +    public long getCreationTimeFor(Component component)
 +    {
 +        return new File(descriptor.filenameFor(component)).lastModified();
 +    }
 +
 +    /**
 +     * @return Number of key cache hit
 +     */
 +    public long getKeyCacheHit()
 +    {
 +        return keyCacheHit.get();
 +    }
 +
 +    /**
 +     * @return Number of key cache request
 +     */
 +    public long getKeyCacheRequest()
 +    {
 +        return keyCacheRequest.get();
 +    }
 +
 +    /**
 +     * Increment the total row read count and read rate for this SSTable.  This should not be incremented for range
 +     * slice queries, row cache hits, or non-query reads, like compaction.
 +     */
 +    public void incrementReadCount()
 +    {
 +        if (readMeter != null)
 +            readMeter.mark();
 +    }
 +
 +    public static class SizeComparator implements Comparator<SSTableReader>
 +    {
 +        public int compare(SSTableReader o1, SSTableReader o2)
 +        {
 +            return Longs.compare(o1.onDiskLength(), o2.onDiskLength());
 +        }
 +    }
 +
 +    public Ref<SSTableReader> tryRef()
 +    {
 +        return selfRef.tryRef();
 +    }
 +
 +    public Ref<SSTableReader> selfRef()
 +    {
 +        return selfRef;
 +    }
 +
 +    public Ref<SSTableReader> ref()
 +    {
 +        return selfRef.ref();
 +    }
 +
 +    void setup(boolean trackHotness)
 +    {
 +        tidy.setup(this, trackHotness);
 +        this.readMeter = tidy.global.readMeter;
 +    }
 +
 +    @VisibleForTesting
 +    public void overrideReadMeter(RestorableMeter readMeter)
 +    {
 +        this.readMeter = tidy.global.readMeter = readMeter;
 +    }
 +
 +    /**
 +     * One instance per SSTableReader we create. This references the type-shared tidy, which in turn references
 +     * the globally shared tidy, i.e.
 +     *
 +     * InstanceTidier => DescriptorTypeTitdy => GlobalTidy
 +     *
 +     * We can create many InstanceTidiers (one for every time we reopen an sstable with MOVED_START for example), but there can only be
 +     * two DescriptorTypeTidy (FINAL and TEMPLINK) and only one GlobalTidy for one single logical sstable.
 +     *
 +     * When the InstanceTidier cleansup, it releases its reference to its DescriptorTypeTidy; when all InstanceTidiers
 +     * for that type have run, the DescriptorTypeTidy cleansup. DescriptorTypeTidy behaves in the same way towards GlobalTidy.
 +     *
 +     * For ease, we stash a direct reference to both our type-shared and global tidier
 +     */
 +    private static final class InstanceTidier implements Tidy
 +    {
 +        private final Descriptor descriptor;
 +        private final CFMetaData metadata;
 +        private IFilter bf;
 +        private IndexSummary summary;
 +
 +        private SegmentedFile dfile;
 +        private SegmentedFile ifile;
 +        private Runnable runOnClose;
 +        private boolean isReplaced = false;
 +
 +        // a reference to our shared per-Descriptor.Type tidy instance, that
 +        // we will release when we are ourselves released
 +        private Ref<DescriptorTypeTidy> typeRef;
 +
 +        // a convenience stashing of the shared per-descriptor-type tidy instance itself
 +        // and the per-logical-sstable globally shared state that it is linked to
 +        private DescriptorTypeTidy type;
 +        private GlobalTidy global;
 +
 +        private boolean setup;
 +
 +        void setup(SSTableReader reader, boolean trackHotness)
 +        {
 +            this.setup = true;
 +            this.bf = reader.bf;
 +            this.summary = reader.indexSummary;
 +            this.dfile = reader.dfile;
 +            this.ifile = reader.ifile;
 +            // get a new reference to the shared descriptor-type tidy
 +            this.typeRef = DescriptorTypeTidy.get(reader);
 +            this.type = typeRef.get();
 +            this.global = type.globalRef.get();
 +            if (trackHotness)
 +                global.ensureReadMeter();
 +        }
 +
 +        InstanceTidier(Descriptor descriptor, CFMetaData metadata)
 +        {
 +            this.descriptor = descriptor;
 +            this.metadata = metadata;
 +        }
 +
 +        public void tidy()
 +        {
 +            // don't try to cleanup if the sstablereader was never fully constructed
 +            if (!setup)
 +                return;
 +
 +            final ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(metadata.cfId);
 +            final OpOrder.Barrier barrier;
 +            if (cfs != null)
 +            {
 +                barrier = cfs.readOrdering.newBarrier();
 +                barrier.issue();
 +            }
 +            else
 +                barrier = null;
 +
 +            ScheduledExecutors.nonPeriodicTasks.execute(new Runnable()
 +            {
 +                public void run()
 +                {
 +                    if (barrier != null)
 +                        barrier.await();
 +                    if (bf != null)
 +                        bf.close();
 +                    if (summary != null)
 +                        summary.close();
 +                    if (runOnClose != null)
 +                        runOnClose.run();
 +                    if (dfile != null)
 +                        dfile.close();
 +                    if (ifile != null)
 +                        ifile.close();
 +                    typeRef.release();
 +                }
 +            });
 +        }
 +
 +        public String name()
 +        {
 +            return descriptor.toString();
 +        }
 +
 +        void releaseSummary()
 +        {
 +            summary.close();
 +            assert summary.isCleanedUp();
 +            summary = null;
 +        }
 +    }
 +
 +    /**
 +     * One shared between all instances of a given Descriptor.Type.
 +     * Performs only two things: the deletion of the sstables for the type,
 +     * if necessary; and the shared reference to the globally shared state.
 +     *
 +     * All InstanceTidiers, on setup(), ask the static get() method for their shared state,
 +     * and stash a reference to it to be released when they are. Once all such references are
 +     * released, the shared tidy will be performed.
 +     */
 +    static final class DescriptorTypeTidy implements Tidy
 +    {
 +        // keyed by REAL descriptor (TMPLINK/FINAL), mapping to the shared DescriptorTypeTidy for that descriptor
 +        static final ConcurrentMap<Descriptor, Ref<DescriptorTypeTidy>> lookup = new ConcurrentHashMap<>();
 +
 +        private final Descriptor desc;
 +        private final Ref<GlobalTidy> globalRef;
 +        private final Set<Component> components;
 +        private long sizeOnDelete;
 +        private Counter totalDiskSpaceUsed;
 +
 +        DescriptorTypeTidy(Descriptor desc, SSTableReader sstable)
 +        {
 +            this.desc = desc;
 +            // get a new reference to the shared global tidy
 +            this.globalRef = GlobalTidy.get(sstable);
 +            this.components = sstable.components;
 +        }
 +
 +        void markObsolete(SSTableReader instance, Tracker tracker)
 +        {
 +            // the tracker is used only to notify listeners of deletion of the sstable;
 +            // since deletion of a non-final file is not really deletion of the sstable,
 +            // we don't want to notify the listeners in this event
 +            if (tracker != null && tracker.cfstore != null && desc.type == Descriptor.Type.FINAL)
 +            {
 +                sizeOnDelete = instance.bytesOnDisk();
 +                totalDiskSpaceUsed = tracker.cfstore.metric.totalDiskSpaceUsed;
 +                tracker.notifyDeleting(instance);
 +            }
 +        }
 +
 +        public void tidy()
 +        {
 +            lookup.remove(desc);
 +            boolean isCompacted = globalRef.get().isCompacted.get();
 +            globalRef.release();
 +            switch (desc.type)
 +            {
 +                case FINAL:
 +                    if (isCompacted)
 +                        new SSTableDeletingTask(desc, components, totalDiskSpaceUsed, sizeOnDelete).run();
 +                    break;
 +                case TEMPLINK:
 +                    new SSTableDeletingTask(desc, components, null, 0).run();
 +                    break;
 +                default:
 +                    throw new IllegalStateException();
 +            }
 +        }
 +
 +        public String name()
 +        {
 +            return desc.toString();
 +        }
 +
 +        // get a new reference to the shared DescriptorTypeTidy for this sstable
 +        @Suppress

<TRUNCATED>

[05/16] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/util/ByteBufferDataInput.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/util/ByteBufferDataInput.java
index bf926e9,0000000..2f00687
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/io/util/ByteBufferDataInput.java
+++ b/src/java/org/apache/cassandra/io/util/ByteBufferDataInput.java
@@@ -1,171 -1,0 +1,177 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.io.util;
 +
 +import java.io.*;
 +import java.nio.ByteBuffer;
 +
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +
 +public class ByteBufferDataInput extends AbstractDataInput implements FileDataInput, DataInput
 +{
 +    private final ByteBuffer buffer;
 +    private final String filename;
 +    private final long segmentOffset;
 +    private int position;
 +
 +    public ByteBufferDataInput(ByteBuffer buffer, String filename, long segmentOffset, int position)
 +    {
 +        assert buffer != null;
 +        this.buffer = buffer;
 +        this.filename = filename;
 +        this.segmentOffset = segmentOffset;
 +        this.position = position;
 +    }
 +
 +    // Only use when we know the seek in within the mapped segment. Throws an
 +    // IOException otherwise.
 +    public void seek(long pos) throws IOException
 +    {
 +        long inSegmentPos = pos - segmentOffset;
-         if (inSegmentPos < 0 || inSegmentPos > buffer.capacity())
++        if (!contains(pos))
 +            throw new IOException(String.format("Seek position %d is not within mmap segment (seg offs: %d, length: %d)", pos, segmentOffset, buffer.capacity()));
 +
 +        position = (int) inSegmentPos;
 +    }
 +
++    public boolean contains(long pos)
++    {
++        long inSegmentPos = pos - segmentOffset;
++        return inSegmentPos >= 0 && inSegmentPos < buffer.capacity();
++    }
++
 +    public long getFilePointer()
 +    {
 +        return segmentOffset + position;
 +    }
 +
 +    public long getPosition()
 +    {
 +        return segmentOffset + position;
 +    }
 +
 +    public long getPositionLimit()
 +    {
 +        return segmentOffset + buffer.capacity();
 +    }
 +
 +    @Override
 +    public boolean markSupported()
 +    {
 +        return false;
 +    }
 +
 +    public void reset(FileMark mark) throws IOException
 +    {
 +        assert mark instanceof MappedFileDataInputMark;
 +        position = ((MappedFileDataInputMark) mark).position;
 +    }
 +
 +    public FileMark mark()
 +    {
 +        return new MappedFileDataInputMark(position);
 +    }
 +
 +    public long bytesPastMark(FileMark mark)
 +    {
 +        assert mark instanceof MappedFileDataInputMark;
 +        assert position >= ((MappedFileDataInputMark) mark).position;
 +        return position - ((MappedFileDataInputMark) mark).position;
 +    }
 +
 +    public boolean isEOF() throws IOException
 +    {
 +        return position == buffer.capacity();
 +    }
 +
 +    public long bytesRemaining() throws IOException
 +    {
 +        return buffer.capacity() - position;
 +    }
 +
 +    public String getPath()
 +    {
 +        return filename;
 +    }
 +
 +    public int read() throws IOException
 +    {
 +        if (isEOF())
 +            return -1;
 +        return buffer.get(position++) & 0xFF;
 +    }
 +
 +    /**
 +     * Does the same thing as <code>readFully</code> do but without copying data (thread safe)
 +     * @param length length of the bytes to read
 +     * @return buffer with portion of file content
 +     * @throws IOException on any fail of I/O operation
 +     */
 +    public ByteBuffer readBytes(int length) throws IOException
 +    {
 +        int remaining = buffer.remaining() - position;
 +        if (length > remaining)
 +            throw new IOException(String.format("mmap segment underflow; remaining is %d but %d requested",
 +                                                remaining, length));
 +
 +        if (length == 0)
 +            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
 +
 +        ByteBuffer bytes = buffer.duplicate();
 +        bytes.position(buffer.position() + position).limit(buffer.position() + position + length);
 +        position += length;
 +
 +        // we have to copy the data in case we unreference the underlying sstable.  See CASSANDRA-3179
 +        ByteBuffer clone = ByteBuffer.allocate(bytes.remaining());
 +        clone.put(bytes);
 +        clone.flip();
 +        return clone;
 +    }
 +
 +    @Override
 +    public final void readFully(byte[] bytes) throws IOException
 +    {
 +        ByteBufferUtil.arrayCopy(buffer, buffer.position() + position, bytes, 0, bytes.length);
 +        position += bytes.length;
 +    }
 +
 +    @Override
 +    public final void readFully(byte[] bytes, int offset, int count) throws IOException
 +    {
 +        ByteBufferUtil.arrayCopy(buffer, buffer.position() + position, bytes, offset, count);
 +        position += count;
 +    }
 +
 +    private static class MappedFileDataInputMark implements FileMark
 +    {
 +        int position;
 +
 +        MappedFileDataInputMark(int position)
 +        {
 +            this.position = position;
 +        }
 +    }
 +
 +    @Override
 +    public String toString() {
 +        return getClass().getSimpleName() + "(" +
 +               "filename='" + filename + "'" +
 +               ", position=" + position +
 +               ")";
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
index a8fae9f,623f65a..808b5ad
--- a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
@@@ -27,6 -28,13 +28,12 @@@ import com.google.common.annotations.Vi
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
+ import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.db.RowIndexEntry;
 -import org.apache.cassandra.io.FSReadError;
+ import org.apache.cassandra.io.sstable.Component;
+ import org.apache.cassandra.io.sstable.Descriptor;
+ import org.apache.cassandra.io.sstable.IndexSummary;
+ import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.utils.JVMStabilityInspector;
  
  public class MmappedSegmentedFile extends SegmentedFile
@@@ -135,6 -141,73 +142,74 @@@
          }
      }
  
+     // see CASSANDRA-10357
+     public static boolean maybeRepair(CFMetaData metadata, Descriptor descriptor, IndexSummary indexSummary, SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
+     {
+         boolean mayNeedRepair = false;
+         if (ibuilder instanceof Builder)
+             mayNeedRepair = ((Builder) ibuilder).mayNeedRepair(descriptor.filenameFor(Component.PRIMARY_INDEX));
+         if (dbuilder instanceof Builder)
+             mayNeedRepair |= ((Builder) dbuilder).mayNeedRepair(descriptor.filenameFor(Component.DATA));
+ 
+         if (mayNeedRepair)
+             forceRepair(metadata, descriptor, indexSummary, ibuilder, dbuilder);
+         return mayNeedRepair;
+     }
+ 
+     // if one of the index/data files have boundaries larger than we can mmap, and they were written by a version that did not guarantee correct boundaries were saved,
+     // rebuild the boundaries and save them again
+     private static void forceRepair(CFMetaData metadata, Descriptor descriptor, IndexSummary indexSummary, SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
+     {
+         if (ibuilder instanceof Builder)
+             ((Builder) ibuilder).boundaries.clear();
+         if (dbuilder instanceof Builder)
+             ((Builder) dbuilder).boundaries.clear();
+ 
++        RowIndexEntry.IndexSerializer rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata);
+         try (RandomAccessFile raf = new RandomAccessFile(descriptor.filenameFor(Component.PRIMARY_INDEX), "r");)
+         {
+             long iprev = 0, dprev = 0;
+             for (int i = 0; i < indexSummary.size(); i++)
+             {
+                 // first read the position in the summary, and read the corresponding position in the data file
+                 long icur = indexSummary.getPosition(i);
+                 raf.seek(icur);
+                 ByteBufferUtil.readWithShortLength(raf);
 -                RowIndexEntry rie = metadata.comparator.rowIndexEntrySerializer().deserialize(raf, descriptor.version);
++                RowIndexEntry rie = rowIndexEntrySerializer.deserialize(raf, descriptor.version);
+                 long dcur = rie.position;
+ 
+                 // if these positions are small enough to map out a segment from the prior version (i.e. less than 2Gb),
+                 // just add these as a boundary and proceed to the next index summary record; most scenarios will be
+                 // served by this, keeping the cost of rebuild to a minimum.
+ 
+                 if (Math.max(icur - iprev , dcur - dprev) > MAX_SEGMENT_SIZE)
+                 {
+                     // otherwise, loop over its index block, providing each RIE as a potential boundary for both files
+                     raf.seek(iprev);
+                     while (raf.getFilePointer() < icur)
+                     {
+                         // add the position of this record in the index file as an index file boundary
+                         ibuilder.addPotentialBoundary(raf.getFilePointer());
+                         // then read the RIE, and add its data file position as a boundary for the data file
+                         ByteBufferUtil.readWithShortLength(raf);
 -                        rie = metadata.comparator.rowIndexEntrySerializer().deserialize(raf, descriptor.version);
++                        rie = rowIndexEntrySerializer.deserialize(raf, descriptor.version);
+                         dbuilder.addPotentialBoundary(rie.position);
+                     }
+                 }
+ 
+                 ibuilder.addPotentialBoundary(icur);
+                 dbuilder.addPotentialBoundary(dcur);
+ 
+                 iprev = icur;
+                 dprev = dcur;
+             }
+         }
+         catch (IOException e)
+         {
+             logger.error("Failed to recalculate boundaries for {}; mmap access may degrade to buffered for this file", descriptor);
+         }
+     }
+ 
      /**
       * Overrides the default behaviour to create segments of a maximum size.
       */
@@@ -153,68 -326,83 +328,58 @@@
          public Builder()
          {
              super();
-             boundaries = new ArrayList<>();
-             boundaries.add(0L);
          }
  
-         public void addPotentialBoundary(long boundary)
+         public long[] boundaries()
          {
-             if (boundary - currentStart <= MAX_SEGMENT_SIZE)
-             {
-                 // boundary fits into current segment: expand it
-                 currentSize = boundary - currentStart;
-                 return;
-             }
+             return boundaries.truncate();
+         }
  
-             // close the current segment to try and make room for the boundary
-             if (currentSize > 0)
-             {
-                 currentStart += currentSize;
-                 boundaries.add(currentStart);
-             }
-             currentSize = boundary - currentStart;
+         // indicates if we may need to repair the mmapped file boundaries. this is a cheap check to see if there
+         // are any spans larger than an mmap segment size, which should be rare to occur in practice.
+         boolean mayNeedRepair(String path)
+         {
+             // old boundaries were created without the length, so add it as a candidate
+             long length = new File(path).length();
+             boundaries.addCandidate(length);
+             long[] boundaries = this.boundaries.truncate();
  
-             // if we couldn't make room, the boundary needs its own segment
-             if (currentSize > MAX_SEGMENT_SIZE)
+             long prev = 0;
+             for (long boundary : boundaries)
              {
-                 currentStart = boundary;
-                 boundaries.add(currentStart);
-                 currentSize = 0;
+                 if (boundary - prev > MAX_SEGMENT_SIZE)
+                     return true;
+                 prev = boundary;
              }
+             return false;
+         }
+ 
+         public void addPotentialBoundary(long boundary)
+         {
+             boundaries.addCandidate(boundary);
          }
  
 -        public SegmentedFile complete(String path, long overrideLength, boolean isFinal)
 +        public SegmentedFile complete(ChannelProxy channel, long overrideLength)
          {
 -            assert !isFinal || overrideLength <= 0;
 -            long length = overrideLength > 0 ? overrideLength : new File(path).length();
 +            long length = overrideLength > 0 ? overrideLength : channel.size();
              // create the segments
-             return new MmappedSegmentedFile(channel, length, createSegments(channel, length));
 -            return new MmappedSegmentedFile(path, length, createSegments(path, length, isFinal));
--        }
 -
 -        private Segment[] createSegments(String path, long length, boolean isFinal)
 -        {
 -            RandomAccessFile raf;
 -            try
 -            {
 -                raf = new RandomAccessFile(path, "r");
 -            }
 -            catch (IOException e)
 -            {
 -                throw new RuntimeException(e);
 -            }
  
-         private Segment[] createSegments(ChannelProxy channel, long length)
-         {
-             // if we're early finishing a range that doesn't span multiple segments, but the finished file now does,
-             // we remove these from the end (we loop incase somehow this spans multiple segments, but that would
-             // be a loco dataset
-             while (length < boundaries.get(boundaries.size() - 1))
-                 boundaries.remove(boundaries.size() -1);
- 
-             // add a sentinel value == length
-             List<Long> boundaries = new ArrayList<>(this.boundaries);
-             if (length != boundaries.get(boundaries.size() - 1))
-                 boundaries.add(length);
- 
-             int segcount = boundaries.size() - 1;
 -            long[] boundaries = this.boundaries.finish(length, isFinal);
++            long[] boundaries = this.boundaries.finish(length, overrideLength <= 0);
+ 
+             int segcount = boundaries.length - 1;
              Segment[] segments = new Segment[segcount];
+ 
 -            try
 -            {
 -                for (int i = 0; i < segcount; i++)
 -                {
 -                    long start = boundaries[i];
 -                    long size = boundaries[i + 1] - start;
 -                    MappedByteBuffer segment = size <= MAX_SEGMENT_SIZE
 -                                               ? raf.getChannel().map(FileChannel.MapMode.READ_ONLY, start, size)
 -                                               : null;
 -                    segments[i] = new Segment(start, segment);
 -                }
 -            }
 -            catch (IOException e)
 -            {
 -                throw new FSReadError(e, path);
 -            }
 -            finally
 +            for (int i = 0; i < segcount; i++)
              {
-                 long start = boundaries.get(i);
-                 long size = boundaries.get(i + 1) - start;
 -                FileUtils.closeQuietly(raf);
++                long start = boundaries[i];
++                long size = boundaries[i + 1] - start;
 +                MappedByteBuffer segment = size <= MAX_SEGMENT_SIZE
 +                                           ? channel.map(FileChannel.MapMode.READ_ONLY, start, size)
 +                                           : null;
 +                segments[i] = new Segment(start, segment);
              }
--            return segments;
++
++            return new MmappedSegmentedFile(channel, length, segments);
          }
  
          @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/util/SegmentedFile.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/util/SegmentedFile.java
index 66898c6,23454bc..30707d8
--- a/src/java/org/apache/cassandra/io/util/SegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/SegmentedFile.java
@@@ -23,9 -23,9 +23,10 @@@ import java.io.File
  import java.io.IOException;
  import java.nio.MappedByteBuffer;
  import java.util.Iterator;
+ import java.util.List;
  import java.util.NoSuchElementException;
  
 +import com.google.common.base.Throwables;
  import com.google.common.util.concurrent.RateLimiter;
  
  import org.apache.cassandra.config.Config;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
----------------------------------------------------------------------
diff --cc test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
index 0000000,e17c6a7..4913b32
mode 000000,100644..100644
--- a/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
+++ b/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
@@@ -1,0 -1,322 +1,324 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ package org.apache.cassandra.io.sstable;
+ 
+ import java.io.*;
+ import java.nio.ByteBuffer;
+ import java.util.Arrays;
+ import java.util.Random;
+ 
+ import com.google.common.io.Files;
+ import org.junit.AfterClass;
+ import org.junit.BeforeClass;
+ import org.junit.Test;
+ 
+ import junit.framework.Assert;
+ import org.apache.cassandra.SchemaLoader;
+ import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.config.Config;
+ import org.apache.cassandra.config.DatabaseDescriptor;
+ import org.apache.cassandra.db.Keyspace;
+ import org.apache.cassandra.dht.ByteOrderedPartitioner;
+ import org.apache.cassandra.exceptions.InvalidRequestException;
++import org.apache.cassandra.io.sstable.format.SSTableReader;
+ import org.apache.cassandra.io.util.DataOutputStreamPlus;
+ import org.apache.cassandra.io.util.FileUtils;
+ import org.apache.cassandra.io.util.MmappedSegmentedFile;
+ import org.apache.cassandra.io.util.MmappedSegmentedFile.Builder.Boundaries;
++import org.apache.cassandra.io.util.WrappedDataOutputStreamPlus;
+ import org.apache.cassandra.service.StorageService;
+ import org.apache.cassandra.utils.ByteBufferUtil;
+ 
+ public class LongSegmentedFileBoundaryTest
+ {
+     @BeforeClass
+     public static void setup() throws Exception
+     {
+         SchemaLoader.cleanupAndLeaveDirs();
+         Keyspace.setInitialized();
+         StorageService.instance.initServer();
+     }
+ 
+     @AfterClass
+     public static void tearDown()
+     {
+         Config.setClientMode(false);
+     }
+ 
+     @Test
+     public void testRandomBoundaries()
+     {
+         long[] candidates = new long[1 + (1 << 16)];
+         int[] indexesToCheck = new int[1 << 8];
+         Random random = new Random();
+ 
+         for (int run = 0; run < 100; run++)
+         {
+ 
+             long seed = random.nextLong();
+             random.setSeed(seed);
+             System.out.println("Seed: " + seed);
+ 
+             // at least 1Ki, and as many as 256Ki, boundaries
+             int candidateCount = (1 + random.nextInt(candidates.length >> 10)) << 10;
+             generateBoundaries(random, candidateCount, candidates, indexesToCheck);
+ 
+             Boundaries builder = new Boundaries();
+             int nextIndexToCheck = indexesToCheck[0];
+             int checkCount = 0;
+             System.out.printf("[0..%d)", candidateCount);
+             for (int i = 1; i < candidateCount - 1; i++)
+             {
+                 if (i == nextIndexToCheck)
+                 {
+                     if (checkCount % 20 == 0)
+                         System.out.printf(" %d", i);
+                     // grow number of samples logarithmically; work will still increase superlinearly, as size of dataset grows linearly
+                     int sampleCount = 1 << (31 - Integer.numberOfLeadingZeros(++checkCount));
+                     checkBoundarySample(random, candidates, i, sampleCount, builder);
+                     // select out next index to check (there may be dups, so skip them)
+                     while ((nextIndexToCheck = checkCount == indexesToCheck.length ? candidateCount : indexesToCheck[checkCount]) == i)
+                         checkCount++;
+                 }
+ 
+                 builder.addCandidate(candidates[i]);
+             }
+             System.out.println();
+             checkBoundaries(candidates, candidateCount - 1, builder, candidates[candidateCount - 1]);
+             Assert.assertEquals(candidateCount, nextIndexToCheck);
+         }
+     }
+ 
+     private static void generateBoundaries(Random random, int candidateCount, long[] candidates, int[] indexesToCheck)
+     {
+         // average averageBoundarySize is 4MiB, max 4GiB, min 4KiB
+         long averageBoundarySize = (4L << 10) * random.nextInt(1 << 20);
+         long prev = 0;
+         for (int i = 1 ; i < candidateCount ; i++)
+             candidates[i] = prev += Math.max(1, averageBoundarySize + (random.nextGaussian() * averageBoundarySize));
+ 
+         // generate indexes we will corroborate our behaviour on
+         for (int i = 0 ; i < indexesToCheck.length ; i++)
+             indexesToCheck[i] = 1 + random.nextInt(candidateCount - 2);
+         Arrays.sort(indexesToCheck);
+     }
+ 
+     private static void checkBoundarySample(Random random, long[] candidates, int candidateCount, int sampleCount, Boundaries builder)
+     {
+         for (int i = 0 ; i < sampleCount ; i++)
+         {
+             // pick a number exponentially less likely to be near the beginning, since we test that area earlier
+             int position = 0 ;
+             while (position <= 0)
+                 position = candidateCount / (Integer.lowestOneBit(random.nextInt()));
+             long upperBound = candidates[position];
+             long lowerBound = random.nextBoolean() ? (rand(random, 0, upperBound) / (Integer.lowestOneBit(random.nextInt())))
+                                                    : candidates[Math.max(0, position - random.nextInt(64))];
+             long length = rand(random, lowerBound, upperBound);
+             checkBoundaries(candidates, candidateCount, builder, length);
+         }
+         checkBoundaries(candidates, candidateCount, builder, candidates[candidateCount]);
+     }
+ 
+     private static long rand(Random random, long lowerBound, long upperBound)
+     {
+         if (upperBound == lowerBound)
+             return upperBound;
+         return lowerBound + ((random.nextLong() & Long.MAX_VALUE) % (upperBound - lowerBound));
+     }
+ 
+     private static void checkBoundaries(long[] candidates, int candidateCount, Boundaries builder, long length)
+     {
+         if (length == 0)
+             return;
+ 
+         long[] boundaries = new long[(int) (10 + 2 * (length / Integer.MAX_VALUE))];
+         int count = 1;
+         int prev = 0;
+         while (true)
+         {
+             int p = candidates[prev + 1] - boundaries[count - 1] >= Integer.MAX_VALUE
+                     ? prev + 1
+                     : Arrays.binarySearch(candidates, prev, candidateCount, boundaries[count - 1] + Integer.MAX_VALUE);
+             if (p < 0) p = -2 -p;
+             if (p >= candidateCount - 1 || candidates[p] >= length)
+                 break;
+             boundaries[count++] = candidates[p];
+             if (candidates[p + 1] >= length)
+                 break;
+             prev = p;
+         }
+         if (candidates[candidateCount - 1] < length && length - boundaries[count - 1] >= Integer.MAX_VALUE)
+             boundaries[count++] = candidates[candidateCount - 1];
+         boundaries[count++] = length;
+         final long[] canon = Arrays.copyOf(boundaries, count);
+         final long[] check = builder.finish(length, false);
+         if (!Arrays.equals(canon, check))
+             Assert.assertTrue("\n" + Arrays.toString(canon) + "\n" + Arrays.toString(check), Arrays.equals(canon, check));
+     }
+ 
+     @Test
+     public void testBoundariesAndRepairSmall() throws InvalidRequestException, IOException
+     {
+         testBoundariesAndRepair(1, 1 << 16);
+     }
+ 
+     @Test
+     public void testBoundariesAndRepairMedium() throws InvalidRequestException, IOException
+     {
+         testBoundariesAndRepair(1, 1 << 20);
+     }
+ 
+     @Test
+     public void testBoundariesAndRepairLarge() throws InvalidRequestException, IOException
+     {
+         testBoundariesAndRepair(1, 100 << 20);
+     }
+ 
+     @Test
+     public void testBoundariesAndRepairHuge() throws InvalidRequestException, IOException
+     {
+         testBoundariesAndRepair(1, Integer.MAX_VALUE - 1024);
+     }
+ 
+     @Test
+     public void testBoundariesAndRepairTooHuge() throws InvalidRequestException, IOException
+     {
+         testBoundariesAndRepair(1, Integer.MAX_VALUE);
+     }
+ 
+     @Test
+     public void testBoundariesAndRepairHugeIndex() throws InvalidRequestException, IOException
+     {
+         testBoundariesAndRepair(1 << 7, 1 << 15);
+     }
+ 
+     @Test
+     public void testBoundariesAndRepairReallyHugeIndex() throws InvalidRequestException, IOException
+     {
+         testBoundariesAndRepair(1 << 14, 1 << 15);
+     }
+ 
+     private void testBoundariesAndRepair(int rows, int rowSize) throws InvalidRequestException, IOException
+     {
+         String KS = "cql_keyspace";
+         String TABLE = "table1";
+ 
+         File tempdir = Files.createTempDir();
+         try
+         {
+             Assert.assertTrue(DatabaseDescriptor.getColumnIndexSize() < rowSize);
+             Assert.assertTrue(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap);
+             Assert.assertTrue(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap);
+             Assert.assertTrue(StorageService.getPartitioner() instanceof ByteOrderedPartitioner);
+             File dataDir = new File(tempdir.getAbsolutePath() + File.separator + KS + File.separator + TABLE);
+             Assert.assertTrue(dataDir.mkdirs());
+ 
+             String schema = "CREATE TABLE cql_keyspace.table" + (rows > 1 ? "2" : "1") + " (k bigint, v1 blob, v2 blob, v3 blob, v4 blob, v5 blob, PRIMARY KEY (k" + (rows > 1 ? ", v1" : "") + ")) WITH compression = { 'sstable_compression':'' };";
+             String insert = "INSERT INTO cql_keyspace.table" + (rows > 1 ? "2" : "1") + " (k, v1, v2, v3, v4, v5) VALUES (?, ?, ?, ?, ?, ?)";
+ 
+             CQLSSTableWriter.Builder builder = CQLSSTableWriter.builder()
+                                                       .inDirectory(dataDir)
+                                                       .forTable(schema)
+                                                       .withPartitioner(StorageService.getPartitioner())
+                                                       .using(insert)
+                                                       .sorted();
+             CQLSSTableWriter writer = builder.build();
+ 
+             // write 8Gb of decorated keys
+             ByteBuffer[] value = new ByteBuffer[rows];
+             for (int row = 0 ; row < rows ; row++)
+             {
+                 // if we're using clustering columns, the clustering key is replicated across every other column
+                 value[row] = ByteBuffer.allocate(rowSize / (rows > 1 ? 8 : 5));
+                 value[row].putInt(0, row);
+             }
+             long targetSize = 8L << 30;
+             long dk = 0;
+             long size = 0;
+             long dkSize = rowSize * rows;
+             while (size < targetSize)
+             {
+                 for (int row = 0 ; row < rows ; row++)
+                     writer.addRow(dk, value[row], value[row], value[row], value[row], value[row]);
+                 size += dkSize;
+                 dk++;
+             }
+ 
+             Descriptor descriptor = writer.getCurrentDescriptor().asType(Descriptor.Type.FINAL);
+             writer.close();
+ 
+             // open (and close) the reader so that the summary file is created
+             SSTableReader reader = SSTableReader.open(descriptor);
+             reader.selfRef().release();
+ 
+             // then check the boundaries are reasonable, and corrupt them
+             checkThenCorruptBoundaries(descriptor, rows * rowSize < Integer.MAX_VALUE);
+ 
+             // then check that reopening corrects the corruption
+             reader = SSTableReader.open(descriptor);
+             reader.selfRef().release();
+             checkThenCorruptBoundaries(descriptor, rows * rowSize < Integer.MAX_VALUE);
+         }
+         finally
+         {
+             FileUtils.deleteRecursive(tempdir);
+         }
+     }
+ 
+     private static void checkThenCorruptBoundaries(Descriptor descriptor, boolean expectDataMmappable) throws IOException
+     {
+         File summaryFile = new File(descriptor.filenameFor(Component.SUMMARY));
+         DataInputStream iStream = new DataInputStream(new FileInputStream(summaryFile));
+         IndexSummary indexSummary = IndexSummary.serializer.deserialize(iStream, StorageService.getPartitioner(), true, CFMetaData.DEFAULT_MIN_INDEX_INTERVAL, CFMetaData.DEFAULT_MAX_INDEX_INTERVAL);
+         ByteBuffer first = ByteBufferUtil.readWithLength(iStream);
+         ByteBuffer last = ByteBufferUtil.readWithLength(iStream);
+         MmappedSegmentedFile.Builder ibuilder = new MmappedSegmentedFile.Builder();
+         MmappedSegmentedFile.Builder dbuilder = new MmappedSegmentedFile.Builder();
+         ibuilder.deserializeBounds(iStream);
+         dbuilder.deserializeBounds(iStream);
+         iStream.close();
+         // index file cannot generally be non-mmappable, as index entries cannot be larger than MAX_SEGMENT_SIZE (due to promotedSize being encoded as an int)
+         assertBoundaries(descriptor.filenameFor(Component.PRIMARY_INDEX), true, ibuilder.boundaries());
+         assertBoundaries(descriptor.filenameFor(Component.DATA), expectDataMmappable, dbuilder.boundaries());
+ 
 -        DataOutputStreamPlus oStream = new DataOutputStreamPlus(new FileOutputStream(summaryFile));
++        DataOutputStreamPlus oStream = new WrappedDataOutputStreamPlus(new FileOutputStream(summaryFile));
+         IndexSummary.serializer.serialize(indexSummary, oStream, true);
+         ByteBufferUtil.writeWithLength(first, oStream);
+         ByteBufferUtil.writeWithLength(last, oStream);
+         oStream.writeInt(1);
+         oStream.writeLong(new File(descriptor.filenameFor(Component.PRIMARY_INDEX)).length());
+         oStream.writeLong(new File(descriptor.filenameFor(Component.DATA)).length());
+         oStream.close();
+     }
+ 
+     private static void assertBoundaries(String path, boolean expectMmappable, long[] boundaries)
+     {
+         long length = new File(path).length();
+         long prev = boundaries[0];
+         for (int i = 1 ; i <= boundaries.length && prev < length ; i++)
+         {
+             long boundary = i == boundaries.length ? length : boundaries[i];
+             Assert.assertEquals(String.format("[%d, %d), %d of %d", boundary, prev, i, boundaries.length),
+                                 expectMmappable, boundary - prev <= Integer.MAX_VALUE);
+             prev = boundary;
+         }
+     }
+ 
+ }


[06/16] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index 87891ae,0000000..2b65792
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@@ -1,2287 -1,0 +1,2304 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.io.sstable.format;
 +
 +import java.io.*;
 +import java.nio.ByteBuffer;
 +import java.util.*;
 +import java.util.concurrent.*;
 +import java.util.concurrent.atomic.AtomicBoolean;
 +import java.util.concurrent.atomic.AtomicLong;
 +
 +import com.google.common.annotations.VisibleForTesting;
 +import com.google.common.base.Predicate;
 +import com.google.common.collect.Iterables;
 +import com.google.common.collect.Iterators;
 +import com.google.common.collect.Ordering;
 +import com.google.common.primitives.Longs;
 +import com.google.common.util.concurrent.RateLimiter;
 +
 +import com.clearspring.analytics.stream.cardinality.CardinalityMergeException;
 +import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus;
 +import com.clearspring.analytics.stream.cardinality.ICardinality;
 +import com.codahale.metrics.Counter;
 +import org.apache.cassandra.cache.CachingOptions;
 +import org.apache.cassandra.cache.InstrumentingCache;
 +import org.apache.cassandra.cache.KeyCacheKey;
 +import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
 +import org.apache.cassandra.concurrent.ScheduledExecutors;
 +import org.apache.cassandra.config.*;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 +import org.apache.cassandra.db.commitlog.ReplayPosition;
 +import org.apache.cassandra.db.composites.CellName;
 +import org.apache.cassandra.db.filter.ColumnSlice;
 +import org.apache.cassandra.db.index.SecondaryIndex;
 +import org.apache.cassandra.db.lifecycle.Tracker;
 +import org.apache.cassandra.dht.*;
 +import org.apache.cassandra.io.FSError;
++import org.apache.cassandra.io.FSReadError;
 +import org.apache.cassandra.io.compress.CompressionMetadata;
 +import org.apache.cassandra.io.sstable.*;
 +import org.apache.cassandra.io.sstable.metadata.*;
 +import org.apache.cassandra.io.util.*;
 +import org.apache.cassandra.metrics.RestorableMeter;
 +import org.apache.cassandra.metrics.StorageMetrics;
 +import org.apache.cassandra.service.ActiveRepairService;
 +import org.apache.cassandra.service.CacheService;
 +import org.apache.cassandra.service.StorageService;
 +import org.apache.cassandra.utils.*;
 +import org.apache.cassandra.utils.concurrent.OpOrder;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +import org.apache.cassandra.utils.concurrent.Ref;
 +import org.apache.cassandra.utils.concurrent.SelfRefCounted;
 +
 +import static org.apache.cassandra.db.Directories.SECONDARY_INDEX_NAME_SEPARATOR;
 +
 +/**
 + * An SSTableReader can be constructed in a number of places, but typically is either
 + * read from disk at startup, or constructed from a flushed memtable, or after compaction
 + * to replace some existing sstables. However once created, an sstablereader may also be modified.
 + *
 + * A reader's OpenReason describes its current stage in its lifecycle, as follows:
 + *
 + *
 + * <pre> {@code
 + * NORMAL
 + * From:       None        => Reader has been read from disk, either at startup or from a flushed memtable
 + *             EARLY       => Reader is the final result of a compaction
 + *             MOVED_START => Reader WAS being compacted, but this failed and it has been restored to NORMAL status
 + *
 + * EARLY
 + * From:       None        => Reader is a compaction replacement that is either incomplete and has been opened
 + *                            to represent its partial result status, or has been finished but the compaction
 + *                            it is a part of has not yet completed fully
 + *             EARLY       => Same as from None, only it is not the first time it has been
 + *
 + * MOVED_START
 + * From:       NORMAL      => Reader is being compacted. This compaction has not finished, but the compaction result
 + *                            is either partially or fully opened, to either partially or fully replace this reader.
 + *                            This reader's start key has been updated to represent this, so that reads only hit
 + *                            one or the other reader.
 + *
 + * METADATA_CHANGE
 + * From:       NORMAL      => Reader has seen low traffic and the amount of memory available for index summaries is
 + *                            constrained, so its index summary has been downsampled.
 + *         METADATA_CHANGE => Same
 + * } </pre>
 + *
 + * Note that in parallel to this, there are two different Descriptor types; TMPLINK and FINAL; the latter corresponds
 + * to NORMAL state readers and all readers that replace a NORMAL one. TMPLINK is used for EARLY state readers and
 + * no others.
 + *
 + * When a reader is being compacted, if the result is large its replacement may be opened as EARLY before compaction
 + * completes in order to present the result to consumers earlier. In this case the reader will itself be changed to
 + * a MOVED_START state, where its start no longer represents its on-disk minimum key. This is to permit reads to be
 + * directed to only one reader when the two represent the same data. The EARLY file can represent a compaction result
 + * that is either partially complete and still in-progress, or a complete and immutable sstable that is part of a larger
 + * macro compaction action that has not yet fully completed.
 + *
 + * Currently ALL compaction results at least briefly go through an EARLY open state prior to completion, regardless
 + * of if early opening is enabled.
 + *
 + * Since a reader can be created multiple times over the same shared underlying resources, and the exact resources
 + * it shares between each instance differ subtly, we track the lifetime of any underlying resource with its own
 + * reference count, which each instance takes a Ref to. Each instance then tracks references to itself, and once these
 + * all expire it releases its Refs to these underlying resources.
 + *
 + * There is some shared cleanup behaviour needed only once all sstablereaders in a certain stage of their lifecycle
 + * (i.e. EARLY or NORMAL opening), and some that must only occur once all readers of any kind over a single logical
 + * sstable have expired. These are managed by the TypeTidy and GlobalTidy classes at the bottom, and are effectively
 + * managed as another resource each instance tracks its own Ref instance to, to ensure all of these resources are
 + * cleaned up safely and can be debugged otherwise.
 + *
 + * TODO: fill in details about Tracker and lifecycle interactions for tools, and for compaction strategies
 + */
 +public abstract class SSTableReader extends SSTable implements SelfRefCounted<SSTableReader>
 +{
 +    private static final Logger logger = LoggerFactory.getLogger(SSTableReader.class);
++    private static final int ACCURATE_BOUNDARIES_MAGIC_NUMBER = 248923458;
 +
 +    private static final ScheduledThreadPoolExecutor syncExecutor = new ScheduledThreadPoolExecutor(1);
 +    static
 +    {
 +        // Immediately remove readMeter sync task when cancelled.
 +        syncExecutor.setRemoveOnCancelPolicy(true);
 +    }
 +    private static final RateLimiter meterSyncThrottle = RateLimiter.create(100.0);
 +
 +    public static final Comparator<SSTableReader> maxTimestampComparator = new Comparator<SSTableReader>()
 +    {
 +        public int compare(SSTableReader o1, SSTableReader o2)
 +        {
 +            long ts1 = o1.getMaxTimestamp();
 +            long ts2 = o2.getMaxTimestamp();
 +            return (ts1 > ts2 ? -1 : (ts1 == ts2 ? 0 : 1));
 +        }
 +    };
 +
 +    // it's just an object, which we use regular Object equality on; we introduce a special class just for easy recognition
 +    public static final class UniqueIdentifier {}
 +
 +    public static final Comparator<SSTableReader> sstableComparator = new Comparator<SSTableReader>()
 +    {
 +        public int compare(SSTableReader o1, SSTableReader o2)
 +        {
 +            return o1.first.compareTo(o2.first);
 +        }
 +    };
 +
 +    public static final Ordering<SSTableReader> sstableOrdering = Ordering.from(sstableComparator);
 +
 +    /**
 +     * maxDataAge is a timestamp in local server time (e.g. System.currentTimeMilli) which represents an upper bound
 +     * to the newest piece of data stored in the sstable. In other words, this sstable does not contain items created
 +     * later than maxDataAge.
 +     *
 +     * The field is not serialized to disk, so relying on it for more than what truncate does is not advised.
 +     *
 +     * When a new sstable is flushed, maxDataAge is set to the time of creation.
 +     * When a sstable is created from compaction, maxDataAge is set to max of all merged sstables.
 +     *
 +     * The age is in milliseconds since epoc and is local to this host.
 +     */
 +    public final long maxDataAge;
 +
 +    public enum OpenReason
 +    {
 +        NORMAL,
 +        EARLY,
 +        METADATA_CHANGE,
 +        MOVED_START
 +    }
 +
 +    public final OpenReason openReason;
 +    public final UniqueIdentifier instanceId = new UniqueIdentifier();
 +
 +    // indexfile and datafile: might be null before a call to load()
 +    protected SegmentedFile ifile;
 +    protected SegmentedFile dfile;
 +    protected IndexSummary indexSummary;
 +    protected IFilter bf;
 +
 +    protected final RowIndexEntry.IndexSerializer rowIndexEntrySerializer;
 +
 +    protected InstrumentingCache<KeyCacheKey, RowIndexEntry> keyCache;
 +
 +    protected final BloomFilterTracker bloomFilterTracker = new BloomFilterTracker();
 +
 +    // technically isCompacted is not necessary since it should never be unreferenced unless it is also compacted,
 +    // but it seems like a good extra layer of protection against reference counting bugs to not delete data based on that alone
 +    protected final AtomicBoolean isSuspect = new AtomicBoolean(false);
 +
 +    // not final since we need to be able to change level on a file.
 +    protected volatile StatsMetadata sstableMetadata;
 +
 +    protected final AtomicLong keyCacheHit = new AtomicLong(0);
 +    protected final AtomicLong keyCacheRequest = new AtomicLong(0);
 +
 +    private final InstanceTidier tidy = new InstanceTidier(descriptor, metadata);
 +    private final Ref<SSTableReader> selfRef = new Ref<>(this, tidy);
 +
 +    private RestorableMeter readMeter;
 +
 +    /**
 +     * Calculate approximate key count.
 +     * If cardinality estimator is available on all given sstables, then this method use them to estimate
 +     * key count.
 +     * If not, then this uses index summaries.
 +     *
 +     * @param sstables SSTables to calculate key count
 +     * @return estimated key count
 +     */
 +    public static long getApproximateKeyCount(Collection<SSTableReader> sstables)
 +    {
 +        long count = -1;
 +
 +        // check if cardinality estimator is available for all SSTables
 +        boolean cardinalityAvailable = !sstables.isEmpty() && Iterators.all(sstables.iterator(), new Predicate<SSTableReader>()
 +        {
 +            public boolean apply(SSTableReader sstable)
 +            {
 +                return sstable.descriptor.version.hasNewStatsFile();
 +            }
 +        });
 +
 +        // if it is, load them to estimate key count
 +        if (cardinalityAvailable)
 +        {
 +            boolean failed = false;
 +            ICardinality cardinality = null;
 +            for (SSTableReader sstable : sstables)
 +            {
 +                if (sstable.openReason == OpenReason.EARLY)
 +                    continue;
 +
 +                try
 +                {
 +                    CompactionMetadata metadata = (CompactionMetadata) sstable.descriptor.getMetadataSerializer().deserialize(sstable.descriptor, MetadataType.COMPACTION);
 +                    assert metadata != null : sstable.getFilename();
 +                    if (cardinality == null)
 +                        cardinality = metadata.cardinalityEstimator;
 +                    else
 +                        cardinality = cardinality.merge(metadata.cardinalityEstimator);
 +                }
 +                catch (IOException e)
 +                {
 +                    logger.warn("Reading cardinality from Statistics.db failed.", e);
 +                    failed = true;
 +                    break;
 +                }
 +                catch (CardinalityMergeException e)
 +                {
 +                    logger.warn("Cardinality merge failed.", e);
 +                    failed = true;
 +                    break;
 +                }
 +            }
 +            if (cardinality != null && !failed)
 +                count = cardinality.cardinality();
 +        }
 +
 +        // if something went wrong above or cardinality is not available, calculate using index summary
 +        if (count < 0)
 +        {
 +            for (SSTableReader sstable : sstables)
 +                count += sstable.estimatedKeys();
 +        }
 +        return count;
 +    }
 +
 +    /**
 +     * Estimates how much of the keys we would keep if the sstables were compacted together
 +     */
 +    public static double estimateCompactionGain(Set<SSTableReader> overlapping)
 +    {
 +        Set<ICardinality> cardinalities = new HashSet<>(overlapping.size());
 +        for (SSTableReader sstable : overlapping)
 +        {
 +            try
 +            {
 +                ICardinality cardinality = ((CompactionMetadata) sstable.descriptor.getMetadataSerializer().deserialize(sstable.descriptor, MetadataType.COMPACTION)).cardinalityEstimator;
 +                if (cardinality != null)
 +                    cardinalities.add(cardinality);
 +                else
 +                    logger.trace("Got a null cardinality estimator in: {}", sstable.getFilename());
 +            }
 +            catch (IOException e)
 +            {
 +                logger.warn("Could not read up compaction metadata for {}", sstable, e);
 +            }
 +        }
 +        long totalKeyCountBefore = 0;
 +        for (ICardinality cardinality : cardinalities)
 +        {
 +            totalKeyCountBefore += cardinality.cardinality();
 +        }
 +        if (totalKeyCountBefore == 0)
 +            return 1;
 +
 +        long totalKeyCountAfter = mergeCardinalities(cardinalities).cardinality();
 +        logger.trace("Estimated compaction gain: {}/{}={}", totalKeyCountAfter, totalKeyCountBefore, ((double)totalKeyCountAfter)/totalKeyCountBefore);
 +        return ((double)totalKeyCountAfter)/totalKeyCountBefore;
 +    }
 +
 +    private static ICardinality mergeCardinalities(Collection<ICardinality> cardinalities)
 +    {
 +        ICardinality base = new HyperLogLogPlus(13, 25); // see MetadataCollector.cardinality
 +        try
 +        {
 +            base = base.merge(cardinalities.toArray(new ICardinality[cardinalities.size()]));
 +        }
 +        catch (CardinalityMergeException e)
 +        {
 +            logger.warn("Could not merge cardinalities", e);
 +        }
 +        return base;
 +    }
 +
 +    public static SSTableReader open(Descriptor descriptor) throws IOException
 +    {
 +        CFMetaData metadata;
 +        if (descriptor.cfname.contains(SECONDARY_INDEX_NAME_SEPARATOR))
 +        {
 +            int i = descriptor.cfname.indexOf(SECONDARY_INDEX_NAME_SEPARATOR);
 +            String parentName = descriptor.cfname.substring(0, i);
 +            CFMetaData parent = Schema.instance.getCFMetaData(descriptor.ksname, parentName);
 +            ColumnDefinition def = parent.getColumnDefinitionForIndex(descriptor.cfname.substring(i + 1));
 +            metadata = CFMetaData.newIndexMetadata(parent, def, SecondaryIndex.getIndexComparator(parent, def));
 +        }
 +        else
 +        {
 +            metadata = Schema.instance.getCFMetaData(descriptor.ksname, descriptor.cfname);
 +        }
 +        return open(descriptor, metadata);
 +    }
 +
 +    public static SSTableReader open(Descriptor desc, CFMetaData metadata) throws IOException
 +    {
 +        IPartitioner p = desc.cfname.contains(SECONDARY_INDEX_NAME_SEPARATOR)
 +                ? new LocalPartitioner(metadata.getKeyValidator())
 +                : StorageService.getPartitioner();
 +        return open(desc, componentsFor(desc), metadata, p);
 +    }
 +
 +    public static SSTableReader open(Descriptor descriptor, Set<Component> components, CFMetaData metadata, IPartitioner partitioner) throws IOException
 +    {
 +        return open(descriptor, components, metadata, partitioner, true, true);
 +    }
 +
 +    // use only for offline or "Standalone" operations
 +    public static SSTableReader openNoValidation(Descriptor descriptor, Set<Component> components, ColumnFamilyStore cfs) throws IOException
 +    {
 +        return open(descriptor, components, cfs.metadata, cfs.partitioner, false, false); // do not track hotness
 +    }
 +
 +    /**
 +     * Open SSTable reader to be used in batch mode(such as sstableloader).
 +     *
 +     * @param descriptor
 +     * @param components
 +     * @param metadata
 +     * @param partitioner
 +     * @return opened SSTableReader
 +     * @throws IOException
 +     */
 +    public static SSTableReader openForBatch(Descriptor descriptor, Set<Component> components, CFMetaData metadata, IPartitioner partitioner) throws IOException
 +    {
 +        // Minimum components without which we can't do anything
 +        assert components.contains(Component.DATA) : "Data component is missing for sstable " + descriptor;
 +        assert components.contains(Component.PRIMARY_INDEX) : "Primary index component is missing for sstable " + descriptor;
 +
 +        Map<MetadataType, MetadataComponent> sstableMetadata = descriptor.getMetadataSerializer().deserialize(descriptor,
 +                EnumSet.of(MetadataType.VALIDATION, MetadataType.STATS));
 +        ValidationMetadata validationMetadata = (ValidationMetadata) sstableMetadata.get(MetadataType.VALIDATION);
 +        StatsMetadata statsMetadata = (StatsMetadata) sstableMetadata.get(MetadataType.STATS);
 +
 +        // Check if sstable is created using same partitioner.
 +        // Partitioner can be null, which indicates older version of sstable or no stats available.
 +        // In that case, we skip the check.
 +        String partitionerName = partitioner.getClass().getCanonicalName();
 +        if (validationMetadata != null && !partitionerName.equals(validationMetadata.partitioner))
 +        {
 +            logger.error(String.format("Cannot open %s; partitioner %s does not match system partitioner %s.  Note that the default partitioner starting with Cassandra 1.2 is Murmur3Partitioner, so you will need to edit that to match your old partitioner if upgrading.",
 +                    descriptor, validationMetadata.partitioner, partitionerName));
 +            System.exit(1);
 +        }
 +
 +        logger.debug("Opening {} ({} bytes)", descriptor, new File(descriptor.filenameFor(Component.DATA)).length());
 +        SSTableReader sstable = internalOpen(descriptor, components, metadata, partitioner, System.currentTimeMillis(),
 +                statsMetadata, OpenReason.NORMAL);
 +
 +        // special implementation of load to use non-pooled SegmentedFile builders
 +        try(SegmentedFile.Builder ibuilder = new BufferedSegmentedFile.Builder();
 +            SegmentedFile.Builder dbuilder = sstable.compression
 +                ? new CompressedSegmentedFile.Builder(null)
 +                : new BufferedSegmentedFile.Builder())
 +        {
 +            if (!sstable.loadSummary(ibuilder, dbuilder))
 +                sstable.buildSummary(false, ibuilder, dbuilder, false, Downsampling.BASE_SAMPLING_LEVEL);
 +            sstable.ifile = ibuilder.complete(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX));
 +            sstable.dfile = dbuilder.complete(sstable.descriptor.filenameFor(Component.DATA));
 +            sstable.bf = FilterFactory.AlwaysPresent;
 +            sstable.setup(false);
 +            return sstable;
 +        }
 +    }
 +
 +    public static SSTableReader open(Descriptor descriptor,
 +                                      Set<Component> components,
 +                                      CFMetaData metadata,
 +                                      IPartitioner partitioner,
 +                                      boolean validate,
 +                                      boolean trackHotness) throws IOException
 +    {
 +        // Minimum components without which we can't do anything
 +        assert components.contains(Component.DATA) : "Data component is missing for sstable " + descriptor;
 +        assert !validate || components.contains(Component.PRIMARY_INDEX) : "Primary index component is missing for sstable " + descriptor;
 +
 +        Map<MetadataType, MetadataComponent> sstableMetadata = descriptor.getMetadataSerializer().deserialize(descriptor,
 +                                                                                                               EnumSet.of(MetadataType.VALIDATION, MetadataType.STATS));
 +        ValidationMetadata validationMetadata = (ValidationMetadata) sstableMetadata.get(MetadataType.VALIDATION);
 +        StatsMetadata statsMetadata = (StatsMetadata) sstableMetadata.get(MetadataType.STATS);
 +
 +        // Check if sstable is created using same partitioner.
 +        // Partitioner can be null, which indicates older version of sstable or no stats available.
 +        // In that case, we skip the check.
 +        String partitionerName = partitioner.getClass().getCanonicalName();
 +        if (validationMetadata != null && !partitionerName.equals(validationMetadata.partitioner))
 +        {
 +            logger.error(String.format("Cannot open %s; partitioner %s does not match system partitioner %s.  Note that the default partitioner starting with Cassandra 1.2 is Murmur3Partitioner, so you will need to edit that to match your old partitioner if upgrading.",
 +                    descriptor, validationMetadata.partitioner, partitionerName));
 +            System.exit(1);
 +        }
 +
 +        logger.debug("Opening {} ({} bytes)", descriptor, new File(descriptor.filenameFor(Component.DATA)).length());
 +        SSTableReader sstable = internalOpen(descriptor, components, metadata, partitioner, System.currentTimeMillis(),
 +                                             statsMetadata, OpenReason.NORMAL);
 +        try
 +        {
 +            // load index and filter
 +            long start = System.nanoTime();
 +            sstable.load(validationMetadata);
 +            logger.trace("INDEX LOAD TIME for {}: {} ms.", descriptor, TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
 +
 +            sstable.setup(trackHotness);
 +            if (validate)
 +                sstable.validate();
 +
 +            if (sstable.getKeyCache() != null)
 +                logger.trace("key cache contains {}/{} keys", sstable.getKeyCache().size(), sstable.getKeyCache().getCapacity());
 +
 +            return sstable;
 +        }
 +        catch (Throwable t)
 +        {
 +            sstable.selfRef().release();
 +            throw t;
 +        }
 +    }
 +
 +    public static void logOpenException(Descriptor descriptor, IOException e)
 +    {
 +        if (e instanceof FileNotFoundException)
 +            logger.error("Missing sstable component in {}; skipped because of {}", descriptor, e.getMessage());
 +        else
 +            logger.error("Corrupt sstable {}; skipped", descriptor, e);
 +    }
 +
 +    public static Collection<SSTableReader> openAll(Set<Map.Entry<Descriptor, Set<Component>>> entries,
 +                                                    final CFMetaData metadata,
 +                                                    final IPartitioner partitioner)
 +    {
 +        final Collection<SSTableReader> sstables = new LinkedBlockingQueue<>();
 +
 +        ExecutorService executor = DebuggableThreadPoolExecutor.createWithFixedPoolSize("SSTableBatchOpen", FBUtilities.getAvailableProcessors());
 +        for (final Map.Entry<Descriptor, Set<Component>> entry : entries)
 +        {
 +            Runnable runnable = new Runnable()
 +            {
 +                public void run()
 +                {
 +                    SSTableReader sstable;
 +                    try
 +                    {
 +                        sstable = open(entry.getKey(), entry.getValue(), metadata, partitioner);
 +                    }
 +                    catch (CorruptSSTableException ex)
 +                    {
 +                        FileUtils.handleCorruptSSTable(ex);
 +                        logger.error("Corrupt sstable {}; skipping table", entry, ex);
 +                        return;
 +                    }
 +                    catch (FSError ex)
 +                    {
 +                        FileUtils.handleFSError(ex);
 +                        logger.error("Cannot read sstable {}; file system error, skipping table", entry, ex);
 +                        return;
 +                    }
 +                    catch (IOException ex)
 +                    {
 +                        logger.error("Cannot read sstable {}; other IO error, skipping table", entry, ex);
 +                        return;
 +                    }
 +                    sstables.add(sstable);
 +                }
 +            };
 +            executor.submit(runnable);
 +        }
 +
 +        executor.shutdown();
 +        try
 +        {
 +            executor.awaitTermination(7, TimeUnit.DAYS);
 +        }
 +        catch (InterruptedException e)
 +        {
 +            throw new AssertionError(e);
 +        }
 +
 +        return sstables;
 +
 +    }
 +
 +    /**
 +     * Open a RowIndexedReader which already has its state initialized (by SSTableWriter).
 +     */
 +    public static SSTableReader internalOpen(Descriptor desc,
 +                                      Set<Component> components,
 +                                      CFMetaData metadata,
 +                                      IPartitioner partitioner,
 +                                      SegmentedFile ifile,
 +                                      SegmentedFile dfile,
 +                                      IndexSummary isummary,
 +                                      IFilter bf,
 +                                      long maxDataAge,
 +                                      StatsMetadata sstableMetadata,
 +                                      OpenReason openReason)
 +    {
 +        assert desc != null && partitioner != null && ifile != null && dfile != null && isummary != null && bf != null && sstableMetadata != null;
 +
 +        SSTableReader reader = internalOpen(desc, components, metadata, partitioner, maxDataAge, sstableMetadata, openReason);
 +
 +        reader.bf = bf;
 +        reader.ifile = ifile;
 +        reader.dfile = dfile;
 +        reader.indexSummary = isummary;
 +        reader.setup(true);
 +
 +        return reader;
 +    }
 +
 +
 +    private static SSTableReader internalOpen(final Descriptor descriptor,
 +                                            Set<Component> components,
 +                                            CFMetaData metadata,
 +                                            IPartitioner partitioner,
 +                                            Long maxDataAge,
 +                                            StatsMetadata sstableMetadata,
 +                                            OpenReason openReason)
 +    {
 +        Factory readerFactory = descriptor.getFormat().getReaderFactory();
 +
 +        return readerFactory.open(descriptor, components, metadata, partitioner, maxDataAge, sstableMetadata, openReason);
 +    }
 +
 +    protected SSTableReader(final Descriptor desc,
 +                            Set<Component> components,
 +                            CFMetaData metadata,
 +                            IPartitioner partitioner,
 +                            long maxDataAge,
 +                            StatsMetadata sstableMetadata,
 +                            OpenReason openReason)
 +    {
 +        super(desc, components, metadata, partitioner);
 +        this.sstableMetadata = sstableMetadata;
 +        this.maxDataAge = maxDataAge;
 +        this.openReason = openReason;
 +        this.rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata);
 +    }
 +
 +    public static long getTotalBytes(Iterable<SSTableReader> sstables)
 +    {
 +        long sum = 0;
 +        for (SSTableReader sstable : sstables)
 +            sum += sstable.onDiskLength();
 +        return sum;
 +    }
 +
 +    public static long getTotalUncompressedBytes(Iterable<SSTableReader> sstables)
 +    {
 +        long sum = 0;
 +        for (SSTableReader sstable : sstables)
 +            sum += sstable.uncompressedLength();
 +
 +        return sum;
 +    }
 +
 +    public boolean equals(Object that)
 +    {
 +        return that instanceof SSTableReader && ((SSTableReader) that).descriptor.equals(this.descriptor);
 +    }
 +
 +    public int hashCode()
 +    {
 +        return this.descriptor.hashCode();
 +    }
 +
 +    public String getFilename()
 +    {
 +        return dfile.path();
 +    }
 +
 +    public void setupKeyCache()
 +    {
 +        // under normal operation we can do this at any time, but SSTR is also used outside C* proper,
 +        // e.g. by BulkLoader, which does not initialize the cache.  As a kludge, we set up the cache
 +        // here when we know we're being wired into the rest of the server infrastructure.
 +        keyCache = CacheService.instance.keyCache;
 +    }
 +
 +    private void load(ValidationMetadata validation) throws IOException
 +    {
 +        if (metadata.getBloomFilterFpChance() == 1.0)
 +        {
 +            // bf is disabled.
 +            load(false, true);
 +            bf = FilterFactory.AlwaysPresent;
 +        }
 +        else if (!components.contains(Component.PRIMARY_INDEX))
 +        {
 +            // avoid any reading of the missing primary index component.
 +            // this should only happen during StandaloneScrubber
 +            load(false, false);
 +        }
 +        else if (!components.contains(Component.FILTER) || validation == null)
 +        {
 +            // bf is enabled, but filter component is missing.
 +            load(true, true);
 +        }
 +        else if (validation.bloomFilterFPChance != metadata.getBloomFilterFpChance())
 +        {
 +            // bf fp chance in sstable metadata and it has changed since compaction.
 +            load(true, true);
 +        }
 +        else
 +        {
 +            // bf is enabled and fp chance matches the currently configured value.
 +            load(false, true);
 +            loadBloomFilter();
 +        }
 +    }
 +
 +    /**
 +     * Load bloom filter from Filter.db file.
 +     *
 +     * @throws IOException
 +     */
 +    private void loadBloomFilter() throws IOException
 +    {
 +        try (DataInputStream stream = new DataInputStream(new BufferedInputStream(new FileInputStream(descriptor.filenameFor(Component.FILTER)))))
 +        {
 +            bf = FilterFactory.deserialize(stream, true);
 +        }
 +    }
 +
 +    /**
 +     * Loads ifile, dfile and indexSummary, and optionally recreates the bloom filter.
 +     * @param saveSummaryIfCreated for bulk loading purposes, if the summary was absent and needed to be built, you can
 +     *                             avoid persisting it to disk by setting this to false
 +     */
 +    private void load(boolean recreateBloomFilter, boolean saveSummaryIfCreated) throws IOException
 +    {
 +        try(SegmentedFile.Builder ibuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode(), false);
 +            SegmentedFile.Builder dbuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode(), compression))
 +        {
 +            boolean summaryLoaded = loadSummary(ibuilder, dbuilder);
 +            boolean builtSummary = false;
 +            if (recreateBloomFilter || !summaryLoaded)
 +            {
 +                buildSummary(recreateBloomFilter, ibuilder, dbuilder, summaryLoaded, Downsampling.BASE_SAMPLING_LEVEL);
 +                builtSummary = true;
 +            }
 +
 +            if (components.contains(Component.PRIMARY_INDEX))
 +                ifile = ibuilder.complete(descriptor.filenameFor(Component.PRIMARY_INDEX));
 +
 +            dfile = dbuilder.complete(descriptor.filenameFor(Component.DATA));
 +
 +            // Check for an index summary that was downsampled even though the serialization format doesn't support
 +            // that.  If it was downsampled, rebuild it.  See CASSANDRA-8993 for details.
 +        if (!descriptor.version.hasSamplingLevel() && !builtSummary && !validateSummarySamplingLevel() && ifile != null)
 +            {
 +                indexSummary.close();
 +                ifile.close();
 +                dfile.close();
 +
 +                logger.info("Detected erroneously downsampled index summary; will rebuild summary at full sampling");
 +                FileUtils.deleteWithConfirm(new File(descriptor.filenameFor(Component.SUMMARY)));
 +
 +                try(SegmentedFile.Builder ibuilderRebuild = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode(), false);
 +                    SegmentedFile.Builder dbuilderRebuild = SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode(), compression))
 +                {
 +                    buildSummary(false, ibuilderRebuild, dbuilderRebuild, false, Downsampling.BASE_SAMPLING_LEVEL);
 +                    ifile = ibuilderRebuild.complete(descriptor.filenameFor(Component.PRIMARY_INDEX));
 +                    dfile = dbuilderRebuild.complete(descriptor.filenameFor(Component.DATA));
 +                    saveSummary(ibuilderRebuild, dbuilderRebuild);
 +                }
 +            }
 +            else if (saveSummaryIfCreated && builtSummary)
 +            {
 +                saveSummary(ibuilder, dbuilder);
 +            }
 +        }
 +        catch (Throwable t)
 +        { // Because the tidier has not been set-up yet in SSTableReader.open(), we must release the files in case of error
 +            if (ifile != null)
 +            {
 +                ifile.close();
 +                ifile = null;
 +            }
 +
 +            if (dfile != null)
 +            {
 +                dfile.close();
 +                dfile = null;
 +            }
 +
 +            if (indexSummary != null)
 +            {
 +                indexSummary.close();
 +                indexSummary = null;
 +            }
 +
 +            throw t;
 +        }
 +    }
 +
 +    /**
 +     * Build index summary(and optionally bloom filter) by reading through Index.db file.
 +     *
 +     * @param recreateBloomFilter true if recreate bloom filter
 +     * @param ibuilder
 +     * @param dbuilder
 +     * @param summaryLoaded true if index summary is already loaded and not need to build again
 +     * @throws IOException
 +     */
 +    private void buildSummary(boolean recreateBloomFilter, SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, boolean summaryLoaded, int samplingLevel) throws IOException
 +    {
 +         if (!components.contains(Component.PRIMARY_INDEX))
 +             return;
 +
 +        // we read the positions in a BRAF so we don't have to worry about an entry spanning a mmap boundary.
 +        try (RandomAccessReader primaryIndex = RandomAccessReader.open(new File(descriptor.filenameFor(Component.PRIMARY_INDEX))))
 +        {
 +            long indexSize = primaryIndex.length();
 +            long histogramCount = sstableMetadata.estimatedRowSize.count();
 +            long estimatedKeys = histogramCount > 0 && !sstableMetadata.estimatedRowSize.isOverflowed()
 +                    ? histogramCount
 +                    : estimateRowsFromIndex(primaryIndex); // statistics is supposed to be optional
 +
 +            if (recreateBloomFilter)
 +                bf = FilterFactory.getFilter(estimatedKeys, metadata.getBloomFilterFpChance(), true);
 +
 +            try (IndexSummaryBuilder summaryBuilder = summaryLoaded ? null : new IndexSummaryBuilder(estimatedKeys, metadata.getMinIndexInterval(), samplingLevel))
 +            {
 +                long indexPosition;
 +                RowIndexEntry.IndexSerializer rowIndexSerializer = descriptor.getFormat().getIndexSerializer(metadata);
 +
 +                while ((indexPosition = primaryIndex.getFilePointer()) != indexSize)
 +                {
 +                    ByteBuffer key = ByteBufferUtil.readWithShortLength(primaryIndex);
 +                    RowIndexEntry indexEntry = rowIndexSerializer.deserialize(primaryIndex, descriptor.version);
 +                    DecoratedKey decoratedKey = partitioner.decorateKey(key);
 +                    if (first == null)
 +                        first = decoratedKey;
 +                    last = decoratedKey;
 +
 +                    if (recreateBloomFilter)
 +                        bf.add(decoratedKey);
 +
 +                    // if summary was already read from disk we don't want to re-populate it using primary index
 +                    if (!summaryLoaded)
 +                    {
 +                        summaryBuilder.maybeAddEntry(decoratedKey, indexPosition);
 +                        ibuilder.addPotentialBoundary(indexPosition);
 +                        dbuilder.addPotentialBoundary(indexEntry.position);
 +                    }
 +                }
 +
 +                if (!summaryLoaded)
 +                    indexSummary = summaryBuilder.build(partitioner);
 +            }
 +        }
 +
 +        first = getMinimalKey(first);
 +        last = getMinimalKey(last);
 +    }
 +
 +    /**
 +     * Load index summary from Summary.db file if it exists.
 +     *
 +     * if loaded index summary has different index interval from current value stored in schema,
 +     * then Summary.db file will be deleted and this returns false to rebuild summary.
 +     *
 +     * @param ibuilder
 +     * @param dbuilder
 +     * @return true if index summary is loaded successfully from Summary.db file.
 +     */
 +    @SuppressWarnings("resource")
 +    public boolean loadSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
 +    {
 +        File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
 +        if (!summariesFile.exists())
 +            return false;
 +
 +        DataInputStream iStream = null;
 +        try
 +        {
 +            iStream = new DataInputStream(new FileInputStream(summariesFile));
 +            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);
 +            dbuilder.deserializeBounds(iStream);
++
++            boolean checkForRepair = true;
++            try
++            {
++                int v = iStream.readInt();
++                // check for our magic number, indicating this summary has been sampled correctly
++                checkForRepair = v != ACCURATE_BOUNDARIES_MAGIC_NUMBER;
++            }
++            catch (Throwable t) {}
++
++            // fix CASSANDRA-10357 on-the-fly
++            if (checkForRepair && MmappedSegmentedFile.maybeRepair(metadata, descriptor, indexSummary, ibuilder, dbuilder))
++                saveSummary(ibuilder, dbuilder);
 +        }
 +        catch (IOException e)
 +        {
 +            if (indexSummary != null)
 +                indexSummary.close();
 +            logger.trace("Cannot deserialize SSTable Summary File {}: {}", summariesFile.getPath(), e.getMessage());
 +            // corrupted; delete it and fall back to creating a new summary
 +            FileUtils.closeQuietly(iStream);
 +            // delete it and fall back to creating a new summary
 +            FileUtils.deleteWithConfirm(summariesFile);
 +            return false;
 +        }
 +        finally
 +        {
 +            FileUtils.closeQuietly(iStream);
 +        }
 +
 +        return true;
 +    }
 +
 +    /**
 +     * Validates that an index summary has full sampling, as expected when the serialization format does not support
 +     * persisting the sampling level.
 +     * @return true if the summary has full sampling, false otherwise
 +     */
 +    private boolean validateSummarySamplingLevel()
 +    {
 +        // We need to check index summary entries against the index to verify that none of them were dropped due to
 +        // downsampling.  Downsampling can drop any of the first BASE_SAMPLING_LEVEL entries (repeating that drop pattern
 +        // for the remainder of the summary).  Unfortunately, the first entry to be dropped is the entry at
 +        // index (BASE_SAMPLING_LEVEL - 1), so we need to check a full set of BASE_SAMPLING_LEVEL entries.
 +        if (ifile == null)
 +            return false;
 +
 +        Iterator<FileDataInput> segments = ifile.iterator(0);
 +        int i = 0;
 +        int summaryEntriesChecked = 0;
 +        int expectedIndexInterval = getMinIndexInterval();
 +        while (segments.hasNext())
 +        {
 +            String path = null;
 +            try (FileDataInput in = segments.next())
 +            {
 +                path = in.getPath();
 +                while (!in.isEOF())
 +                {
 +                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
 +                    if (i % expectedIndexInterval == 0)
 +                    {
 +                        ByteBuffer summaryKey = ByteBuffer.wrap(indexSummary.getKey(i / expectedIndexInterval));
 +                        if (!summaryKey.equals(indexKey))
 +                            return false;
 +                        summaryEntriesChecked++;
 +
 +                        if (summaryEntriesChecked == Downsampling.BASE_SAMPLING_LEVEL)
 +                            return true;
 +                    }
 +                    RowIndexEntry.Serializer.skip(in);
 +                    i++;
 +                }
 +            }
 +            catch (IOException e)
 +            {
 +                markSuspect();
 +                throw new CorruptSSTableException(e, path);
 +            }
 +        }
 +
 +        return true;
 +    }
 +
 +    /**
 +     * Save index summary to Summary.db file.
 +     *
 +     * @param ibuilder
 +     * @param dbuilder
 +     */
 +
 +    public void saveSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
 +    {
 +        saveSummary(this.descriptor, this.first, this.last, ibuilder, dbuilder, indexSummary);
 +    }
 +
 +    private void saveSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, IndexSummary newSummary)
 +    {
 +        saveSummary(this.descriptor, this.first, this.last, ibuilder, dbuilder, newSummary);
 +    }
 +    /**
 +     * Save index summary to Summary.db file.
 +     */
 +    public static void saveSummary(Descriptor descriptor, DecoratedKey first, DecoratedKey last,
 +                                   SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, IndexSummary summary)
 +    {
 +        File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
 +        if (summariesFile.exists())
 +            FileUtils.deleteWithConfirm(summariesFile);
 +
 +        try (DataOutputStreamPlus oStream = new BufferedDataOutputStreamPlus(new FileOutputStream(summariesFile));)
 +        {
 +            IndexSummary.serializer.serialize(summary, oStream, descriptor.version.hasSamplingLevel());
 +            ByteBufferUtil.writeWithLength(first.getKey(), oStream);
 +            ByteBufferUtil.writeWithLength(last.getKey(), oStream);
 +            ibuilder.serializeBounds(oStream);
 +            dbuilder.serializeBounds(oStream);
++            // write a magic number, to indicate this summary has been sampled correctly
++            oStream.writeInt(ACCURATE_BOUNDARIES_MAGIC_NUMBER);
 +        }
 +        catch (IOException e)
 +        {
 +            logger.trace("Cannot save SSTable Summary: ", e);
 +
 +            // corrupted hence delete it and let it load it now.
 +            if (summariesFile.exists())
 +                FileUtils.deleteWithConfirm(summariesFile);
 +        }
 +    }
 +
 +    public void setReplaced()
 +    {
 +        synchronized (tidy.global)
 +        {
 +            assert !tidy.isReplaced;
 +            tidy.isReplaced = true;
 +        }
 +    }
 +
 +    public boolean isReplaced()
 +    {
 +        synchronized (tidy.global)
 +        {
 +            return tidy.isReplaced;
 +        }
 +    }
 +
 +    // runOnClose must NOT be an anonymous or non-static inner class, nor must it retain a reference chain to this reader
 +    public void runOnClose(final Runnable runOnClose)
 +    {
 +        synchronized (tidy.global)
 +        {
 +            final Runnable existing = tidy.runOnClose;
 +            tidy.runOnClose = AndThen.get(existing, runOnClose);
 +        }
 +    }
 +
 +    private static class AndThen implements Runnable
 +    {
 +        final Runnable runFirst;
 +        final Runnable runSecond;
 +
 +        private AndThen(Runnable runFirst, Runnable runSecond)
 +        {
 +            this.runFirst = runFirst;
 +            this.runSecond = runSecond;
 +        }
 +
 +        public void run()
 +        {
 +            runFirst.run();
 +            runSecond.run();
 +        }
 +
 +        static Runnable get(Runnable runFirst, Runnable runSecond)
 +        {
 +            if (runFirst == null)
 +                return runSecond;
 +            return new AndThen(runFirst, runSecond);
 +        }
 +    }
 +
 +    /**
 +     * Clone this reader with the provided start and open reason, and set the clone as replacement.
 +     *
 +     * @param newFirst the first key for the replacement (which can be different from the original due to the pre-emptive
 +     * opening of compaction results).
 +     * @param reason the {@code OpenReason} for the replacement.
 +     *
 +     * @return the cloned reader. That reader is set as a replacement by the method.
 +     */
 +    private SSTableReader cloneAndReplace(DecoratedKey newFirst, OpenReason reason)
 +    {
 +        return cloneAndReplace(newFirst, reason, indexSummary.sharedCopy());
 +    }
 +
 +    /**
 +     * Clone this reader with the new values and set the clone as replacement.
 +     *
 +     * @param newFirst the first key for the replacement (which can be different from the original due to the pre-emptive
 +     * opening of compaction results).
 +     * @param reason the {@code OpenReason} for the replacement.
 +     * @param newSummary the index summary for the replacement.
 +     *
 +     * @return the cloned reader. That reader is set as a replacement by the method.
 +     */
 +    private SSTableReader cloneAndReplace(DecoratedKey newFirst, OpenReason reason, IndexSummary newSummary)
 +    {
 +        SSTableReader replacement = internalOpen(descriptor,
 +                                                 components,
 +                                                 metadata,
 +                                                 partitioner,
 +                                                 ifile != null ? ifile.sharedCopy() : null,
 +                                                 dfile.sharedCopy(),
 +                                                 newSummary,
 +                                                 bf.sharedCopy(),
 +                                                 maxDataAge,
 +                                                 sstableMetadata,
 +                                                 reason);
 +        replacement.first = newFirst;
 +        replacement.last = last;
 +        replacement.isSuspect.set(isSuspect.get());
 +        return replacement;
 +    }
 +
 +    // runOnClose must NOT be an anonymous or non-static inner class, nor must it retain a reference chain to this reader
 +    public SSTableReader cloneWithNewStart(DecoratedKey newStart, final Runnable runOnClose)
 +    {
 +        synchronized (tidy.global)
 +        {
 +            assert openReason != OpenReason.EARLY;
 +            // TODO: merge with caller's firstKeyBeyond() work,to save time
 +            if (newStart.compareTo(first) > 0)
 +            {
 +                final long dataStart = getPosition(newStart, Operator.EQ).position;
 +                final long indexStart = getIndexScanPosition(newStart);
 +                this.tidy.runOnClose = new DropPageCache(dfile, dataStart, ifile, indexStart, runOnClose);
 +            }
 +
 +            return cloneAndReplace(newStart, OpenReason.MOVED_START);
 +        }
 +    }
 +
 +    private static class DropPageCache implements Runnable
 +    {
 +        final SegmentedFile dfile;
 +        final long dfilePosition;
 +        final SegmentedFile ifile;
 +        final long ifilePosition;
 +        final Runnable andThen;
 +
 +        private DropPageCache(SegmentedFile dfile, long dfilePosition, SegmentedFile ifile, long ifilePosition, Runnable andThen)
 +        {
 +            this.dfile = dfile;
 +            this.dfilePosition = dfilePosition;
 +            this.ifile = ifile;
 +            this.ifilePosition = ifilePosition;
 +            this.andThen = andThen;
 +        }
 +
 +        public void run()
 +        {
 +            dfile.dropPageCache(dfilePosition);
 +
 +            if (ifile != null)
 +                ifile.dropPageCache(ifilePosition);
 +            andThen.run();
 +        }
 +    }
 +
 +    /**
 +     * Returns a new SSTableReader with the same properties as this SSTableReader except that a new IndexSummary will
 +     * be built at the target samplingLevel.  This (original) SSTableReader instance will be marked as replaced, have
 +     * its DeletingTask removed, and have its periodic read-meter sync task cancelled.
 +     * @param samplingLevel the desired sampling level for the index summary on the new SSTableReader
 +     * @return a new SSTableReader
 +     * @throws IOException
 +     */
 +    @SuppressWarnings("resource")
 +    public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
 +    {
 +        assert descriptor.version.hasSamplingLevel();
 +
 +        synchronized (tidy.global)
 +        {
 +            assert openReason != OpenReason.EARLY;
 +
 +            int minIndexInterval = metadata.getMinIndexInterval();
 +            int maxIndexInterval = metadata.getMaxIndexInterval();
 +            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
 +
 +            IndexSummary newSummary;
 +            long oldSize = bytesOnDisk();
 +
 +            // 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 = 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);
 +
 +                try(SegmentedFile.Builder ibuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode(), false);
 +                    SegmentedFile.Builder dbuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode(), compression))
 +                {
 +                    saveSummary(ibuilder, dbuilder, newSummary);
 +                }
 +            }
 +            else
 +            {
 +                throw new AssertionError("Attempted to clone SSTableReader with the same index summary sampling level and " +
 +                        "no adjustments to min/max_index_interval");
 +            }
 +
 +            long newSize = bytesOnDisk();
 +            StorageMetrics.load.inc(newSize - oldSize);
 +            parent.metric.liveDiskSpaceUsed.inc(newSize - oldSize);
 +
 +            return cloneAndReplace(first, OpenReason.METADATA_CHANGE, newSummary);
 +        }
 +    }
 +
 +    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();
 +            try (IndexSummaryBuilder summaryBuilder = new IndexSummaryBuilder(estimatedKeys(), metadata.getMinIndexInterval(), newSamplingLevel))
 +            {
 +                long indexPosition;
 +                while ((indexPosition = primaryIndex.getFilePointer()) != indexSize)
 +                {
 +                    summaryBuilder.maybeAddEntry(partitioner.decorateKey(ByteBufferUtil.readWithShortLength(primaryIndex)), indexPosition);
 +                    RowIndexEntry.Serializer.skip(primaryIndex);
 +                }
 +
 +                return summaryBuilder.build(partitioner);
 +            }
 +        }
 +        finally
 +        {
 +            FileUtils.closeQuietly(primaryIndex);
 +        }
 +    }
 +
 +    public RestorableMeter getReadMeter()
 +    {
 +        return readMeter;
 +    }
 +
 +    public int getIndexSummarySamplingLevel()
 +    {
 +        return indexSummary.getSamplingLevel();
 +    }
 +
 +    public long getIndexSummaryOffHeapSize()
 +    {
 +        return indexSummary.getOffHeapSize();
 +    }
 +
 +    public int getMinIndexInterval()
 +    {
 +        return indexSummary.getMinIndexInterval();
 +    }
 +
 +    public double getEffectiveIndexInterval()
 +    {
 +        return indexSummary.getEffectiveIndexInterval();
 +    }
 +
 +    public void releaseSummary()
 +    {
 +        tidy.releaseSummary();
 +        indexSummary = null;
 +    }
 +
 +    private void validate()
 +    {
 +        if (this.first.compareTo(this.last) > 0)
 +        {
 +            selfRef().release();
 +            throw new IllegalStateException(String.format("SSTable first key %s > last key %s", this.first, this.last));
 +        }
 +    }
 +
 +    /**
 +     * Gets the position in the index file to start scanning to find the given key (at most indexInterval keys away,
 +     * modulo downsampling of the index summary). Always returns a value >= 0
 +     */
 +    public long getIndexScanPosition(RowPosition key)
 +    {
 +        if (openReason == OpenReason.MOVED_START && key.compareTo(first) < 0)
 +            key = first;
 +
 +        return getIndexScanPositionFromBinarySearchResult(indexSummary.binarySearch(key), indexSummary);
 +    }
 +
 +    @VisibleForTesting
 +    public static long getIndexScanPositionFromBinarySearchResult(int binarySearchResult, IndexSummary referencedIndexSummary)
 +    {
 +        if (binarySearchResult == -1)
 +            return 0;
 +        else
 +            return referencedIndexSummary.getPosition(getIndexSummaryIndexFromBinarySearchResult(binarySearchResult));
 +    }
 +
 +    public static int getIndexSummaryIndexFromBinarySearchResult(int binarySearchResult)
 +    {
 +        if (binarySearchResult < 0)
 +        {
 +            // binary search gives us the first index _greater_ than the key searched for,
 +            // i.e., its insertion position
 +            int greaterThan = (binarySearchResult + 1) * -1;
 +            if (greaterThan == 0)
 +                return -1;
 +            return greaterThan - 1;
 +        }
 +        else
 +        {
 +            return binarySearchResult;
 +        }
 +    }
 +
 +    /**
 +     * Returns the compression metadata for this sstable.
 +     * @throws IllegalStateException if the sstable is not compressed
 +     */
 +    public CompressionMetadata getCompressionMetadata()
 +    {
 +        if (!compression)
 +            throw new IllegalStateException(this + " is not compressed");
 +
 +        CompressionMetadata cmd = ((ICompressedFile) dfile).getMetadata();
 +
 +        //We need the parent cf metadata
 +        String cfName = metadata.isSecondaryIndex() ? metadata.getParentColumnFamilyName() : metadata.cfName;
 +        cmd.parameters.setLiveMetadata(Schema.instance.getCFMetaData(metadata.ksName, cfName));
 +
 +        return cmd;
 +    }
 +
 +    /**
 +     * Returns the amount of memory in bytes used off heap by the compression meta-data.
 +     * @return the amount of memory in bytes used off heap by the compression meta-data
 +     */
 +    public long getCompressionMetadataOffHeapSize()
 +    {
 +        if (!compression)
 +            return 0;
 +
 +        return getCompressionMetadata().offHeapSize();
 +    }
 +
 +    /**
 +     * For testing purposes only.
 +     */
 +    public void forceFilterFailures()
 +    {
 +        bf = FilterFactory.AlwaysPresent;
 +    }
 +
 +    public IFilter getBloomFilter()
 +    {
 +        return bf;
 +    }
 +
 +    public long getBloomFilterSerializedSize()
 +    {
 +        return bf.serializedSize();
 +    }
 +
 +    /**
 +     * Returns the amount of memory in bytes used off heap by the bloom filter.
 +     * @return the amount of memory in bytes used off heap by the bloom filter
 +     */
 +    public long getBloomFilterOffHeapSize()
 +    {
 +        return bf.offHeapSize();
 +    }
 +
 +    /**
 +     * @return An estimate of the number of keys in this SSTable based on the index summary.
 +     */
 +    public long estimatedKeys()
 +    {
 +        return indexSummary.getEstimatedKeyCount();
 +    }
 +
 +    /**
 +     * @param ranges
 +     * @return An estimate of the number of keys for given ranges in this SSTable.
 +     */
 +    public long estimatedKeysForRanges(Collection<Range<Token>> ranges)
 +    {
 +        long sampleKeyCount = 0;
 +        List<Pair<Integer, Integer>> sampleIndexes = getSampleIndexesForRanges(indexSummary, ranges);
 +        for (Pair<Integer, Integer> sampleIndexRange : sampleIndexes)
 +            sampleKeyCount += (sampleIndexRange.right - sampleIndexRange.left + 1);
 +
 +        // adjust for the current sampling level: (BSL / SL) * index_interval_at_full_sampling
 +        long estimatedKeys = sampleKeyCount * ((long) Downsampling.BASE_SAMPLING_LEVEL * indexSummary.getMinIndexInterval()) / indexSummary.getSamplingLevel();
 +        return Math.max(1, estimatedKeys);
 +    }
 +
 +    /**
 +     * Returns the number of entries in the IndexSummary.  At full sampling, this is approximately 1/INDEX_INTERVALth of
 +     * the keys in this SSTable.
 +     */
 +    public int getIndexSummarySize()
 +    {
 +        return indexSummary.size();
 +    }
 +
 +    /**
 +     * Returns the approximate number of entries the IndexSummary would contain if it were at full sampling.
 +     */
 +    public int getMaxIndexSummarySize()
 +    {
 +        return indexSummary.getMaxNumberOfEntries();
 +    }
 +
 +    /**
 +     * Returns the key for the index summary entry at `index`.
 +     */
 +    public byte[] getIndexSummaryKey(int index)
 +    {
 +        return indexSummary.getKey(index);
 +    }
 +
 +    private static List<Pair<Integer,Integer>> getSampleIndexesForRanges(IndexSummary summary, Collection<Range<Token>> ranges)
 +    {
 +        // use the index to determine a minimal section for each range
 +        List<Pair<Integer,Integer>> positions = new ArrayList<>();
 +
 +        for (Range<Token> range : Range.normalize(ranges))
 +        {
 +            RowPosition leftPosition = range.left.maxKeyBound();
 +            RowPosition rightPosition = range.right.maxKeyBound();
 +
 +            int left = summary.binarySearch(leftPosition);
 +            if (left < 0)
 +                left = (left + 1) * -1;
 +            else
 +                // left range are start exclusive
 +                left = left + 1;
 +            if (left == summary.size())
 +                // left is past the end of the sampling
 +                continue;
 +
 +            int right = Range.isWrapAround(range.left, range.right)
 +                    ? summary.size() - 1
 +                    : summary.binarySearch(rightPosition);
 +            if (right < 0)
 +            {
 +                // range are end inclusive so we use the previous index from what binarySearch give us
 +                // since that will be the last index we will return
 +                right = (right + 1) * -1;
 +                if (right == 0)
 +                    // Means the first key is already stricly greater that the right bound
 +                    continue;
 +                right--;
 +            }
 +
 +            if (left > right)
 +                // empty range
 +                continue;
 +            positions.add(Pair.create(left, right));
 +        }
 +        return positions;
 +    }
 +
 +    public Iterable<DecoratedKey> getKeySamples(final Range<Token> range)
 +    {
 +        final List<Pair<Integer, Integer>> indexRanges = getSampleIndexesForRanges(indexSummary, Collections.singletonList(range));
 +
 +        if (indexRanges.isEmpty())
 +            return Collections.emptyList();
 +
 +        return new Iterable<DecoratedKey>()
 +        {
 +            public Iterator<DecoratedKey> iterator()
 +            {
 +                return new Iterator<DecoratedKey>()
 +                {
 +                    private Iterator<Pair<Integer, Integer>> rangeIter = indexRanges.iterator();
 +                    private Pair<Integer, Integer> current;
 +                    private int idx;
 +
 +                    public boolean hasNext()
 +                    {
 +                        if (current == null || idx > current.right)
 +                        {
 +                            if (rangeIter.hasNext())
 +                            {
 +                                current = rangeIter.next();
 +                                idx = current.left;
 +                                return true;
 +                            }
 +                            return false;
 +                        }
 +
 +                        return true;
 +                    }
 +
 +                    public DecoratedKey next()
 +                    {
 +                        byte[] bytes = indexSummary.getKey(idx++);
 +                        return partitioner.decorateKey(ByteBuffer.wrap(bytes));
 +                    }
 +
 +                    public void remove()
 +                    {
 +                        throw new UnsupportedOperationException();
 +                    }
 +                };
 +            }
 +        };
 +    }
 +
 +    /**
 +     * Determine the minimal set of sections that can be extracted from this SSTable to cover the given ranges.
 +     * @return A sorted list of (offset,end) pairs that cover the given ranges in the datafile for this SSTable.
 +     */
 +    public List<Pair<Long,Long>> getPositionsForRanges(Collection<Range<Token>> ranges)
 +    {
 +        // use the index to determine a minimal section for each range
 +        List<Pair<Long,Long>> positions = new ArrayList<>();
 +        for (Range<Token> range : Range.normalize(ranges))
 +        {
 +            assert !range.isWrapAround() || range.right.isMinimum();
 +            // truncate the range so it at most covers the sstable
 +            AbstractBounds<RowPosition> bounds = Range.makeRowRange(range);
 +            RowPosition leftBound = bounds.left.compareTo(first) > 0 ? bounds.left : first.getToken().minKeyBound();
 +            RowPosition rightBound = bounds.right.isMinimum() ? last.getToken().maxKeyBound() : bounds.right;
 +
 +            if (leftBound.compareTo(last) > 0 || rightBound.compareTo(first) < 0)
 +                continue;
 +
 +            long left = getPosition(leftBound, Operator.GT).position;
 +            long right = (rightBound.compareTo(last) > 0)
 +                         ? uncompressedLength()
 +                         : getPosition(rightBound, Operator.GT).position;
 +
 +            if (left == right)
 +                // empty range
 +                continue;
 +
 +            assert left < right : String.format("Range=%s openReason=%s first=%s last=%s left=%d right=%d", range, openReason, first, last, left, right);
 +            positions.add(Pair.create(left, right));
 +        }
 +        return positions;
 +    }
 +
 +    public KeyCacheKey getCacheKey(DecoratedKey key)
 +    {
 +        return new KeyCacheKey(metadata.ksAndCFName, descriptor, key.getKey());
 +    }
 +
 +    public void cacheKey(DecoratedKey key, RowIndexEntry info)
 +    {
 +        CachingOptions caching = metadata.getCaching();
 +
 +        if (!caching.keyCache.isEnabled()
 +                || keyCache == null
 +                || keyCache.getCapacity() == 0)
 +        {
 +            return;
 +        }
 +
 +        KeyCacheKey cacheKey = new KeyCacheKey(metadata.ksAndCFName, descriptor, key.getKey());
 +        logger.trace("Adding cache entry for {} -> {}", cacheKey, info);
 +        keyCache.put(cacheKey, info);
 +    }
 +
 +    public RowIndexEntry getCachedPosition(DecoratedKey key, boolean updateStats)
 +    {
 +        return getCachedPosition(new KeyCacheKey(metadata.ksAndCFName, descriptor, key.getKey()), updateStats);
 +    }
 +
 +    protected RowIndexEntry getCachedPosition(KeyCacheKey unifiedKey, boolean updateStats)
 +    {
 +        if (keyCache != null && keyCache.getCapacity() > 0 && metadata.getCaching().keyCache.isEnabled()) {
 +            if (updateStats)
 +            {
 +                RowIndexEntry cachedEntry = keyCache.get(unifiedKey);
 +                keyCacheRequest.incrementAndGet();
 +                if (cachedEntry != null)
 +                {
 +                    keyCacheHit.incrementAndGet();
 +                    bloomFilterTracker.addTruePositive();
 +                }
 +                return cachedEntry;
 +            }
 +            else
 +            {
 +                return keyCache.getInternal(unifiedKey);
 +            }
 +        }
 +        return null;
 +    }
 +
 +    /**
 +     * Get position updating key cache and stats.
 +     * @see #getPosition(org.apache.cassandra.db.RowPosition, SSTableReader.Operator, boolean)
 +     */
 +    public RowIndexEntry getPosition(RowPosition key, Operator op)
 +    {
 +        return getPosition(key, op, true, false);
 +    }
 +
 +    public RowIndexEntry getPosition(RowPosition key, Operator op, boolean updateCacheAndStats)
 +    {
 +        return getPosition(key, op, updateCacheAndStats, false);
 +    }
 +    /**
 +     * @param key The key to apply as the rhs to the given Operator. A 'fake' key is allowed to
 +     * allow key selection by token bounds but only if op != * EQ
 +     * @param op The Operator defining matching keys: the nearest key to the target matching the operator wins.
 +     * @param updateCacheAndStats true if updating stats and cache
 +     * @return The index entry corresponding to the key, or null if the key is not present
 +     */
 +    protected abstract RowIndexEntry getPosition(RowPosition key, Operator op, boolean updateCacheAndStats, boolean permitMatchPastLast);
 +
 +    //Corresponds to a name column
 +    public abstract OnDiskAtomIterator iterator(DecoratedKey key, SortedSet<CellName> columns);
 +    public abstract OnDiskAtomIterator iterator(FileDataInput file, DecoratedKey key, SortedSet<CellName> columns, RowIndexEntry indexEntry);
 +
 +    //Corresponds to a slice query
 +    public abstract OnDiskAtomIterator iterator(DecoratedKey key, ColumnSlice[] slices, boolean reverse);
 +    public abstract OnDiskAtomIterator iterator(FileDataInput file, DecoratedKey key, ColumnSlice[] slices, boolean reversed, RowIndexEntry indexEntry);
 +
 +    /**
 +     * Finds and returns the first key beyond a given token in this SSTable or null if no such key exists.
 +     */
 +    public DecoratedKey firstKeyBeyond(RowPosition token)
 +    {
 +        if (token.compareTo(first) < 0)
 +            return first;
 +
 +        long sampledPosition = getIndexScanPosition(token);
 +
 +        if (ifile == null)
 +            return null;
 +
 +        Iterator<FileDataInput> segments = ifile.iterator(sampledPosition);
 +        while (segments.hasNext())
 +        {
 +            String path = null;
 +            try (FileDataInput in = segments.next();)
 +            {
 +                path = in.getPath();
 +                while (!in.isEOF())
 +                {
 +                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
 +                    DecoratedKey indexDecoratedKey = partitioner.decorateKey(indexKey);
 +                    if (indexDecoratedKey.compareTo(token) > 0)
 +                        return indexDecoratedKey;
 +
 +                    RowIndexEntry.Serializer.skip(in);
 +                }
 +            }
 +            catch (IOException e)
 +            {
 +                markSuspect();
 +                throw new CorruptSSTableException(e, path);
 +            }
 +        }
 +
 +        return null;
 +    }
 +
 +    /**
 +     * @return The length in bytes of the data for this SSTable. For
 +     * compressed files, this is not the same thing as the on disk size (see
 +     * onDiskLength())
 +     */
 +    public long uncompressedLength()
 +    {
 +        return dfile.length;
 +    }
 +
 +    /**
 +     * @return The length in bytes of the on disk size for this SSTable. For
 +     * compressed files, this is not the same thing as the data length (see
 +     * length())
 +     */
 +    public long onDiskLength()
 +    {
 +        return dfile.onDiskLength;
 +    }
 +
 +    /**
 +     * Mark the sstable as obsolete, i.e., compacted into newer sstables.
 +     *
 +     * When calling this function, the caller must ensure that the SSTableReader is not referenced anywhere
 +     * except for threads holding a reference.
 +     *
 +     * @return true if the this is the first time the file was marked obsolete.  Calling this
 +     * multiple times is usually buggy (see exceptions in Tracker.unmarkCompacting and removeOldSSTablesSize).
 +     */
 +    public boolean markObsolete(Tracker tracker)
 +    {
 +        if (logger.isTraceEnabled())
 +            logger.trace("Marking {} compacted", getFilename());
 +
 +        synchronized (tidy.global)
 +        {
 +            assert !tidy.isReplaced;
 +        }
 +        if (!tidy.global.isCompacted.getAndSet(true))
 +        {
 +            tidy.type.markObsolete(this, tracker);
 +            return true;
 +        }
 +        return false;
 +    }
 +
 +    public boolean isMarkedCompacted()
 +    {
 +        return tidy.global.isCompacted.get();
 +    }
 +
 +    public void markSuspect()
 +    {
 +        if (logger.isTraceEnabled())
 +            logger.trace("Marking {} as a suspect for blacklisting.", getFilename());
 +
 +        isSuspect.getAndSet(true);
 +    }
 +
 +    public boolean isMarkedSuspect()
 +    {
 +        return isSuspect.get();
 +    }
 +
 +
 +    /**
 +     * I/O SSTableScanner
 +     * @return A Scanner for seeking over the rows of the SSTable.
 +     */
 +    public ISSTableScanner getScanner()
 +    {
 +        return getScanner((RateLimiter) null);
 +    }
 +
 +    public ISSTableScanner getScanner(RateLimiter limiter)
 +    {
 +        return getScanner(DataRange.allData(partitioner), limiter);
 +    }
 +
 +    /**
 +     *
 +     * @param dataRange filter to use when reading the columns
 +     * @return A Scanner for seeking over the rows of the SSTable.
 +     */
 +    public ISSTableScanner getScanner(DataRange dataRange)
 +    {
 +        return getScanner(dataRange, null);
 +    }
 +
 +    /**
 +     * Direct I/O SSTableScanner over a defined range of tokens.
 +     *
 +     * @param range the range of keys to cover
 +     * @return A Scanner for seeking over the rows of the SSTable.
 +     */
 +    public ISSTableScanner getScanner(Range<Token> range, RateLimiter limiter)
 +    {
 +        if (range == null)
 +            return getScanner(limiter);
 +        return getScanner(Collections.singletonList(range), limiter);
 +    }
 +
 +    /**
 +     * Direct I/O SSTableScanner over a defined collection of ranges of tokens.
 +     *
 +     * @param ranges the range of keys to cover
 +     * @return A Scanner for seeking over the rows of the SSTable.
 +     */
 +    public abstract ISSTableScanner getScanner(Collection<Range<Token>> ranges, RateLimiter limiter);
 +
 +    /**
 +     *
 +     * @param dataRange filter to use when reading the columns
 +     * @return A Scanner for seeking over the rows of the SSTable.
 +     */
 +    public abstract ISSTableScanner getScanner(DataRange dataRange, RateLimiter limiter);
 +
 +
 +
 +    public FileDataInput getFileDataInput(long position)
 +    {
 +        return dfile.getSegment(position);
 +    }
 +
 +    /**
 +     * Tests if the sstable contains data newer than the given age param (in localhost currentMilli time).
 +     * This works in conjunction with maxDataAge which is an upper bound on the create of data in this sstable.
 +     * @param age The age to compare the maxDataAre of this sstable. Measured in millisec since epoc on this host
 +     * @return True iff this sstable contains data that's newer than the given age parameter.
 +     */
 +    public boolean newSince(long age)
 +    {
 +        return maxDataAge > age;
 +    }
 +
 +    public void createLinks(String snapshotDirectoryPath)
 +    {
 +        for (Component component : components)
 +        {
 +            File sourceFile = new File(descriptor.filenameFor(component));
 +            File targetLink = new File(snapshotDirectoryPath, sourceFile.getName());
 +            FileUtils.createHardLink(sourceFile, targetLink);
 +        }
 +    }
 +
 +    public boolean isRepaired()
 +    {
 +        return sstableMetadata.repairedAt != ActiveRepairService.UNREPAIRED_SSTABLE;
 +    }
 +
 +    /**
 +     * TODO: Move someplace reusable
 +     */
 +    public abstract static class Operator
 +    {
 +        public static final Operator EQ = new Equals();
 +        public static final Operator GE = new GreaterThanOrEqualTo();
 +        public static final Operator GT = new GreaterThan();
 +
 +        /**
 +         * @param comparison The result of a call to compare/compareTo, with the desired field on the rhs.
 +         * @return less than 0 if the operator cannot match forward, 0 if it matches, greater than 0 if it might match forward.
 +         */
 +        public abstract int apply(int comparison);
 +
 +        final static class Equals extends Operator
 +        {
 +            public int apply(int comparison) { return -comparison; }
 +        }
 +
 +        final static class GreaterThanOrEqualTo extends Operator
 +        {
 +            public int apply(int comparison) { return comparison >= 0 ? 0 : 1; }
 +        }
 +
 +        final static class GreaterThan extends Operator
 +        {
 +            public int apply(int comparison) { return comparison > 0 ? 0 : 1; }
 +        }
 +    }
 +
 +    public long getBloomFilterFalsePositiveCount()
 +    {
 +        return bloomFilterTracker.getFalsePositiveCount();
 +    }
 +
 +    public long getRecentBloomFilterFalsePositiveCount()
 +    {
 +        return bloomFilterTracker.getRecentFalsePositiveCount();
 +    }
 +
 +    public long getBloomFilterTruePositiveCount()
 +    {
 +        return bloomFilterTracker.getTruePositiveCount();
 +    }
 +
 +    public long getRecentBloomFilterTruePositiveCount()
 +    {
 +        return bloomFilterTracker.getRecentTruePositiveCount();
 +    }
 +
 +    public InstrumentingCache<KeyCacheKey, RowIndexEntry> getKeyCache()
 +    {
 +        return keyCache;
 +    }
 +
 +    public EstimatedHistogram getEstimatedRowSize()
 +    {
 +        return sstableMetadata.estimatedRowSize;
 +    }
 +
 +    public EstimatedHistogram getEstimatedColumnCount()
 +    {
 +        return sstableMetadata.estimatedColumnCount;
 +    }
 +
 +    public double getEstimatedDroppableTombstoneRatio(int gcBefore)
 +    {
 +        return sstableMetadata.getEstimatedDroppableTombstoneRatio(gcBefore);
 +    }
 +
 +    public double getDroppableTombstonesBefore(int gcBefore)
 +    {
 +        return sstableMetadata.getDroppableTombstonesBefore(gcBefore);
 +    }
 +
 +    public double getCompressionRatio()
 +    {
 +        return sstableMetadata.compressionRatio;
 +    }
 +
 +    public ReplayPosition getReplayPosition()
 +    {
 +        return sstableMetadata.replayPosition;
 +    }
 +
 +    public long getMinTimestamp()
 +    {
 +        return sstableMetadata.minTimestamp;
 +    }
 +
 +    public long getMaxTimestamp()
 +    {
 +        return sstableMetadata.maxTimestamp;
 +    }
 +
 +    public Set<Integer> getAncestors()
 +    {
 +        try
 +        {
 +            CompactionMetadata compactionMetadata = (CompactionMetadata) descriptor.getMetadataSerializer().deserialize(descriptor, MetadataType.COMPACTION);
 +            if (compactionMetadata != null)
 +                return compactionMetadata.ancestors;
 +            return Collections.emptySet();
 +        }
 +        catch (IOException e)
 +        {
 +            SSTableReader.logOpenException(descriptor, e);
 +            return Collections.emptySet();
 +        }
 +    }
 +
 +    public int getSSTableLevel()
 +    {
 +        return sstableMetadata.sstableLevel;
 +    }
 +
 +    /**
 +     * Reloads the sstable metadata from disk.
 +     *
 +     * Called after level is changed on sstable, for example if the sstable is dropped to L0
 +     *
 +     * Might be possible to remove in future versions
 +     *
 +     * @throws IOException
 +     */
 +    public void reloadSSTableMetadata() throws IOException
 +    {
 +        this.sstableMetadata = (StatsMetadata) descriptor.getMetadataSerializer().deserialize(descriptor, MetadataType.STATS);
 +    }
 +
 +    public StatsMetadata getSSTableMetadata()
 +    {
 +        return sstableMetadata;
 +    }
 +
 +    public RandomAccessReader openDataReader(RateLimiter limiter)
 +    {
 +        assert limiter != null;
 +        return dfile.createThrottledReader(limiter);
 +    }
 +
 +    public RandomAccessReader openDataReader()
 +    {
 +        return dfile.createReader();
 +    }
 +
 +    public RandomAccessReader openIndexReader()
 +    {
 +        if (ifile != null)
 +            return ifile.createReader();
 +        return null;
 +    }
 +
 +    /**
 +     * @param component component to get timestamp.
 +     * @return last modified time for given component. 0 if given component does not exist or IO error occurs.
 +     */
 +    public long getCreationTimeFor(Component component)
 +    {
 +        return new File(descriptor.filenameFor(component)).lastModified();
 +    }
 +
 +    /**
 +     * @return Number of key cache hit
 +     */
 +    public long getKeyCacheHit()
 +    {
 +        return keyCacheHit.get();
 +    }
 +
 +    /**
 +     * @return Number of key cache request
 +     */
 +    public long getKeyCacheRequest()
 +    {
 +        return keyCacheRequest.get();
 +    }
 +
 +    /**
 +     * Increment the total row read count and read rate for this SSTable.  This should not be incremented for range
 +     * slice queries, row cache hits, or non-query reads, like compaction.
 +     */
 +    public void incrementReadCount()
 +    {
 +        if (readMeter != null)
 +            readMeter.mark();
 +    }
 +
 +    public static class SizeComparator implements Comparator<SSTableReader>
 +    {
 +        public int compare(SSTableReader o1, SSTableReader o2)
 +        {
 +            return Longs.compare(o1.onDiskLength(), o2.onDiskLength());
 +        }
 +    }
 +
 +    public Ref<SSTableReader> tryRef()
 +    {
 +        return selfRef.tryRef();
 +    }
 +
 +    public Ref<SSTableReader> selfRef()
 +    {
 +        return selfRef;
 +    }
 +
 +    public Ref<SSTableReader> ref()
 +    {
 +        return selfRef.ref();
 +    }
 +
 +    void setup(boolean trackHotness)
 +    {
 +        tidy.setup(this, trackHotness);
 +        this.readMeter = tidy.global.readMeter;
 +    }
 +
 +    @VisibleForTesting
 +    public void overrideReadMeter(RestorableMeter readMeter)
 +    {
 +        this.readMeter = tidy.global.readMeter = readMeter;
 +    }
 +
 +    /**
 +     * One instance per SSTableReader we create. This references the type-shared tidy, which in turn references
 +     * the globally shared tidy, i.e.
 +     *
 +     * InstanceTidier => DescriptorTypeTitdy => GlobalTidy
 +     *
 +     * We can create many InstanceTidiers (one for every time we reopen an sstable with MOVED_START for example), but there can only be
 +     * two DescriptorTypeTidy (FINAL and TEMPLINK) and only one GlobalTidy for one single logical sstable.
 +     *
 +     * When the InstanceTidier cleansup, it releases its reference to its DescriptorTypeTidy; when all InstanceTidiers
 +     * for that type have run, the DescriptorTypeTidy cleansup. DescriptorTypeTidy behaves in the same way towards GlobalTidy.
 +     *
 +     * For ease, we stash a direct reference to both our type-shared and global tidier
 +     */
 +    private static final class InstanceTidier implements Tidy
 +    {
 +        private final Descriptor descriptor;
 +        private final CFMetaData metadata;
 +        private IFilter bf;
 +        private IndexSummary summary;
 +
 +        private SegmentedFile dfile;
 +        private SegmentedFile ifile;
 +        private Runnable runOnClose;
 +        private boolean isReplaced = false;
 +
 +        // a reference to our shared per-Descriptor.Type tidy instance, that
 +        // we will release when we are ourselves released
 +        private Ref<DescriptorTypeTidy> typeRef;
 +
 +        // a convenience stashing of the shared per-descriptor-type tidy instance itself
 +        // and the per-logical-sstable globally shared state that it is linked to
 +        private DescriptorTypeTidy type;
 +        private GlobalTidy global;
 +
 +        private boolean setup;
 +
 +        void setup(SSTableReader reader, boolean trackHotness)
 +        {
 +            this.setup = true;
 +            this.bf = reader.bf;
 +            this.summary = reader.indexSummary;
 +            this.dfile = reader.dfile;
 +            this.ifile = reader.ifile;
 +            // get a new reference to the shared descriptor-type tidy
 +            this.typeRef = DescriptorTypeTidy.get(reader);
 +            this.type = typeRef.get();
 +            this.global = type.globalRef.get();
 +            if (trackHotness)
 +                global.ensureReadMeter();
 +        }
 +
 +        InstanceTidier(Descriptor descriptor, CFMetaData metadata)
 +        {
 +            this.descriptor = descriptor;
 +            this.metadata = metadata;
 +        }
 +
 +        public void tidy()
 +        {
 +            // don't try to cleanup if the sstablereader was never fully constructed
 +            if (!setup)
 +                return;
 +
 +            final ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(metadata.cfId);
 +            final OpOrder.Barrier barrier;
 +            if (cfs != null)
 +            {
 +                barrier = cfs.readOrdering.newBarrier();
 +                barrier.issue();
 +            }
 +            else
 +                barrier = null;
 +
 +            ScheduledExecutors.nonPeriodicTasks.execute(new Runnable()
 +            {
 +                public void run()
 +                {
 +                    if (barrier != null)
 +                        barrier.await();
 +                    if (bf != null)
 +                        bf.close();
 +                    if (summary != null)
 +                        summary.close();
 +                    if (runOnClose != null)
 +                        runOnClose.run();
 +                    if (dfile != null)
 +                        dfile.close();
 +                    if (ifile != null)
 +                        ifile.close();
 +                    typeRef.release();
 +                }
 +            });
 +        }
 +
 +        public String name()
 +        {
 +            return descriptor.toString();
 +        }
 +
 +        void releaseSummary()
 +        {
 +            summary.close();
 +            assert summary.isCleanedUp();
 +            summary = null;
 +        }
 +    }
 +
 +    /**
 +     * One shared between all instances of a given Descriptor.Type.
 +     * Performs only two things: the deletion of the sstables for the type,
 +     * if necessary; and the shared reference to the globally shared state.
 +     *
 +     * All InstanceTidiers, on setup(), ask the static get() method for their shared state,
 +     * and stash a reference to it to be released when they are. Once all such references are
 +     * released, the shared tidy will be performed.
 +     */
 +    static final class DescriptorTypeTidy implements Tidy
 +    {
 +        // keyed by REAL descriptor (TMPLINK/FINAL), mapping to the shared DescriptorTypeTidy for that descriptor
 +        static final ConcurrentMap<Descriptor, Ref<DescriptorTypeTidy>> lookup = new ConcurrentHashMap<>();
 +
 +        private final Descriptor desc;
 +        private final Ref<GlobalTidy> globalRef;
 +        private final Set<Component> components;
 +        private long sizeOnDelete;
 +        private Counter totalDiskSpaceUsed;
 +
 +        DescriptorTypeTidy(Descriptor desc, SSTableReader sstable)
 +        {
 +            this.desc = desc;
 +            // get a new reference to the shared global tidy
 +            this.globalRef = GlobalTidy.get(sstable);
 +            this.components = sstable.components;
 +        }
 +
 +        void markObsolete(SSTableReader instance, Tracker tracker)
 +        {
 +            // the tracker is used only to notify listeners of deletion of the sstable;
 +            // since deletion of a non-final file is not really deletion of the sstable,
 +            // we don't want to notify the listeners in this event
 +            if (tracker != null && tracker.cfstore != null && desc.type == Descriptor.Type.FINAL)
 +            {
 +                sizeOnDelete = instance.bytesOnDisk();
 +                totalDiskSpaceUsed = tracker.cfstore.metric.totalDiskSpaceUsed;
 +                tracker.notifyDeleting(instance);
 +            }
 +        }
 +
 +        public void tidy()
 +        {
 +            lookup.remove(desc);
 +            boolean isCompacted = globalRef.get().isCompacted.get();
 +            globalRef.release();
 +            switch (desc.type)
 +            {
 +                case FINAL:
 +                    if (isCompacted)
 +                        new SSTableDeletingTask(desc, components, totalDiskSpaceUsed, sizeOnDelete).run();
 +                    break;
 +                case TEMPLINK:
 +                    new SSTableDeletingTask(desc, components, null, 0).run();
 +                    break;
 +                default:
 +                    throw new IllegalStateException();
 +            }
 +        }
 +
 +        public String name()
 +        {
 +            return desc.toString();
 +        }
 +
 +        // get a new reference to the shared DescriptorTypeTidy for this sstable
 +        @Suppress

<TRUNCATED>

[16/16] cassandra git commit: Merge branch 'cassandra-3.0' into trunk

Posted by be...@apache.org.
Merge branch 'cassandra-3.0' into trunk


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

Branch: refs/heads/trunk
Commit: b06e703d750dff27809eabebfb1fb2531dd4e0fe
Parents: 092281f e3d5844
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Wed Sep 30 19:47:52 2015 +0100
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Wed Sep 30 19:47:52 2015 +0100

----------------------------------------------------------------------

----------------------------------------------------------------------



[09/16] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index 87891ae,0000000..2b65792
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@@ -1,2287 -1,0 +1,2304 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.io.sstable.format;
 +
 +import java.io.*;
 +import java.nio.ByteBuffer;
 +import java.util.*;
 +import java.util.concurrent.*;
 +import java.util.concurrent.atomic.AtomicBoolean;
 +import java.util.concurrent.atomic.AtomicLong;
 +
 +import com.google.common.annotations.VisibleForTesting;
 +import com.google.common.base.Predicate;
 +import com.google.common.collect.Iterables;
 +import com.google.common.collect.Iterators;
 +import com.google.common.collect.Ordering;
 +import com.google.common.primitives.Longs;
 +import com.google.common.util.concurrent.RateLimiter;
 +
 +import com.clearspring.analytics.stream.cardinality.CardinalityMergeException;
 +import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus;
 +import com.clearspring.analytics.stream.cardinality.ICardinality;
 +import com.codahale.metrics.Counter;
 +import org.apache.cassandra.cache.CachingOptions;
 +import org.apache.cassandra.cache.InstrumentingCache;
 +import org.apache.cassandra.cache.KeyCacheKey;
 +import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
 +import org.apache.cassandra.concurrent.ScheduledExecutors;
 +import org.apache.cassandra.config.*;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 +import org.apache.cassandra.db.commitlog.ReplayPosition;
 +import org.apache.cassandra.db.composites.CellName;
 +import org.apache.cassandra.db.filter.ColumnSlice;
 +import org.apache.cassandra.db.index.SecondaryIndex;
 +import org.apache.cassandra.db.lifecycle.Tracker;
 +import org.apache.cassandra.dht.*;
 +import org.apache.cassandra.io.FSError;
++import org.apache.cassandra.io.FSReadError;
 +import org.apache.cassandra.io.compress.CompressionMetadata;
 +import org.apache.cassandra.io.sstable.*;
 +import org.apache.cassandra.io.sstable.metadata.*;
 +import org.apache.cassandra.io.util.*;
 +import org.apache.cassandra.metrics.RestorableMeter;
 +import org.apache.cassandra.metrics.StorageMetrics;
 +import org.apache.cassandra.service.ActiveRepairService;
 +import org.apache.cassandra.service.CacheService;
 +import org.apache.cassandra.service.StorageService;
 +import org.apache.cassandra.utils.*;
 +import org.apache.cassandra.utils.concurrent.OpOrder;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +import org.apache.cassandra.utils.concurrent.Ref;
 +import org.apache.cassandra.utils.concurrent.SelfRefCounted;
 +
 +import static org.apache.cassandra.db.Directories.SECONDARY_INDEX_NAME_SEPARATOR;
 +
 +/**
 + * An SSTableReader can be constructed in a number of places, but typically is either
 + * read from disk at startup, or constructed from a flushed memtable, or after compaction
 + * to replace some existing sstables. However once created, an sstablereader may also be modified.
 + *
 + * A reader's OpenReason describes its current stage in its lifecycle, as follows:
 + *
 + *
 + * <pre> {@code
 + * NORMAL
 + * From:       None        => Reader has been read from disk, either at startup or from a flushed memtable
 + *             EARLY       => Reader is the final result of a compaction
 + *             MOVED_START => Reader WAS being compacted, but this failed and it has been restored to NORMAL status
 + *
 + * EARLY
 + * From:       None        => Reader is a compaction replacement that is either incomplete and has been opened
 + *                            to represent its partial result status, or has been finished but the compaction
 + *                            it is a part of has not yet completed fully
 + *             EARLY       => Same as from None, only it is not the first time it has been
 + *
 + * MOVED_START
 + * From:       NORMAL      => Reader is being compacted. This compaction has not finished, but the compaction result
 + *                            is either partially or fully opened, to either partially or fully replace this reader.
 + *                            This reader's start key has been updated to represent this, so that reads only hit
 + *                            one or the other reader.
 + *
 + * METADATA_CHANGE
 + * From:       NORMAL      => Reader has seen low traffic and the amount of memory available for index summaries is
 + *                            constrained, so its index summary has been downsampled.
 + *         METADATA_CHANGE => Same
 + * } </pre>
 + *
 + * Note that in parallel to this, there are two different Descriptor types; TMPLINK and FINAL; the latter corresponds
 + * to NORMAL state readers and all readers that replace a NORMAL one. TMPLINK is used for EARLY state readers and
 + * no others.
 + *
 + * When a reader is being compacted, if the result is large its replacement may be opened as EARLY before compaction
 + * completes in order to present the result to consumers earlier. In this case the reader will itself be changed to
 + * a MOVED_START state, where its start no longer represents its on-disk minimum key. This is to permit reads to be
 + * directed to only one reader when the two represent the same data. The EARLY file can represent a compaction result
 + * that is either partially complete and still in-progress, or a complete and immutable sstable that is part of a larger
 + * macro compaction action that has not yet fully completed.
 + *
 + * Currently ALL compaction results at least briefly go through an EARLY open state prior to completion, regardless
 + * of if early opening is enabled.
 + *
 + * Since a reader can be created multiple times over the same shared underlying resources, and the exact resources
 + * it shares between each instance differ subtly, we track the lifetime of any underlying resource with its own
 + * reference count, which each instance takes a Ref to. Each instance then tracks references to itself, and once these
 + * all expire it releases its Refs to these underlying resources.
 + *
 + * There is some shared cleanup behaviour needed only once all sstablereaders in a certain stage of their lifecycle
 + * (i.e. EARLY or NORMAL opening), and some that must only occur once all readers of any kind over a single logical
 + * sstable have expired. These are managed by the TypeTidy and GlobalTidy classes at the bottom, and are effectively
 + * managed as another resource each instance tracks its own Ref instance to, to ensure all of these resources are
 + * cleaned up safely and can be debugged otherwise.
 + *
 + * TODO: fill in details about Tracker and lifecycle interactions for tools, and for compaction strategies
 + */
 +public abstract class SSTableReader extends SSTable implements SelfRefCounted<SSTableReader>
 +{
 +    private static final Logger logger = LoggerFactory.getLogger(SSTableReader.class);
++    private static final int ACCURATE_BOUNDARIES_MAGIC_NUMBER = 248923458;
 +
 +    private static final ScheduledThreadPoolExecutor syncExecutor = new ScheduledThreadPoolExecutor(1);
 +    static
 +    {
 +        // Immediately remove readMeter sync task when cancelled.
 +        syncExecutor.setRemoveOnCancelPolicy(true);
 +    }
 +    private static final RateLimiter meterSyncThrottle = RateLimiter.create(100.0);
 +
 +    public static final Comparator<SSTableReader> maxTimestampComparator = new Comparator<SSTableReader>()
 +    {
 +        public int compare(SSTableReader o1, SSTableReader o2)
 +        {
 +            long ts1 = o1.getMaxTimestamp();
 +            long ts2 = o2.getMaxTimestamp();
 +            return (ts1 > ts2 ? -1 : (ts1 == ts2 ? 0 : 1));
 +        }
 +    };
 +
 +    // it's just an object, which we use regular Object equality on; we introduce a special class just for easy recognition
 +    public static final class UniqueIdentifier {}
 +
 +    public static final Comparator<SSTableReader> sstableComparator = new Comparator<SSTableReader>()
 +    {
 +        public int compare(SSTableReader o1, SSTableReader o2)
 +        {
 +            return o1.first.compareTo(o2.first);
 +        }
 +    };
 +
 +    public static final Ordering<SSTableReader> sstableOrdering = Ordering.from(sstableComparator);
 +
 +    /**
 +     * maxDataAge is a timestamp in local server time (e.g. System.currentTimeMilli) which represents an upper bound
 +     * to the newest piece of data stored in the sstable. In other words, this sstable does not contain items created
 +     * later than maxDataAge.
 +     *
 +     * The field is not serialized to disk, so relying on it for more than what truncate does is not advised.
 +     *
 +     * When a new sstable is flushed, maxDataAge is set to the time of creation.
 +     * When a sstable is created from compaction, maxDataAge is set to max of all merged sstables.
 +     *
 +     * The age is in milliseconds since epoc and is local to this host.
 +     */
 +    public final long maxDataAge;
 +
 +    public enum OpenReason
 +    {
 +        NORMAL,
 +        EARLY,
 +        METADATA_CHANGE,
 +        MOVED_START
 +    }
 +
 +    public final OpenReason openReason;
 +    public final UniqueIdentifier instanceId = new UniqueIdentifier();
 +
 +    // indexfile and datafile: might be null before a call to load()
 +    protected SegmentedFile ifile;
 +    protected SegmentedFile dfile;
 +    protected IndexSummary indexSummary;
 +    protected IFilter bf;
 +
 +    protected final RowIndexEntry.IndexSerializer rowIndexEntrySerializer;
 +
 +    protected InstrumentingCache<KeyCacheKey, RowIndexEntry> keyCache;
 +
 +    protected final BloomFilterTracker bloomFilterTracker = new BloomFilterTracker();
 +
 +    // technically isCompacted is not necessary since it should never be unreferenced unless it is also compacted,
 +    // but it seems like a good extra layer of protection against reference counting bugs to not delete data based on that alone
 +    protected final AtomicBoolean isSuspect = new AtomicBoolean(false);
 +
 +    // not final since we need to be able to change level on a file.
 +    protected volatile StatsMetadata sstableMetadata;
 +
 +    protected final AtomicLong keyCacheHit = new AtomicLong(0);
 +    protected final AtomicLong keyCacheRequest = new AtomicLong(0);
 +
 +    private final InstanceTidier tidy = new InstanceTidier(descriptor, metadata);
 +    private final Ref<SSTableReader> selfRef = new Ref<>(this, tidy);
 +
 +    private RestorableMeter readMeter;
 +
 +    /**
 +     * Calculate approximate key count.
 +     * If cardinality estimator is available on all given sstables, then this method use them to estimate
 +     * key count.
 +     * If not, then this uses index summaries.
 +     *
 +     * @param sstables SSTables to calculate key count
 +     * @return estimated key count
 +     */
 +    public static long getApproximateKeyCount(Collection<SSTableReader> sstables)
 +    {
 +        long count = -1;
 +
 +        // check if cardinality estimator is available for all SSTables
 +        boolean cardinalityAvailable = !sstables.isEmpty() && Iterators.all(sstables.iterator(), new Predicate<SSTableReader>()
 +        {
 +            public boolean apply(SSTableReader sstable)
 +            {
 +                return sstable.descriptor.version.hasNewStatsFile();
 +            }
 +        });
 +
 +        // if it is, load them to estimate key count
 +        if (cardinalityAvailable)
 +        {
 +            boolean failed = false;
 +            ICardinality cardinality = null;
 +            for (SSTableReader sstable : sstables)
 +            {
 +                if (sstable.openReason == OpenReason.EARLY)
 +                    continue;
 +
 +                try
 +                {
 +                    CompactionMetadata metadata = (CompactionMetadata) sstable.descriptor.getMetadataSerializer().deserialize(sstable.descriptor, MetadataType.COMPACTION);
 +                    assert metadata != null : sstable.getFilename();
 +                    if (cardinality == null)
 +                        cardinality = metadata.cardinalityEstimator;
 +                    else
 +                        cardinality = cardinality.merge(metadata.cardinalityEstimator);
 +                }
 +                catch (IOException e)
 +                {
 +                    logger.warn("Reading cardinality from Statistics.db failed.", e);
 +                    failed = true;
 +                    break;
 +                }
 +                catch (CardinalityMergeException e)
 +                {
 +                    logger.warn("Cardinality merge failed.", e);
 +                    failed = true;
 +                    break;
 +                }
 +            }
 +            if (cardinality != null && !failed)
 +                count = cardinality.cardinality();
 +        }
 +
 +        // if something went wrong above or cardinality is not available, calculate using index summary
 +        if (count < 0)
 +        {
 +            for (SSTableReader sstable : sstables)
 +                count += sstable.estimatedKeys();
 +        }
 +        return count;
 +    }
 +
 +    /**
 +     * Estimates how much of the keys we would keep if the sstables were compacted together
 +     */
 +    public static double estimateCompactionGain(Set<SSTableReader> overlapping)
 +    {
 +        Set<ICardinality> cardinalities = new HashSet<>(overlapping.size());
 +        for (SSTableReader sstable : overlapping)
 +        {
 +            try
 +            {
 +                ICardinality cardinality = ((CompactionMetadata) sstable.descriptor.getMetadataSerializer().deserialize(sstable.descriptor, MetadataType.COMPACTION)).cardinalityEstimator;
 +                if (cardinality != null)
 +                    cardinalities.add(cardinality);
 +                else
 +                    logger.trace("Got a null cardinality estimator in: {}", sstable.getFilename());
 +            }
 +            catch (IOException e)
 +            {
 +                logger.warn("Could not read up compaction metadata for {}", sstable, e);
 +            }
 +        }
 +        long totalKeyCountBefore = 0;
 +        for (ICardinality cardinality : cardinalities)
 +        {
 +            totalKeyCountBefore += cardinality.cardinality();
 +        }
 +        if (totalKeyCountBefore == 0)
 +            return 1;
 +
 +        long totalKeyCountAfter = mergeCardinalities(cardinalities).cardinality();
 +        logger.trace("Estimated compaction gain: {}/{}={}", totalKeyCountAfter, totalKeyCountBefore, ((double)totalKeyCountAfter)/totalKeyCountBefore);
 +        return ((double)totalKeyCountAfter)/totalKeyCountBefore;
 +    }
 +
 +    private static ICardinality mergeCardinalities(Collection<ICardinality> cardinalities)
 +    {
 +        ICardinality base = new HyperLogLogPlus(13, 25); // see MetadataCollector.cardinality
 +        try
 +        {
 +            base = base.merge(cardinalities.toArray(new ICardinality[cardinalities.size()]));
 +        }
 +        catch (CardinalityMergeException e)
 +        {
 +            logger.warn("Could not merge cardinalities", e);
 +        }
 +        return base;
 +    }
 +
 +    public static SSTableReader open(Descriptor descriptor) throws IOException
 +    {
 +        CFMetaData metadata;
 +        if (descriptor.cfname.contains(SECONDARY_INDEX_NAME_SEPARATOR))
 +        {
 +            int i = descriptor.cfname.indexOf(SECONDARY_INDEX_NAME_SEPARATOR);
 +            String parentName = descriptor.cfname.substring(0, i);
 +            CFMetaData parent = Schema.instance.getCFMetaData(descriptor.ksname, parentName);
 +            ColumnDefinition def = parent.getColumnDefinitionForIndex(descriptor.cfname.substring(i + 1));
 +            metadata = CFMetaData.newIndexMetadata(parent, def, SecondaryIndex.getIndexComparator(parent, def));
 +        }
 +        else
 +        {
 +            metadata = Schema.instance.getCFMetaData(descriptor.ksname, descriptor.cfname);
 +        }
 +        return open(descriptor, metadata);
 +    }
 +
 +    public static SSTableReader open(Descriptor desc, CFMetaData metadata) throws IOException
 +    {
 +        IPartitioner p = desc.cfname.contains(SECONDARY_INDEX_NAME_SEPARATOR)
 +                ? new LocalPartitioner(metadata.getKeyValidator())
 +                : StorageService.getPartitioner();
 +        return open(desc, componentsFor(desc), metadata, p);
 +    }
 +
 +    public static SSTableReader open(Descriptor descriptor, Set<Component> components, CFMetaData metadata, IPartitioner partitioner) throws IOException
 +    {
 +        return open(descriptor, components, metadata, partitioner, true, true);
 +    }
 +
 +    // use only for offline or "Standalone" operations
 +    public static SSTableReader openNoValidation(Descriptor descriptor, Set<Component> components, ColumnFamilyStore cfs) throws IOException
 +    {
 +        return open(descriptor, components, cfs.metadata, cfs.partitioner, false, false); // do not track hotness
 +    }
 +
 +    /**
 +     * Open SSTable reader to be used in batch mode(such as sstableloader).
 +     *
 +     * @param descriptor
 +     * @param components
 +     * @param metadata
 +     * @param partitioner
 +     * @return opened SSTableReader
 +     * @throws IOException
 +     */
 +    public static SSTableReader openForBatch(Descriptor descriptor, Set<Component> components, CFMetaData metadata, IPartitioner partitioner) throws IOException
 +    {
 +        // Minimum components without which we can't do anything
 +        assert components.contains(Component.DATA) : "Data component is missing for sstable " + descriptor;
 +        assert components.contains(Component.PRIMARY_INDEX) : "Primary index component is missing for sstable " + descriptor;
 +
 +        Map<MetadataType, MetadataComponent> sstableMetadata = descriptor.getMetadataSerializer().deserialize(descriptor,
 +                EnumSet.of(MetadataType.VALIDATION, MetadataType.STATS));
 +        ValidationMetadata validationMetadata = (ValidationMetadata) sstableMetadata.get(MetadataType.VALIDATION);
 +        StatsMetadata statsMetadata = (StatsMetadata) sstableMetadata.get(MetadataType.STATS);
 +
 +        // Check if sstable is created using same partitioner.
 +        // Partitioner can be null, which indicates older version of sstable or no stats available.
 +        // In that case, we skip the check.
 +        String partitionerName = partitioner.getClass().getCanonicalName();
 +        if (validationMetadata != null && !partitionerName.equals(validationMetadata.partitioner))
 +        {
 +            logger.error(String.format("Cannot open %s; partitioner %s does not match system partitioner %s.  Note that the default partitioner starting with Cassandra 1.2 is Murmur3Partitioner, so you will need to edit that to match your old partitioner if upgrading.",
 +                    descriptor, validationMetadata.partitioner, partitionerName));
 +            System.exit(1);
 +        }
 +
 +        logger.debug("Opening {} ({} bytes)", descriptor, new File(descriptor.filenameFor(Component.DATA)).length());
 +        SSTableReader sstable = internalOpen(descriptor, components, metadata, partitioner, System.currentTimeMillis(),
 +                statsMetadata, OpenReason.NORMAL);
 +
 +        // special implementation of load to use non-pooled SegmentedFile builders
 +        try(SegmentedFile.Builder ibuilder = new BufferedSegmentedFile.Builder();
 +            SegmentedFile.Builder dbuilder = sstable.compression
 +                ? new CompressedSegmentedFile.Builder(null)
 +                : new BufferedSegmentedFile.Builder())
 +        {
 +            if (!sstable.loadSummary(ibuilder, dbuilder))
 +                sstable.buildSummary(false, ibuilder, dbuilder, false, Downsampling.BASE_SAMPLING_LEVEL);
 +            sstable.ifile = ibuilder.complete(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX));
 +            sstable.dfile = dbuilder.complete(sstable.descriptor.filenameFor(Component.DATA));
 +            sstable.bf = FilterFactory.AlwaysPresent;
 +            sstable.setup(false);
 +            return sstable;
 +        }
 +    }
 +
 +    public static SSTableReader open(Descriptor descriptor,
 +                                      Set<Component> components,
 +                                      CFMetaData metadata,
 +                                      IPartitioner partitioner,
 +                                      boolean validate,
 +                                      boolean trackHotness) throws IOException
 +    {
 +        // Minimum components without which we can't do anything
 +        assert components.contains(Component.DATA) : "Data component is missing for sstable " + descriptor;
 +        assert !validate || components.contains(Component.PRIMARY_INDEX) : "Primary index component is missing for sstable " + descriptor;
 +
 +        Map<MetadataType, MetadataComponent> sstableMetadata = descriptor.getMetadataSerializer().deserialize(descriptor,
 +                                                                                                               EnumSet.of(MetadataType.VALIDATION, MetadataType.STATS));
 +        ValidationMetadata validationMetadata = (ValidationMetadata) sstableMetadata.get(MetadataType.VALIDATION);
 +        StatsMetadata statsMetadata = (StatsMetadata) sstableMetadata.get(MetadataType.STATS);
 +
 +        // Check if sstable is created using same partitioner.
 +        // Partitioner can be null, which indicates older version of sstable or no stats available.
 +        // In that case, we skip the check.
 +        String partitionerName = partitioner.getClass().getCanonicalName();
 +        if (validationMetadata != null && !partitionerName.equals(validationMetadata.partitioner))
 +        {
 +            logger.error(String.format("Cannot open %s; partitioner %s does not match system partitioner %s.  Note that the default partitioner starting with Cassandra 1.2 is Murmur3Partitioner, so you will need to edit that to match your old partitioner if upgrading.",
 +                    descriptor, validationMetadata.partitioner, partitionerName));
 +            System.exit(1);
 +        }
 +
 +        logger.debug("Opening {} ({} bytes)", descriptor, new File(descriptor.filenameFor(Component.DATA)).length());
 +        SSTableReader sstable = internalOpen(descriptor, components, metadata, partitioner, System.currentTimeMillis(),
 +                                             statsMetadata, OpenReason.NORMAL);
 +        try
 +        {
 +            // load index and filter
 +            long start = System.nanoTime();
 +            sstable.load(validationMetadata);
 +            logger.trace("INDEX LOAD TIME for {}: {} ms.", descriptor, TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
 +
 +            sstable.setup(trackHotness);
 +            if (validate)
 +                sstable.validate();
 +
 +            if (sstable.getKeyCache() != null)
 +                logger.trace("key cache contains {}/{} keys", sstable.getKeyCache().size(), sstable.getKeyCache().getCapacity());
 +
 +            return sstable;
 +        }
 +        catch (Throwable t)
 +        {
 +            sstable.selfRef().release();
 +            throw t;
 +        }
 +    }
 +
 +    public static void logOpenException(Descriptor descriptor, IOException e)
 +    {
 +        if (e instanceof FileNotFoundException)
 +            logger.error("Missing sstable component in {}; skipped because of {}", descriptor, e.getMessage());
 +        else
 +            logger.error("Corrupt sstable {}; skipped", descriptor, e);
 +    }
 +
 +    public static Collection<SSTableReader> openAll(Set<Map.Entry<Descriptor, Set<Component>>> entries,
 +                                                    final CFMetaData metadata,
 +                                                    final IPartitioner partitioner)
 +    {
 +        final Collection<SSTableReader> sstables = new LinkedBlockingQueue<>();
 +
 +        ExecutorService executor = DebuggableThreadPoolExecutor.createWithFixedPoolSize("SSTableBatchOpen", FBUtilities.getAvailableProcessors());
 +        for (final Map.Entry<Descriptor, Set<Component>> entry : entries)
 +        {
 +            Runnable runnable = new Runnable()
 +            {
 +                public void run()
 +                {
 +                    SSTableReader sstable;
 +                    try
 +                    {
 +                        sstable = open(entry.getKey(), entry.getValue(), metadata, partitioner);
 +                    }
 +                    catch (CorruptSSTableException ex)
 +                    {
 +                        FileUtils.handleCorruptSSTable(ex);
 +                        logger.error("Corrupt sstable {}; skipping table", entry, ex);
 +                        return;
 +                    }
 +                    catch (FSError ex)
 +                    {
 +                        FileUtils.handleFSError(ex);
 +                        logger.error("Cannot read sstable {}; file system error, skipping table", entry, ex);
 +                        return;
 +                    }
 +                    catch (IOException ex)
 +                    {
 +                        logger.error("Cannot read sstable {}; other IO error, skipping table", entry, ex);
 +                        return;
 +                    }
 +                    sstables.add(sstable);
 +                }
 +            };
 +            executor.submit(runnable);
 +        }
 +
 +        executor.shutdown();
 +        try
 +        {
 +            executor.awaitTermination(7, TimeUnit.DAYS);
 +        }
 +        catch (InterruptedException e)
 +        {
 +            throw new AssertionError(e);
 +        }
 +
 +        return sstables;
 +
 +    }
 +
 +    /**
 +     * Open a RowIndexedReader which already has its state initialized (by SSTableWriter).
 +     */
 +    public static SSTableReader internalOpen(Descriptor desc,
 +                                      Set<Component> components,
 +                                      CFMetaData metadata,
 +                                      IPartitioner partitioner,
 +                                      SegmentedFile ifile,
 +                                      SegmentedFile dfile,
 +                                      IndexSummary isummary,
 +                                      IFilter bf,
 +                                      long maxDataAge,
 +                                      StatsMetadata sstableMetadata,
 +                                      OpenReason openReason)
 +    {
 +        assert desc != null && partitioner != null && ifile != null && dfile != null && isummary != null && bf != null && sstableMetadata != null;
 +
 +        SSTableReader reader = internalOpen(desc, components, metadata, partitioner, maxDataAge, sstableMetadata, openReason);
 +
 +        reader.bf = bf;
 +        reader.ifile = ifile;
 +        reader.dfile = dfile;
 +        reader.indexSummary = isummary;
 +        reader.setup(true);
 +
 +        return reader;
 +    }
 +
 +
 +    private static SSTableReader internalOpen(final Descriptor descriptor,
 +                                            Set<Component> components,
 +                                            CFMetaData metadata,
 +                                            IPartitioner partitioner,
 +                                            Long maxDataAge,
 +                                            StatsMetadata sstableMetadata,
 +                                            OpenReason openReason)
 +    {
 +        Factory readerFactory = descriptor.getFormat().getReaderFactory();
 +
 +        return readerFactory.open(descriptor, components, metadata, partitioner, maxDataAge, sstableMetadata, openReason);
 +    }
 +
 +    protected SSTableReader(final Descriptor desc,
 +                            Set<Component> components,
 +                            CFMetaData metadata,
 +                            IPartitioner partitioner,
 +                            long maxDataAge,
 +                            StatsMetadata sstableMetadata,
 +                            OpenReason openReason)
 +    {
 +        super(desc, components, metadata, partitioner);
 +        this.sstableMetadata = sstableMetadata;
 +        this.maxDataAge = maxDataAge;
 +        this.openReason = openReason;
 +        this.rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata);
 +    }
 +
 +    public static long getTotalBytes(Iterable<SSTableReader> sstables)
 +    {
 +        long sum = 0;
 +        for (SSTableReader sstable : sstables)
 +            sum += sstable.onDiskLength();
 +        return sum;
 +    }
 +
 +    public static long getTotalUncompressedBytes(Iterable<SSTableReader> sstables)
 +    {
 +        long sum = 0;
 +        for (SSTableReader sstable : sstables)
 +            sum += sstable.uncompressedLength();
 +
 +        return sum;
 +    }
 +
 +    public boolean equals(Object that)
 +    {
 +        return that instanceof SSTableReader && ((SSTableReader) that).descriptor.equals(this.descriptor);
 +    }
 +
 +    public int hashCode()
 +    {
 +        return this.descriptor.hashCode();
 +    }
 +
 +    public String getFilename()
 +    {
 +        return dfile.path();
 +    }
 +
 +    public void setupKeyCache()
 +    {
 +        // under normal operation we can do this at any time, but SSTR is also used outside C* proper,
 +        // e.g. by BulkLoader, which does not initialize the cache.  As a kludge, we set up the cache
 +        // here when we know we're being wired into the rest of the server infrastructure.
 +        keyCache = CacheService.instance.keyCache;
 +    }
 +
 +    private void load(ValidationMetadata validation) throws IOException
 +    {
 +        if (metadata.getBloomFilterFpChance() == 1.0)
 +        {
 +            // bf is disabled.
 +            load(false, true);
 +            bf = FilterFactory.AlwaysPresent;
 +        }
 +        else if (!components.contains(Component.PRIMARY_INDEX))
 +        {
 +            // avoid any reading of the missing primary index component.
 +            // this should only happen during StandaloneScrubber
 +            load(false, false);
 +        }
 +        else if (!components.contains(Component.FILTER) || validation == null)
 +        {
 +            // bf is enabled, but filter component is missing.
 +            load(true, true);
 +        }
 +        else if (validation.bloomFilterFPChance != metadata.getBloomFilterFpChance())
 +        {
 +            // bf fp chance in sstable metadata and it has changed since compaction.
 +            load(true, true);
 +        }
 +        else
 +        {
 +            // bf is enabled and fp chance matches the currently configured value.
 +            load(false, true);
 +            loadBloomFilter();
 +        }
 +    }
 +
 +    /**
 +     * Load bloom filter from Filter.db file.
 +     *
 +     * @throws IOException
 +     */
 +    private void loadBloomFilter() throws IOException
 +    {
 +        try (DataInputStream stream = new DataInputStream(new BufferedInputStream(new FileInputStream(descriptor.filenameFor(Component.FILTER)))))
 +        {
 +            bf = FilterFactory.deserialize(stream, true);
 +        }
 +    }
 +
 +    /**
 +     * Loads ifile, dfile and indexSummary, and optionally recreates the bloom filter.
 +     * @param saveSummaryIfCreated for bulk loading purposes, if the summary was absent and needed to be built, you can
 +     *                             avoid persisting it to disk by setting this to false
 +     */
 +    private void load(boolean recreateBloomFilter, boolean saveSummaryIfCreated) throws IOException
 +    {
 +        try(SegmentedFile.Builder ibuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode(), false);
 +            SegmentedFile.Builder dbuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode(), compression))
 +        {
 +            boolean summaryLoaded = loadSummary(ibuilder, dbuilder);
 +            boolean builtSummary = false;
 +            if (recreateBloomFilter || !summaryLoaded)
 +            {
 +                buildSummary(recreateBloomFilter, ibuilder, dbuilder, summaryLoaded, Downsampling.BASE_SAMPLING_LEVEL);
 +                builtSummary = true;
 +            }
 +
 +            if (components.contains(Component.PRIMARY_INDEX))
 +                ifile = ibuilder.complete(descriptor.filenameFor(Component.PRIMARY_INDEX));
 +
 +            dfile = dbuilder.complete(descriptor.filenameFor(Component.DATA));
 +
 +            // Check for an index summary that was downsampled even though the serialization format doesn't support
 +            // that.  If it was downsampled, rebuild it.  See CASSANDRA-8993 for details.
 +        if (!descriptor.version.hasSamplingLevel() && !builtSummary && !validateSummarySamplingLevel() && ifile != null)
 +            {
 +                indexSummary.close();
 +                ifile.close();
 +                dfile.close();
 +
 +                logger.info("Detected erroneously downsampled index summary; will rebuild summary at full sampling");
 +                FileUtils.deleteWithConfirm(new File(descriptor.filenameFor(Component.SUMMARY)));
 +
 +                try(SegmentedFile.Builder ibuilderRebuild = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode(), false);
 +                    SegmentedFile.Builder dbuilderRebuild = SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode(), compression))
 +                {
 +                    buildSummary(false, ibuilderRebuild, dbuilderRebuild, false, Downsampling.BASE_SAMPLING_LEVEL);
 +                    ifile = ibuilderRebuild.complete(descriptor.filenameFor(Component.PRIMARY_INDEX));
 +                    dfile = dbuilderRebuild.complete(descriptor.filenameFor(Component.DATA));
 +                    saveSummary(ibuilderRebuild, dbuilderRebuild);
 +                }
 +            }
 +            else if (saveSummaryIfCreated && builtSummary)
 +            {
 +                saveSummary(ibuilder, dbuilder);
 +            }
 +        }
 +        catch (Throwable t)
 +        { // Because the tidier has not been set-up yet in SSTableReader.open(), we must release the files in case of error
 +            if (ifile != null)
 +            {
 +                ifile.close();
 +                ifile = null;
 +            }
 +
 +            if (dfile != null)
 +            {
 +                dfile.close();
 +                dfile = null;
 +            }
 +
 +            if (indexSummary != null)
 +            {
 +                indexSummary.close();
 +                indexSummary = null;
 +            }
 +
 +            throw t;
 +        }
 +    }
 +
 +    /**
 +     * Build index summary(and optionally bloom filter) by reading through Index.db file.
 +     *
 +     * @param recreateBloomFilter true if recreate bloom filter
 +     * @param ibuilder
 +     * @param dbuilder
 +     * @param summaryLoaded true if index summary is already loaded and not need to build again
 +     * @throws IOException
 +     */
 +    private void buildSummary(boolean recreateBloomFilter, SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, boolean summaryLoaded, int samplingLevel) throws IOException
 +    {
 +         if (!components.contains(Component.PRIMARY_INDEX))
 +             return;
 +
 +        // we read the positions in a BRAF so we don't have to worry about an entry spanning a mmap boundary.
 +        try (RandomAccessReader primaryIndex = RandomAccessReader.open(new File(descriptor.filenameFor(Component.PRIMARY_INDEX))))
 +        {
 +            long indexSize = primaryIndex.length();
 +            long histogramCount = sstableMetadata.estimatedRowSize.count();
 +            long estimatedKeys = histogramCount > 0 && !sstableMetadata.estimatedRowSize.isOverflowed()
 +                    ? histogramCount
 +                    : estimateRowsFromIndex(primaryIndex); // statistics is supposed to be optional
 +
 +            if (recreateBloomFilter)
 +                bf = FilterFactory.getFilter(estimatedKeys, metadata.getBloomFilterFpChance(), true);
 +
 +            try (IndexSummaryBuilder summaryBuilder = summaryLoaded ? null : new IndexSummaryBuilder(estimatedKeys, metadata.getMinIndexInterval(), samplingLevel))
 +            {
 +                long indexPosition;
 +                RowIndexEntry.IndexSerializer rowIndexSerializer = descriptor.getFormat().getIndexSerializer(metadata);
 +
 +                while ((indexPosition = primaryIndex.getFilePointer()) != indexSize)
 +                {
 +                    ByteBuffer key = ByteBufferUtil.readWithShortLength(primaryIndex);
 +                    RowIndexEntry indexEntry = rowIndexSerializer.deserialize(primaryIndex, descriptor.version);
 +                    DecoratedKey decoratedKey = partitioner.decorateKey(key);
 +                    if (first == null)
 +                        first = decoratedKey;
 +                    last = decoratedKey;
 +
 +                    if (recreateBloomFilter)
 +                        bf.add(decoratedKey);
 +
 +                    // if summary was already read from disk we don't want to re-populate it using primary index
 +                    if (!summaryLoaded)
 +                    {
 +                        summaryBuilder.maybeAddEntry(decoratedKey, indexPosition);
 +                        ibuilder.addPotentialBoundary(indexPosition);
 +                        dbuilder.addPotentialBoundary(indexEntry.position);
 +                    }
 +                }
 +
 +                if (!summaryLoaded)
 +                    indexSummary = summaryBuilder.build(partitioner);
 +            }
 +        }
 +
 +        first = getMinimalKey(first);
 +        last = getMinimalKey(last);
 +    }
 +
 +    /**
 +     * Load index summary from Summary.db file if it exists.
 +     *
 +     * if loaded index summary has different index interval from current value stored in schema,
 +     * then Summary.db file will be deleted and this returns false to rebuild summary.
 +     *
 +     * @param ibuilder
 +     * @param dbuilder
 +     * @return true if index summary is loaded successfully from Summary.db file.
 +     */
 +    @SuppressWarnings("resource")
 +    public boolean loadSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
 +    {
 +        File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
 +        if (!summariesFile.exists())
 +            return false;
 +
 +        DataInputStream iStream = null;
 +        try
 +        {
 +            iStream = new DataInputStream(new FileInputStream(summariesFile));
 +            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);
 +            dbuilder.deserializeBounds(iStream);
++
++            boolean checkForRepair = true;
++            try
++            {
++                int v = iStream.readInt();
++                // check for our magic number, indicating this summary has been sampled correctly
++                checkForRepair = v != ACCURATE_BOUNDARIES_MAGIC_NUMBER;
++            }
++            catch (Throwable t) {}
++
++            // fix CASSANDRA-10357 on-the-fly
++            if (checkForRepair && MmappedSegmentedFile.maybeRepair(metadata, descriptor, indexSummary, ibuilder, dbuilder))
++                saveSummary(ibuilder, dbuilder);
 +        }
 +        catch (IOException e)
 +        {
 +            if (indexSummary != null)
 +                indexSummary.close();
 +            logger.trace("Cannot deserialize SSTable Summary File {}: {}", summariesFile.getPath(), e.getMessage());
 +            // corrupted; delete it and fall back to creating a new summary
 +            FileUtils.closeQuietly(iStream);
 +            // delete it and fall back to creating a new summary
 +            FileUtils.deleteWithConfirm(summariesFile);
 +            return false;
 +        }
 +        finally
 +        {
 +            FileUtils.closeQuietly(iStream);
 +        }
 +
 +        return true;
 +    }
 +
 +    /**
 +     * Validates that an index summary has full sampling, as expected when the serialization format does not support
 +     * persisting the sampling level.
 +     * @return true if the summary has full sampling, false otherwise
 +     */
 +    private boolean validateSummarySamplingLevel()
 +    {
 +        // We need to check index summary entries against the index to verify that none of them were dropped due to
 +        // downsampling.  Downsampling can drop any of the first BASE_SAMPLING_LEVEL entries (repeating that drop pattern
 +        // for the remainder of the summary).  Unfortunately, the first entry to be dropped is the entry at
 +        // index (BASE_SAMPLING_LEVEL - 1), so we need to check a full set of BASE_SAMPLING_LEVEL entries.
 +        if (ifile == null)
 +            return false;
 +
 +        Iterator<FileDataInput> segments = ifile.iterator(0);
 +        int i = 0;
 +        int summaryEntriesChecked = 0;
 +        int expectedIndexInterval = getMinIndexInterval();
 +        while (segments.hasNext())
 +        {
 +            String path = null;
 +            try (FileDataInput in = segments.next())
 +            {
 +                path = in.getPath();
 +                while (!in.isEOF())
 +                {
 +                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
 +                    if (i % expectedIndexInterval == 0)
 +                    {
 +                        ByteBuffer summaryKey = ByteBuffer.wrap(indexSummary.getKey(i / expectedIndexInterval));
 +                        if (!summaryKey.equals(indexKey))
 +                            return false;
 +                        summaryEntriesChecked++;
 +
 +                        if (summaryEntriesChecked == Downsampling.BASE_SAMPLING_LEVEL)
 +                            return true;
 +                    }
 +                    RowIndexEntry.Serializer.skip(in);
 +                    i++;
 +                }
 +            }
 +            catch (IOException e)
 +            {
 +                markSuspect();
 +                throw new CorruptSSTableException(e, path);
 +            }
 +        }
 +
 +        return true;
 +    }
 +
 +    /**
 +     * Save index summary to Summary.db file.
 +     *
 +     * @param ibuilder
 +     * @param dbuilder
 +     */
 +
 +    public void saveSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
 +    {
 +        saveSummary(this.descriptor, this.first, this.last, ibuilder, dbuilder, indexSummary);
 +    }
 +
 +    private void saveSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, IndexSummary newSummary)
 +    {
 +        saveSummary(this.descriptor, this.first, this.last, ibuilder, dbuilder, newSummary);
 +    }
 +    /**
 +     * Save index summary to Summary.db file.
 +     */
 +    public static void saveSummary(Descriptor descriptor, DecoratedKey first, DecoratedKey last,
 +                                   SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, IndexSummary summary)
 +    {
 +        File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
 +        if (summariesFile.exists())
 +            FileUtils.deleteWithConfirm(summariesFile);
 +
 +        try (DataOutputStreamPlus oStream = new BufferedDataOutputStreamPlus(new FileOutputStream(summariesFile));)
 +        {
 +            IndexSummary.serializer.serialize(summary, oStream, descriptor.version.hasSamplingLevel());
 +            ByteBufferUtil.writeWithLength(first.getKey(), oStream);
 +            ByteBufferUtil.writeWithLength(last.getKey(), oStream);
 +            ibuilder.serializeBounds(oStream);
 +            dbuilder.serializeBounds(oStream);
++            // write a magic number, to indicate this summary has been sampled correctly
++            oStream.writeInt(ACCURATE_BOUNDARIES_MAGIC_NUMBER);
 +        }
 +        catch (IOException e)
 +        {
 +            logger.trace("Cannot save SSTable Summary: ", e);
 +
 +            // corrupted hence delete it and let it load it now.
 +            if (summariesFile.exists())
 +                FileUtils.deleteWithConfirm(summariesFile);
 +        }
 +    }
 +
 +    public void setReplaced()
 +    {
 +        synchronized (tidy.global)
 +        {
 +            assert !tidy.isReplaced;
 +            tidy.isReplaced = true;
 +        }
 +    }
 +
 +    public boolean isReplaced()
 +    {
 +        synchronized (tidy.global)
 +        {
 +            return tidy.isReplaced;
 +        }
 +    }
 +
 +    // runOnClose must NOT be an anonymous or non-static inner class, nor must it retain a reference chain to this reader
 +    public void runOnClose(final Runnable runOnClose)
 +    {
 +        synchronized (tidy.global)
 +        {
 +            final Runnable existing = tidy.runOnClose;
 +            tidy.runOnClose = AndThen.get(existing, runOnClose);
 +        }
 +    }
 +
 +    private static class AndThen implements Runnable
 +    {
 +        final Runnable runFirst;
 +        final Runnable runSecond;
 +
 +        private AndThen(Runnable runFirst, Runnable runSecond)
 +        {
 +            this.runFirst = runFirst;
 +            this.runSecond = runSecond;
 +        }
 +
 +        public void run()
 +        {
 +            runFirst.run();
 +            runSecond.run();
 +        }
 +
 +        static Runnable get(Runnable runFirst, Runnable runSecond)
 +        {
 +            if (runFirst == null)
 +                return runSecond;
 +            return new AndThen(runFirst, runSecond);
 +        }
 +    }
 +
 +    /**
 +     * Clone this reader with the provided start and open reason, and set the clone as replacement.
 +     *
 +     * @param newFirst the first key for the replacement (which can be different from the original due to the pre-emptive
 +     * opening of compaction results).
 +     * @param reason the {@code OpenReason} for the replacement.
 +     *
 +     * @return the cloned reader. That reader is set as a replacement by the method.
 +     */
 +    private SSTableReader cloneAndReplace(DecoratedKey newFirst, OpenReason reason)
 +    {
 +        return cloneAndReplace(newFirst, reason, indexSummary.sharedCopy());
 +    }
 +
 +    /**
 +     * Clone this reader with the new values and set the clone as replacement.
 +     *
 +     * @param newFirst the first key for the replacement (which can be different from the original due to the pre-emptive
 +     * opening of compaction results).
 +     * @param reason the {@code OpenReason} for the replacement.
 +     * @param newSummary the index summary for the replacement.
 +     *
 +     * @return the cloned reader. That reader is set as a replacement by the method.
 +     */
 +    private SSTableReader cloneAndReplace(DecoratedKey newFirst, OpenReason reason, IndexSummary newSummary)
 +    {
 +        SSTableReader replacement = internalOpen(descriptor,
 +                                                 components,
 +                                                 metadata,
 +                                                 partitioner,
 +                                                 ifile != null ? ifile.sharedCopy() : null,
 +                                                 dfile.sharedCopy(),
 +                                                 newSummary,
 +                                                 bf.sharedCopy(),
 +                                                 maxDataAge,
 +                                                 sstableMetadata,
 +                                                 reason);
 +        replacement.first = newFirst;
 +        replacement.last = last;
 +        replacement.isSuspect.set(isSuspect.get());
 +        return replacement;
 +    }
 +
 +    // runOnClose must NOT be an anonymous or non-static inner class, nor must it retain a reference chain to this reader
 +    public SSTableReader cloneWithNewStart(DecoratedKey newStart, final Runnable runOnClose)
 +    {
 +        synchronized (tidy.global)
 +        {
 +            assert openReason != OpenReason.EARLY;
 +            // TODO: merge with caller's firstKeyBeyond() work,to save time
 +            if (newStart.compareTo(first) > 0)
 +            {
 +                final long dataStart = getPosition(newStart, Operator.EQ).position;
 +                final long indexStart = getIndexScanPosition(newStart);
 +                this.tidy.runOnClose = new DropPageCache(dfile, dataStart, ifile, indexStart, runOnClose);
 +            }
 +
 +            return cloneAndReplace(newStart, OpenReason.MOVED_START);
 +        }
 +    }
 +
 +    private static class DropPageCache implements Runnable
 +    {
 +        final SegmentedFile dfile;
 +        final long dfilePosition;
 +        final SegmentedFile ifile;
 +        final long ifilePosition;
 +        final Runnable andThen;
 +
 +        private DropPageCache(SegmentedFile dfile, long dfilePosition, SegmentedFile ifile, long ifilePosition, Runnable andThen)
 +        {
 +            this.dfile = dfile;
 +            this.dfilePosition = dfilePosition;
 +            this.ifile = ifile;
 +            this.ifilePosition = ifilePosition;
 +            this.andThen = andThen;
 +        }
 +
 +        public void run()
 +        {
 +            dfile.dropPageCache(dfilePosition);
 +
 +            if (ifile != null)
 +                ifile.dropPageCache(ifilePosition);
 +            andThen.run();
 +        }
 +    }
 +
 +    /**
 +     * Returns a new SSTableReader with the same properties as this SSTableReader except that a new IndexSummary will
 +     * be built at the target samplingLevel.  This (original) SSTableReader instance will be marked as replaced, have
 +     * its DeletingTask removed, and have its periodic read-meter sync task cancelled.
 +     * @param samplingLevel the desired sampling level for the index summary on the new SSTableReader
 +     * @return a new SSTableReader
 +     * @throws IOException
 +     */
 +    @SuppressWarnings("resource")
 +    public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
 +    {
 +        assert descriptor.version.hasSamplingLevel();
 +
 +        synchronized (tidy.global)
 +        {
 +            assert openReason != OpenReason.EARLY;
 +
 +            int minIndexInterval = metadata.getMinIndexInterval();
 +            int maxIndexInterval = metadata.getMaxIndexInterval();
 +            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
 +
 +            IndexSummary newSummary;
 +            long oldSize = bytesOnDisk();
 +
 +            // 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 = 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);
 +
 +                try(SegmentedFile.Builder ibuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode(), false);
 +                    SegmentedFile.Builder dbuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode(), compression))
 +                {
 +                    saveSummary(ibuilder, dbuilder, newSummary);
 +                }
 +            }
 +            else
 +            {
 +                throw new AssertionError("Attempted to clone SSTableReader with the same index summary sampling level and " +
 +                        "no adjustments to min/max_index_interval");
 +            }
 +
 +            long newSize = bytesOnDisk();
 +            StorageMetrics.load.inc(newSize - oldSize);
 +            parent.metric.liveDiskSpaceUsed.inc(newSize - oldSize);
 +
 +            return cloneAndReplace(first, OpenReason.METADATA_CHANGE, newSummary);
 +        }
 +    }
 +
 +    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();
 +            try (IndexSummaryBuilder summaryBuilder = new IndexSummaryBuilder(estimatedKeys(), metadata.getMinIndexInterval(), newSamplingLevel))
 +            {
 +                long indexPosition;
 +                while ((indexPosition = primaryIndex.getFilePointer()) != indexSize)
 +                {
 +                    summaryBuilder.maybeAddEntry(partitioner.decorateKey(ByteBufferUtil.readWithShortLength(primaryIndex)), indexPosition);
 +                    RowIndexEntry.Serializer.skip(primaryIndex);
 +                }
 +
 +                return summaryBuilder.build(partitioner);
 +            }
 +        }
 +        finally
 +        {
 +            FileUtils.closeQuietly(primaryIndex);
 +        }
 +    }
 +
 +    public RestorableMeter getReadMeter()
 +    {
 +        return readMeter;
 +    }
 +
 +    public int getIndexSummarySamplingLevel()
 +    {
 +        return indexSummary.getSamplingLevel();
 +    }
 +
 +    public long getIndexSummaryOffHeapSize()
 +    {
 +        return indexSummary.getOffHeapSize();
 +    }
 +
 +    public int getMinIndexInterval()
 +    {
 +        return indexSummary.getMinIndexInterval();
 +    }
 +
 +    public double getEffectiveIndexInterval()
 +    {
 +        return indexSummary.getEffectiveIndexInterval();
 +    }
 +
 +    public void releaseSummary()
 +    {
 +        tidy.releaseSummary();
 +        indexSummary = null;
 +    }
 +
 +    private void validate()
 +    {
 +        if (this.first.compareTo(this.last) > 0)
 +        {
 +            selfRef().release();
 +            throw new IllegalStateException(String.format("SSTable first key %s > last key %s", this.first, this.last));
 +        }
 +    }
 +
 +    /**
 +     * Gets the position in the index file to start scanning to find the given key (at most indexInterval keys away,
 +     * modulo downsampling of the index summary). Always returns a value >= 0
 +     */
 +    public long getIndexScanPosition(RowPosition key)
 +    {
 +        if (openReason == OpenReason.MOVED_START && key.compareTo(first) < 0)
 +            key = first;
 +
 +        return getIndexScanPositionFromBinarySearchResult(indexSummary.binarySearch(key), indexSummary);
 +    }
 +
 +    @VisibleForTesting
 +    public static long getIndexScanPositionFromBinarySearchResult(int binarySearchResult, IndexSummary referencedIndexSummary)
 +    {
 +        if (binarySearchResult == -1)
 +            return 0;
 +        else
 +            return referencedIndexSummary.getPosition(getIndexSummaryIndexFromBinarySearchResult(binarySearchResult));
 +    }
 +
 +    public static int getIndexSummaryIndexFromBinarySearchResult(int binarySearchResult)
 +    {
 +        if (binarySearchResult < 0)
 +        {
 +            // binary search gives us the first index _greater_ than the key searched for,
 +            // i.e., its insertion position
 +            int greaterThan = (binarySearchResult + 1) * -1;
 +            if (greaterThan == 0)
 +                return -1;
 +            return greaterThan - 1;
 +        }
 +        else
 +        {
 +            return binarySearchResult;
 +        }
 +    }
 +
 +    /**
 +     * Returns the compression metadata for this sstable.
 +     * @throws IllegalStateException if the sstable is not compressed
 +     */
 +    public CompressionMetadata getCompressionMetadata()
 +    {
 +        if (!compression)
 +            throw new IllegalStateException(this + " is not compressed");
 +
 +        CompressionMetadata cmd = ((ICompressedFile) dfile).getMetadata();
 +
 +        //We need the parent cf metadata
 +        String cfName = metadata.isSecondaryIndex() ? metadata.getParentColumnFamilyName() : metadata.cfName;
 +        cmd.parameters.setLiveMetadata(Schema.instance.getCFMetaData(metadata.ksName, cfName));
 +
 +        return cmd;
 +    }
 +
 +    /**
 +     * Returns the amount of memory in bytes used off heap by the compression meta-data.
 +     * @return the amount of memory in bytes used off heap by the compression meta-data
 +     */
 +    public long getCompressionMetadataOffHeapSize()
 +    {
 +        if (!compression)
 +            return 0;
 +
 +        return getCompressionMetadata().offHeapSize();
 +    }
 +
 +    /**
 +     * For testing purposes only.
 +     */
 +    public void forceFilterFailures()
 +    {
 +        bf = FilterFactory.AlwaysPresent;
 +    }
 +
 +    public IFilter getBloomFilter()
 +    {
 +        return bf;
 +    }
 +
 +    public long getBloomFilterSerializedSize()
 +    {
 +        return bf.serializedSize();
 +    }
 +
 +    /**
 +     * Returns the amount of memory in bytes used off heap by the bloom filter.
 +     * @return the amount of memory in bytes used off heap by the bloom filter
 +     */
 +    public long getBloomFilterOffHeapSize()
 +    {
 +        return bf.offHeapSize();
 +    }
 +
 +    /**
 +     * @return An estimate of the number of keys in this SSTable based on the index summary.
 +     */
 +    public long estimatedKeys()
 +    {
 +        return indexSummary.getEstimatedKeyCount();
 +    }
 +
 +    /**
 +     * @param ranges
 +     * @return An estimate of the number of keys for given ranges in this SSTable.
 +     */
 +    public long estimatedKeysForRanges(Collection<Range<Token>> ranges)
 +    {
 +        long sampleKeyCount = 0;
 +        List<Pair<Integer, Integer>> sampleIndexes = getSampleIndexesForRanges(indexSummary, ranges);
 +        for (Pair<Integer, Integer> sampleIndexRange : sampleIndexes)
 +            sampleKeyCount += (sampleIndexRange.right - sampleIndexRange.left + 1);
 +
 +        // adjust for the current sampling level: (BSL / SL) * index_interval_at_full_sampling
 +        long estimatedKeys = sampleKeyCount * ((long) Downsampling.BASE_SAMPLING_LEVEL * indexSummary.getMinIndexInterval()) / indexSummary.getSamplingLevel();
 +        return Math.max(1, estimatedKeys);
 +    }
 +
 +    /**
 +     * Returns the number of entries in the IndexSummary.  At full sampling, this is approximately 1/INDEX_INTERVALth of
 +     * the keys in this SSTable.
 +     */
 +    public int getIndexSummarySize()
 +    {
 +        return indexSummary.size();
 +    }
 +
 +    /**
 +     * Returns the approximate number of entries the IndexSummary would contain if it were at full sampling.
 +     */
 +    public int getMaxIndexSummarySize()
 +    {
 +        return indexSummary.getMaxNumberOfEntries();
 +    }
 +
 +    /**
 +     * Returns the key for the index summary entry at `index`.
 +     */
 +    public byte[] getIndexSummaryKey(int index)
 +    {
 +        return indexSummary.getKey(index);
 +    }
 +
 +    private static List<Pair<Integer,Integer>> getSampleIndexesForRanges(IndexSummary summary, Collection<Range<Token>> ranges)
 +    {
 +        // use the index to determine a minimal section for each range
 +        List<Pair<Integer,Integer>> positions = new ArrayList<>();
 +
 +        for (Range<Token> range : Range.normalize(ranges))
 +        {
 +            RowPosition leftPosition = range.left.maxKeyBound();
 +            RowPosition rightPosition = range.right.maxKeyBound();
 +
 +            int left = summary.binarySearch(leftPosition);
 +            if (left < 0)
 +                left = (left + 1) * -1;
 +            else
 +                // left range are start exclusive
 +                left = left + 1;
 +            if (left == summary.size())
 +                // left is past the end of the sampling
 +                continue;
 +
 +            int right = Range.isWrapAround(range.left, range.right)
 +                    ? summary.size() - 1
 +                    : summary.binarySearch(rightPosition);
 +            if (right < 0)
 +            {
 +                // range are end inclusive so we use the previous index from what binarySearch give us
 +                // since that will be the last index we will return
 +                right = (right + 1) * -1;
 +                if (right == 0)
 +                    // Means the first key is already stricly greater that the right bound
 +                    continue;
 +                right--;
 +            }
 +
 +            if (left > right)
 +                // empty range
 +                continue;
 +            positions.add(Pair.create(left, right));
 +        }
 +        return positions;
 +    }
 +
 +    public Iterable<DecoratedKey> getKeySamples(final Range<Token> range)
 +    {
 +        final List<Pair<Integer, Integer>> indexRanges = getSampleIndexesForRanges(indexSummary, Collections.singletonList(range));
 +
 +        if (indexRanges.isEmpty())
 +            return Collections.emptyList();
 +
 +        return new Iterable<DecoratedKey>()
 +        {
 +            public Iterator<DecoratedKey> iterator()
 +            {
 +                return new Iterator<DecoratedKey>()
 +                {
 +                    private Iterator<Pair<Integer, Integer>> rangeIter = indexRanges.iterator();
 +                    private Pair<Integer, Integer> current;
 +                    private int idx;
 +
 +                    public boolean hasNext()
 +                    {
 +                        if (current == null || idx > current.right)
 +                        {
 +                            if (rangeIter.hasNext())
 +                            {
 +                                current = rangeIter.next();
 +                                idx = current.left;
 +                                return true;
 +                            }
 +                            return false;
 +                        }
 +
 +                        return true;
 +                    }
 +
 +                    public DecoratedKey next()
 +                    {
 +                        byte[] bytes = indexSummary.getKey(idx++);
 +                        return partitioner.decorateKey(ByteBuffer.wrap(bytes));
 +                    }
 +
 +                    public void remove()
 +                    {
 +                        throw new UnsupportedOperationException();
 +                    }
 +                };
 +            }
 +        };
 +    }
 +
 +    /**
 +     * Determine the minimal set of sections that can be extracted from this SSTable to cover the given ranges.
 +     * @return A sorted list of (offset,end) pairs that cover the given ranges in the datafile for this SSTable.
 +     */
 +    public List<Pair<Long,Long>> getPositionsForRanges(Collection<Range<Token>> ranges)
 +    {
 +        // use the index to determine a minimal section for each range
 +        List<Pair<Long,Long>> positions = new ArrayList<>();
 +        for (Range<Token> range : Range.normalize(ranges))
 +        {
 +            assert !range.isWrapAround() || range.right.isMinimum();
 +            // truncate the range so it at most covers the sstable
 +            AbstractBounds<RowPosition> bounds = Range.makeRowRange(range);
 +            RowPosition leftBound = bounds.left.compareTo(first) > 0 ? bounds.left : first.getToken().minKeyBound();
 +            RowPosition rightBound = bounds.right.isMinimum() ? last.getToken().maxKeyBound() : bounds.right;
 +
 +            if (leftBound.compareTo(last) > 0 || rightBound.compareTo(first) < 0)
 +                continue;
 +
 +            long left = getPosition(leftBound, Operator.GT).position;
 +            long right = (rightBound.compareTo(last) > 0)
 +                         ? uncompressedLength()
 +                         : getPosition(rightBound, Operator.GT).position;
 +
 +            if (left == right)
 +                // empty range
 +                continue;
 +
 +            assert left < right : String.format("Range=%s openReason=%s first=%s last=%s left=%d right=%d", range, openReason, first, last, left, right);
 +            positions.add(Pair.create(left, right));
 +        }
 +        return positions;
 +    }
 +
 +    public KeyCacheKey getCacheKey(DecoratedKey key)
 +    {
 +        return new KeyCacheKey(metadata.ksAndCFName, descriptor, key.getKey());
 +    }
 +
 +    public void cacheKey(DecoratedKey key, RowIndexEntry info)
 +    {
 +        CachingOptions caching = metadata.getCaching();
 +
 +        if (!caching.keyCache.isEnabled()
 +                || keyCache == null
 +                || keyCache.getCapacity() == 0)
 +        {
 +            return;
 +        }
 +
 +        KeyCacheKey cacheKey = new KeyCacheKey(metadata.ksAndCFName, descriptor, key.getKey());
 +        logger.trace("Adding cache entry for {} -> {}", cacheKey, info);
 +        keyCache.put(cacheKey, info);
 +    }
 +
 +    public RowIndexEntry getCachedPosition(DecoratedKey key, boolean updateStats)
 +    {
 +        return getCachedPosition(new KeyCacheKey(metadata.ksAndCFName, descriptor, key.getKey()), updateStats);
 +    }
 +
 +    protected RowIndexEntry getCachedPosition(KeyCacheKey unifiedKey, boolean updateStats)
 +    {
 +        if (keyCache != null && keyCache.getCapacity() > 0 && metadata.getCaching().keyCache.isEnabled()) {
 +            if (updateStats)
 +            {
 +                RowIndexEntry cachedEntry = keyCache.get(unifiedKey);
 +                keyCacheRequest.incrementAndGet();
 +                if (cachedEntry != null)
 +                {
 +                    keyCacheHit.incrementAndGet();
 +                    bloomFilterTracker.addTruePositive();
 +                }
 +                return cachedEntry;
 +            }
 +            else
 +            {
 +                return keyCache.getInternal(unifiedKey);
 +            }
 +        }
 +        return null;
 +    }
 +
 +    /**
 +     * Get position updating key cache and stats.
 +     * @see #getPosition(org.apache.cassandra.db.RowPosition, SSTableReader.Operator, boolean)
 +     */
 +    public RowIndexEntry getPosition(RowPosition key, Operator op)
 +    {
 +        return getPosition(key, op, true, false);
 +    }
 +
 +    public RowIndexEntry getPosition(RowPosition key, Operator op, boolean updateCacheAndStats)
 +    {
 +        return getPosition(key, op, updateCacheAndStats, false);
 +    }
 +    /**
 +     * @param key The key to apply as the rhs to the given Operator. A 'fake' key is allowed to
 +     * allow key selection by token bounds but only if op != * EQ
 +     * @param op The Operator defining matching keys: the nearest key to the target matching the operator wins.
 +     * @param updateCacheAndStats true if updating stats and cache
 +     * @return The index entry corresponding to the key, or null if the key is not present
 +     */
 +    protected abstract RowIndexEntry getPosition(RowPosition key, Operator op, boolean updateCacheAndStats, boolean permitMatchPastLast);
 +
 +    //Corresponds to a name column
 +    public abstract OnDiskAtomIterator iterator(DecoratedKey key, SortedSet<CellName> columns);
 +    public abstract OnDiskAtomIterator iterator(FileDataInput file, DecoratedKey key, SortedSet<CellName> columns, RowIndexEntry indexEntry);
 +
 +    //Corresponds to a slice query
 +    public abstract OnDiskAtomIterator iterator(DecoratedKey key, ColumnSlice[] slices, boolean reverse);
 +    public abstract OnDiskAtomIterator iterator(FileDataInput file, DecoratedKey key, ColumnSlice[] slices, boolean reversed, RowIndexEntry indexEntry);
 +
 +    /**
 +     * Finds and returns the first key beyond a given token in this SSTable or null if no such key exists.
 +     */
 +    public DecoratedKey firstKeyBeyond(RowPosition token)
 +    {
 +        if (token.compareTo(first) < 0)
 +            return first;
 +
 +        long sampledPosition = getIndexScanPosition(token);
 +
 +        if (ifile == null)
 +            return null;
 +
 +        Iterator<FileDataInput> segments = ifile.iterator(sampledPosition);
 +        while (segments.hasNext())
 +        {
 +            String path = null;
 +            try (FileDataInput in = segments.next();)
 +            {
 +                path = in.getPath();
 +                while (!in.isEOF())
 +                {
 +                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
 +                    DecoratedKey indexDecoratedKey = partitioner.decorateKey(indexKey);
 +                    if (indexDecoratedKey.compareTo(token) > 0)
 +                        return indexDecoratedKey;
 +
 +                    RowIndexEntry.Serializer.skip(in);
 +                }
 +            }
 +            catch (IOException e)
 +            {
 +                markSuspect();
 +                throw new CorruptSSTableException(e, path);
 +            }
 +        }
 +
 +        return null;
 +    }
 +
 +    /**
 +     * @return The length in bytes of the data for this SSTable. For
 +     * compressed files, this is not the same thing as the on disk size (see
 +     * onDiskLength())
 +     */
 +    public long uncompressedLength()
 +    {
 +        return dfile.length;
 +    }
 +
 +    /**
 +     * @return The length in bytes of the on disk size for this SSTable. For
 +     * compressed files, this is not the same thing as the data length (see
 +     * length())
 +     */
 +    public long onDiskLength()
 +    {
 +        return dfile.onDiskLength;
 +    }
 +
 +    /**
 +     * Mark the sstable as obsolete, i.e., compacted into newer sstables.
 +     *
 +     * When calling this function, the caller must ensure that the SSTableReader is not referenced anywhere
 +     * except for threads holding a reference.
 +     *
 +     * @return true if the this is the first time the file was marked obsolete.  Calling this
 +     * multiple times is usually buggy (see exceptions in Tracker.unmarkCompacting and removeOldSSTablesSize).
 +     */
 +    public boolean markObsolete(Tracker tracker)
 +    {
 +        if (logger.isTraceEnabled())
 +            logger.trace("Marking {} compacted", getFilename());
 +
 +        synchronized (tidy.global)
 +        {
 +            assert !tidy.isReplaced;
 +        }
 +        if (!tidy.global.isCompacted.getAndSet(true))
 +        {
 +            tidy.type.markObsolete(this, tracker);
 +            return true;
 +        }
 +        return false;
 +    }
 +
 +    public boolean isMarkedCompacted()
 +    {
 +        return tidy.global.isCompacted.get();
 +    }
 +
 +    public void markSuspect()
 +    {
 +        if (logger.isTraceEnabled())
 +            logger.trace("Marking {} as a suspect for blacklisting.", getFilename());
 +
 +        isSuspect.getAndSet(true);
 +    }
 +
 +    public boolean isMarkedSuspect()
 +    {
 +        return isSuspect.get();
 +    }
 +
 +
 +    /**
 +     * I/O SSTableScanner
 +     * @return A Scanner for seeking over the rows of the SSTable.
 +     */
 +    public ISSTableScanner getScanner()
 +    {
 +        return getScanner((RateLimiter) null);
 +    }
 +
 +    public ISSTableScanner getScanner(RateLimiter limiter)
 +    {
 +        return getScanner(DataRange.allData(partitioner), limiter);
 +    }
 +
 +    /**
 +     *
 +     * @param dataRange filter to use when reading the columns
 +     * @return A Scanner for seeking over the rows of the SSTable.
 +     */
 +    public ISSTableScanner getScanner(DataRange dataRange)
 +    {
 +        return getScanner(dataRange, null);
 +    }
 +
 +    /**
 +     * Direct I/O SSTableScanner over a defined range of tokens.
 +     *
 +     * @param range the range of keys to cover
 +     * @return A Scanner for seeking over the rows of the SSTable.
 +     */
 +    public ISSTableScanner getScanner(Range<Token> range, RateLimiter limiter)
 +    {
 +        if (range == null)
 +            return getScanner(limiter);
 +        return getScanner(Collections.singletonList(range), limiter);
 +    }
 +
 +    /**
 +     * Direct I/O SSTableScanner over a defined collection of ranges of tokens.
 +     *
 +     * @param ranges the range of keys to cover
 +     * @return A Scanner for seeking over the rows of the SSTable.
 +     */
 +    public abstract ISSTableScanner getScanner(Collection<Range<Token>> ranges, RateLimiter limiter);
 +
 +    /**
 +     *
 +     * @param dataRange filter to use when reading the columns
 +     * @return A Scanner for seeking over the rows of the SSTable.
 +     */
 +    public abstract ISSTableScanner getScanner(DataRange dataRange, RateLimiter limiter);
 +
 +
 +
 +    public FileDataInput getFileDataInput(long position)
 +    {
 +        return dfile.getSegment(position);
 +    }
 +
 +    /**
 +     * Tests if the sstable contains data newer than the given age param (in localhost currentMilli time).
 +     * This works in conjunction with maxDataAge which is an upper bound on the create of data in this sstable.
 +     * @param age The age to compare the maxDataAre of this sstable. Measured in millisec since epoc on this host
 +     * @return True iff this sstable contains data that's newer than the given age parameter.
 +     */
 +    public boolean newSince(long age)
 +    {
 +        return maxDataAge > age;
 +    }
 +
 +    public void createLinks(String snapshotDirectoryPath)
 +    {
 +        for (Component component : components)
 +        {
 +            File sourceFile = new File(descriptor.filenameFor(component));
 +            File targetLink = new File(snapshotDirectoryPath, sourceFile.getName());
 +            FileUtils.createHardLink(sourceFile, targetLink);
 +        }
 +    }
 +
 +    public boolean isRepaired()
 +    {
 +        return sstableMetadata.repairedAt != ActiveRepairService.UNREPAIRED_SSTABLE;
 +    }
 +
 +    /**
 +     * TODO: Move someplace reusable
 +     */
 +    public abstract static class Operator
 +    {
 +        public static final Operator EQ = new Equals();
 +        public static final Operator GE = new GreaterThanOrEqualTo();
 +        public static final Operator GT = new GreaterThan();
 +
 +        /**
 +         * @param comparison The result of a call to compare/compareTo, with the desired field on the rhs.
 +         * @return less than 0 if the operator cannot match forward, 0 if it matches, greater than 0 if it might match forward.
 +         */
 +        public abstract int apply(int comparison);
 +
 +        final static class Equals extends Operator
 +        {
 +            public int apply(int comparison) { return -comparison; }
 +        }
 +
 +        final static class GreaterThanOrEqualTo extends Operator
 +        {
 +            public int apply(int comparison) { return comparison >= 0 ? 0 : 1; }
 +        }
 +
 +        final static class GreaterThan extends Operator
 +        {
 +            public int apply(int comparison) { return comparison > 0 ? 0 : 1; }
 +        }
 +    }
 +
 +    public long getBloomFilterFalsePositiveCount()
 +    {
 +        return bloomFilterTracker.getFalsePositiveCount();
 +    }
 +
 +    public long getRecentBloomFilterFalsePositiveCount()
 +    {
 +        return bloomFilterTracker.getRecentFalsePositiveCount();
 +    }
 +
 +    public long getBloomFilterTruePositiveCount()
 +    {
 +        return bloomFilterTracker.getTruePositiveCount();
 +    }
 +
 +    public long getRecentBloomFilterTruePositiveCount()
 +    {
 +        return bloomFilterTracker.getRecentTruePositiveCount();
 +    }
 +
 +    public InstrumentingCache<KeyCacheKey, RowIndexEntry> getKeyCache()
 +    {
 +        return keyCache;
 +    }
 +
 +    public EstimatedHistogram getEstimatedRowSize()
 +    {
 +        return sstableMetadata.estimatedRowSize;
 +    }
 +
 +    public EstimatedHistogram getEstimatedColumnCount()
 +    {
 +        return sstableMetadata.estimatedColumnCount;
 +    }
 +
 +    public double getEstimatedDroppableTombstoneRatio(int gcBefore)
 +    {
 +        return sstableMetadata.getEstimatedDroppableTombstoneRatio(gcBefore);
 +    }
 +
 +    public double getDroppableTombstonesBefore(int gcBefore)
 +    {
 +        return sstableMetadata.getDroppableTombstonesBefore(gcBefore);
 +    }
 +
 +    public double getCompressionRatio()
 +    {
 +        return sstableMetadata.compressionRatio;
 +    }
 +
 +    public ReplayPosition getReplayPosition()
 +    {
 +        return sstableMetadata.replayPosition;
 +    }
 +
 +    public long getMinTimestamp()
 +    {
 +        return sstableMetadata.minTimestamp;
 +    }
 +
 +    public long getMaxTimestamp()
 +    {
 +        return sstableMetadata.maxTimestamp;
 +    }
 +
 +    public Set<Integer> getAncestors()
 +    {
 +        try
 +        {
 +            CompactionMetadata compactionMetadata = (CompactionMetadata) descriptor.getMetadataSerializer().deserialize(descriptor, MetadataType.COMPACTION);
 +            if (compactionMetadata != null)
 +                return compactionMetadata.ancestors;
 +            return Collections.emptySet();
 +        }
 +        catch (IOException e)
 +        {
 +            SSTableReader.logOpenException(descriptor, e);
 +            return Collections.emptySet();
 +        }
 +    }
 +
 +    public int getSSTableLevel()
 +    {
 +        return sstableMetadata.sstableLevel;
 +    }
 +
 +    /**
 +     * Reloads the sstable metadata from disk.
 +     *
 +     * Called after level is changed on sstable, for example if the sstable is dropped to L0
 +     *
 +     * Might be possible to remove in future versions
 +     *
 +     * @throws IOException
 +     */
 +    public void reloadSSTableMetadata() throws IOException
 +    {
 +        this.sstableMetadata = (StatsMetadata) descriptor.getMetadataSerializer().deserialize(descriptor, MetadataType.STATS);
 +    }
 +
 +    public StatsMetadata getSSTableMetadata()
 +    {
 +        return sstableMetadata;
 +    }
 +
 +    public RandomAccessReader openDataReader(RateLimiter limiter)
 +    {
 +        assert limiter != null;
 +        return dfile.createThrottledReader(limiter);
 +    }
 +
 +    public RandomAccessReader openDataReader()
 +    {
 +        return dfile.createReader();
 +    }
 +
 +    public RandomAccessReader openIndexReader()
 +    {
 +        if (ifile != null)
 +            return ifile.createReader();
 +        return null;
 +    }
 +
 +    /**
 +     * @param component component to get timestamp.
 +     * @return last modified time for given component. 0 if given component does not exist or IO error occurs.
 +     */
 +    public long getCreationTimeFor(Component component)
 +    {
 +        return new File(descriptor.filenameFor(component)).lastModified();
 +    }
 +
 +    /**
 +     * @return Number of key cache hit
 +     */
 +    public long getKeyCacheHit()
 +    {
 +        return keyCacheHit.get();
 +    }
 +
 +    /**
 +     * @return Number of key cache request
 +     */
 +    public long getKeyCacheRequest()
 +    {
 +        return keyCacheRequest.get();
 +    }
 +
 +    /**
 +     * Increment the total row read count and read rate for this SSTable.  This should not be incremented for range
 +     * slice queries, row cache hits, or non-query reads, like compaction.
 +     */
 +    public void incrementReadCount()
 +    {
 +        if (readMeter != null)
 +            readMeter.mark();
 +    }
 +
 +    public static class SizeComparator implements Comparator<SSTableReader>
 +    {
 +        public int compare(SSTableReader o1, SSTableReader o2)
 +        {
 +            return Longs.compare(o1.onDiskLength(), o2.onDiskLength());
 +        }
 +    }
 +
 +    public Ref<SSTableReader> tryRef()
 +    {
 +        return selfRef.tryRef();
 +    }
 +
 +    public Ref<SSTableReader> selfRef()
 +    {
 +        return selfRef;
 +    }
 +
 +    public Ref<SSTableReader> ref()
 +    {
 +        return selfRef.ref();
 +    }
 +
 +    void setup(boolean trackHotness)
 +    {
 +        tidy.setup(this, trackHotness);
 +        this.readMeter = tidy.global.readMeter;
 +    }
 +
 +    @VisibleForTesting
 +    public void overrideReadMeter(RestorableMeter readMeter)
 +    {
 +        this.readMeter = tidy.global.readMeter = readMeter;
 +    }
 +
 +    /**
 +     * One instance per SSTableReader we create. This references the type-shared tidy, which in turn references
 +     * the globally shared tidy, i.e.
 +     *
 +     * InstanceTidier => DescriptorTypeTitdy => GlobalTidy
 +     *
 +     * We can create many InstanceTidiers (one for every time we reopen an sstable with MOVED_START for example), but there can only be
 +     * two DescriptorTypeTidy (FINAL and TEMPLINK) and only one GlobalTidy for one single logical sstable.
 +     *
 +     * When the InstanceTidier cleansup, it releases its reference to its DescriptorTypeTidy; when all InstanceTidiers
 +     * for that type have run, the DescriptorTypeTidy cleansup. DescriptorTypeTidy behaves in the same way towards GlobalTidy.
 +     *
 +     * For ease, we stash a direct reference to both our type-shared and global tidier
 +     */
 +    private static final class InstanceTidier implements Tidy
 +    {
 +        private final Descriptor descriptor;
 +        private final CFMetaData metadata;
 +        private IFilter bf;
 +        private IndexSummary summary;
 +
 +        private SegmentedFile dfile;
 +        private SegmentedFile ifile;
 +        private Runnable runOnClose;
 +        private boolean isReplaced = false;
 +
 +        // a reference to our shared per-Descriptor.Type tidy instance, that
 +        // we will release when we are ourselves released
 +        private Ref<DescriptorTypeTidy> typeRef;
 +
 +        // a convenience stashing of the shared per-descriptor-type tidy instance itself
 +        // and the per-logical-sstable globally shared state that it is linked to
 +        private DescriptorTypeTidy type;
 +        private GlobalTidy global;
 +
 +        private boolean setup;
 +
 +        void setup(SSTableReader reader, boolean trackHotness)
 +        {
 +            this.setup = true;
 +            this.bf = reader.bf;
 +            this.summary = reader.indexSummary;
 +            this.dfile = reader.dfile;
 +            this.ifile = reader.ifile;
 +            // get a new reference to the shared descriptor-type tidy
 +            this.typeRef = DescriptorTypeTidy.get(reader);
 +            this.type = typeRef.get();
 +            this.global = type.globalRef.get();
 +            if (trackHotness)
 +                global.ensureReadMeter();
 +        }
 +
 +        InstanceTidier(Descriptor descriptor, CFMetaData metadata)
 +        {
 +            this.descriptor = descriptor;
 +            this.metadata = metadata;
 +        }
 +
 +        public void tidy()
 +        {
 +            // don't try to cleanup if the sstablereader was never fully constructed
 +            if (!setup)
 +                return;
 +
 +            final ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(metadata.cfId);
 +            final OpOrder.Barrier barrier;
 +            if (cfs != null)
 +            {
 +                barrier = cfs.readOrdering.newBarrier();
 +                barrier.issue();
 +            }
 +            else
 +                barrier = null;
 +
 +            ScheduledExecutors.nonPeriodicTasks.execute(new Runnable()
 +            {
 +                public void run()
 +                {
 +                    if (barrier != null)
 +                        barrier.await();
 +                    if (bf != null)
 +                        bf.close();
 +                    if (summary != null)
 +                        summary.close();
 +                    if (runOnClose != null)
 +                        runOnClose.run();
 +                    if (dfile != null)
 +                        dfile.close();
 +                    if (ifile != null)
 +                        ifile.close();
 +                    typeRef.release();
 +                }
 +            });
 +        }
 +
 +        public String name()
 +        {
 +            return descriptor.toString();
 +        }
 +
 +        void releaseSummary()
 +        {
 +            summary.close();
 +            assert summary.isCleanedUp();
 +            summary = null;
 +        }
 +    }
 +
 +    /**
 +     * One shared between all instances of a given Descriptor.Type.
 +     * Performs only two things: the deletion of the sstables for the type,
 +     * if necessary; and the shared reference to the globally shared state.
 +     *
 +     * All InstanceTidiers, on setup(), ask the static get() method for their shared state,
 +     * and stash a reference to it to be released when they are. Once all such references are
 +     * released, the shared tidy will be performed.
 +     */
 +    static final class DescriptorTypeTidy implements Tidy
 +    {
 +        // keyed by REAL descriptor (TMPLINK/FINAL), mapping to the shared DescriptorTypeTidy for that descriptor
 +        static final ConcurrentMap<Descriptor, Ref<DescriptorTypeTidy>> lookup = new ConcurrentHashMap<>();
 +
 +        private final Descriptor desc;
 +        private final Ref<GlobalTidy> globalRef;
 +        private final Set<Component> components;
 +        private long sizeOnDelete;
 +        private Counter totalDiskSpaceUsed;
 +
 +        DescriptorTypeTidy(Descriptor desc, SSTableReader sstable)
 +        {
 +            this.desc = desc;
 +            // get a new reference to the shared global tidy
 +            this.globalRef = GlobalTidy.get(sstable);
 +            this.components = sstable.components;
 +        }
 +
 +        void markObsolete(SSTableReader instance, Tracker tracker)
 +        {
 +            // the tracker is used only to notify listeners of deletion of the sstable;
 +            // since deletion of a non-final file is not really deletion of the sstable,
 +            // we don't want to notify the listeners in this event
 +            if (tracker != null && tracker.cfstore != null && desc.type == Descriptor.Type.FINAL)
 +            {
 +                sizeOnDelete = instance.bytesOnDisk();
 +                totalDiskSpaceUsed = tracker.cfstore.metric.totalDiskSpaceUsed;
 +                tracker.notifyDeleting(instance);
 +            }
 +        }
 +
 +        public void tidy()
 +        {
 +            lookup.remove(desc);
 +            boolean isCompacted = globalRef.get().isCompacted.get();
 +            globalRef.release();
 +            switch (desc.type)
 +            {
 +                case FINAL:
 +                    if (isCompacted)
 +                        new SSTableDeletingTask(desc, components, totalDiskSpaceUsed, sizeOnDelete).run();
 +                    break;
 +                case TEMPLINK:
 +                    new SSTableDeletingTask(desc, components, null, 0).run();
 +                    break;
 +                default:
 +                    throw new IllegalStateException();
 +            }
 +        }
 +
 +        public String name()
 +        {
 +            return desc.toString();
 +        }
 +
 +        // get a new reference to the shared DescriptorTypeTidy for this sstable
 +        @Suppress

<TRUNCATED>

[03/16] cassandra git commit: Fix Mmapped File Boundaries

Posted by be...@apache.org.
Fix Mmapped File Boundaries

This patch fixes two bugs with mmap segment boundary
tracking, and introduces automated correction of
this bug on startup

patch by benedict; reviewed by tjake for CASSANDRA-10357


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

Branch: refs/heads/cassandra-3.0
Commit: c37562e345c24720c55428a8644191df68319812
Parents: f6cab37
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Wed Sep 16 18:09:32 2015 +0100
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Wed Sep 30 19:45:49 2015 +0100

----------------------------------------------------------------------
 .../io/sstable/AbstractSSTableSimpleWriter.java |   2 +
 .../cassandra/io/sstable/CQLSSTableWriter.java  |  15 +
 .../cassandra/io/sstable/SSTableReader.java     |  34 +-
 .../io/sstable/SSTableSimpleUnsortedWriter.java |   6 +
 .../io/sstable/SSTableSimpleWriter.java         |  12 +
 .../cassandra/io/util/MappedFileDataInput.java  |   8 +-
 .../cassandra/io/util/MmappedSegmentedFile.java | 270 +++++++++++++---
 .../apache/cassandra/io/util/SegmentedFile.java |   1 +
 .../sstable/LongSegmentedFileBoundaryTest.java  | 322 +++++++++++++++++++
 9 files changed, 601 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
index 165a4b2..557c3de 100644
--- a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
@@ -200,4 +200,6 @@ public abstract class AbstractSSTableSimpleWriter implements Closeable
     protected abstract void writeRow(DecoratedKey key, ColumnFamily columnFamily) throws IOException;
 
     protected abstract ColumnFamily getColumnFamily() throws IOException;
+
+    public abstract Descriptor getCurrentDescriptor();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
index b211a90..c364171 100644
--- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
@@ -271,6 +271,16 @@ public class CQLSSTableWriter implements Closeable
         writer.close();
     }
 
+    public Descriptor getCurrentDescriptor()
+    {
+        return writer.getCurrentDescriptor();
+    }
+
+    public CFMetaData getCFMetaData()
+    {
+        return writer.metadata;
+    }
+
     /**
      * A Builder for a CQLSSTableWriter object.
      */
@@ -366,6 +376,11 @@ public class CQLSSTableWriter implements Closeable
             }
         }
 
+        CFMetaData metadata()
+        {
+            return schema;
+        }
+
         /**
          * Adds the specified column family to the specified keyspace.
          *

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/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 0f307b0..84add6f 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
@@ -17,13 +17,7 @@
  */
 package org.apache.cassandra.io.sstable;
 
-import java.io.BufferedInputStream;
-import java.io.DataInputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.IOException;
+import java.io.*;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.*;
@@ -70,20 +64,14 @@ import org.apache.cassandra.dht.LocalPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.FSError;
+import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.io.compress.CompressionMetadata;
 import org.apache.cassandra.io.sstable.metadata.CompactionMetadata;
 import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
 import org.apache.cassandra.io.sstable.metadata.MetadataType;
 import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
 import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
-import org.apache.cassandra.io.util.BufferedSegmentedFile;
-import org.apache.cassandra.io.util.CompressedSegmentedFile;
-import org.apache.cassandra.io.util.DataOutputStreamAndChannel;
-import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.io.util.ICompressedFile;
-import org.apache.cassandra.io.util.RandomAccessReader;
-import org.apache.cassandra.io.util.SegmentedFile;
+import org.apache.cassandra.io.util.*;
 import org.apache.cassandra.metrics.RestorableMeter;
 import org.apache.cassandra.metrics.StorageMetrics;
 import org.apache.cassandra.service.ActiveRepairService;
@@ -162,6 +150,7 @@ import static org.apache.cassandra.db.Directories.SECONDARY_INDEX_NAME_SEPARATOR
 public class SSTableReader extends SSTable implements SelfRefCounted<SSTableReader>
 {
     private static final Logger logger = LoggerFactory.getLogger(SSTableReader.class);
+    private static final int ACCURATE_BOUNDARIES_MAGIC_NUMBER = 248923458;
 
     private static final ScheduledThreadPoolExecutor syncExecutor = new ScheduledThreadPoolExecutor(1);
     static
@@ -892,6 +881,19 @@ public class SSTableReader extends SSTable implements SelfRefCounted<SSTableRead
             last = partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
             ibuilder.deserializeBounds(iStream);
             dbuilder.deserializeBounds(iStream);
+
+            boolean checkForRepair = true;
+            try
+            {
+                int v = iStream.readInt();
+                // check for our magic number, indicating this summary has been sampled correctly
+                checkForRepair = v != ACCURATE_BOUNDARIES_MAGIC_NUMBER;
+            }
+            catch (Throwable t) {}
+
+            // fix CASSANDRA-10357 on-the-fly
+            if (checkForRepair && MmappedSegmentedFile.maybeRepair(metadata, descriptor, indexSummary, ibuilder, dbuilder))
+                saveSummary(ibuilder, dbuilder);
         }
         catch (IOException e)
         {
@@ -992,6 +994,8 @@ public class SSTableReader extends SSTable implements SelfRefCounted<SSTableRead
             ByteBufferUtil.writeWithLength(last.getKey(), oStream);
             ibuilder.serializeBounds(oStream);
             dbuilder.serializeBounds(oStream);
+            // write a magic number, to indicate this summary has been sampled correctly
+            oStream.writeInt(ACCURATE_BOUNDARIES_MAGIC_NUMBER);
         }
         catch (IOException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
index 9ee9ea1..25ec354 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
@@ -140,6 +140,12 @@ public class SSTableSimpleUnsortedWriter extends AbstractSSTableSimpleWriter
         return previous;
     }
 
+    public Descriptor getCurrentDescriptor()
+    {
+        // can be implemented, but isn't necessary
+        throw new UnsupportedOperationException();
+    }
+
     protected ColumnFamily createColumnFamily() throws IOException
     {
         return ArrayBackedSortedColumns.factory.create(metadata);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
index 87c8e33..23da501 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
@@ -65,6 +65,13 @@ public class SSTableSimpleWriter extends AbstractSSTableSimpleWriter
         writer = getWriter();
     }
 
+    SSTableReader closeAndOpenReader()
+    {
+        if (currentKey != null)
+            writeRow(currentKey, columnFamily);
+        return writer.closeAndOpenReader();
+    }
+
     public void close()
     {
         try
@@ -89,4 +96,9 @@ public class SSTableSimpleWriter extends AbstractSSTableSimpleWriter
     {
         return ArrayBackedSortedColumns.factory.create(metadata);
     }
+
+    public Descriptor getCurrentDescriptor()
+    {
+        return writer.descriptor;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java b/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java
index d056240..f93ce72 100644
--- a/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java
+++ b/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java
@@ -51,12 +51,18 @@ public class MappedFileDataInput extends AbstractDataInput implements FileDataIn
     public void seek(long pos) throws IOException
     {
         long inSegmentPos = pos - segmentOffset;
-        if (inSegmentPos < 0 || inSegmentPos > buffer.capacity())
+        if (!contains(pos))
             throw new IOException(String.format("Seek position %d is not within mmap segment (seg offs: %d, length: %d)", pos, segmentOffset, buffer.capacity()));
 
         seekInternal((int) inSegmentPos);
     }
 
+    public boolean contains(long pos)
+    {
+        long inSegmentPos = pos - segmentOffset;
+        return inSegmentPos >= 0 && inSegmentPos < buffer.capacity();
+    }
+
     public long getFilePointer()
     {
         return segmentOffset + (long)position;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
index 1b23343..623f65a 100644
--- a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
@@ -24,11 +24,17 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.RowIndexEntry;
 import org.apache.cassandra.io.FSReadError;
-import org.apache.cassandra.io.sstable.SSTableWriter;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummary;
+import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 
 public class MmappedSegmentedFile extends SegmentedFile
@@ -135,52 +141,220 @@ public class MmappedSegmentedFile extends SegmentedFile
         }
     }
 
+    // see CASSANDRA-10357
+    public static boolean maybeRepair(CFMetaData metadata, Descriptor descriptor, IndexSummary indexSummary, SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
+    {
+        boolean mayNeedRepair = false;
+        if (ibuilder instanceof Builder)
+            mayNeedRepair = ((Builder) ibuilder).mayNeedRepair(descriptor.filenameFor(Component.PRIMARY_INDEX));
+        if (dbuilder instanceof Builder)
+            mayNeedRepair |= ((Builder) dbuilder).mayNeedRepair(descriptor.filenameFor(Component.DATA));
+
+        if (mayNeedRepair)
+            forceRepair(metadata, descriptor, indexSummary, ibuilder, dbuilder);
+        return mayNeedRepair;
+    }
+
+    // if one of the index/data files have boundaries larger than we can mmap, and they were written by a version that did not guarantee correct boundaries were saved,
+    // rebuild the boundaries and save them again
+    private static void forceRepair(CFMetaData metadata, Descriptor descriptor, IndexSummary indexSummary, SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
+    {
+        if (ibuilder instanceof Builder)
+            ((Builder) ibuilder).boundaries.clear();
+        if (dbuilder instanceof Builder)
+            ((Builder) dbuilder).boundaries.clear();
+
+        try (RandomAccessFile raf = new RandomAccessFile(descriptor.filenameFor(Component.PRIMARY_INDEX), "r");)
+        {
+            long iprev = 0, dprev = 0;
+            for (int i = 0; i < indexSummary.size(); i++)
+            {
+                // first read the position in the summary, and read the corresponding position in the data file
+                long icur = indexSummary.getPosition(i);
+                raf.seek(icur);
+                ByteBufferUtil.readWithShortLength(raf);
+                RowIndexEntry rie = metadata.comparator.rowIndexEntrySerializer().deserialize(raf, descriptor.version);
+                long dcur = rie.position;
+
+                // if these positions are small enough to map out a segment from the prior version (i.e. less than 2Gb),
+                // just add these as a boundary and proceed to the next index summary record; most scenarios will be
+                // served by this, keeping the cost of rebuild to a minimum.
+
+                if (Math.max(icur - iprev , dcur - dprev) > MAX_SEGMENT_SIZE)
+                {
+                    // otherwise, loop over its index block, providing each RIE as a potential boundary for both files
+                    raf.seek(iprev);
+                    while (raf.getFilePointer() < icur)
+                    {
+                        // add the position of this record in the index file as an index file boundary
+                        ibuilder.addPotentialBoundary(raf.getFilePointer());
+                        // then read the RIE, and add its data file position as a boundary for the data file
+                        ByteBufferUtil.readWithShortLength(raf);
+                        rie = metadata.comparator.rowIndexEntrySerializer().deserialize(raf, descriptor.version);
+                        dbuilder.addPotentialBoundary(rie.position);
+                    }
+                }
+
+                ibuilder.addPotentialBoundary(icur);
+                dbuilder.addPotentialBoundary(dcur);
+
+                iprev = icur;
+                dprev = dcur;
+            }
+        }
+        catch (IOException e)
+        {
+            logger.error("Failed to recalculate boundaries for {}; mmap access may degrade to buffered for this file", descriptor);
+        }
+    }
+
     /**
      * Overrides the default behaviour to create segments of a maximum size.
      */
-    static class Builder extends SegmentedFile.Builder
+    public static class Builder extends SegmentedFile.Builder
     {
-        // planned segment boundaries
-        private List<Long> boundaries;
+        @VisibleForTesting
+        public static class Boundaries
+        {
+            private long[] boundaries;
+
+            // number of boundaries we have "fixed" (i.e. have determined the final value of)
+            private int fixedCount;
+
+            public Boundaries()
+            {
+                // we always have a boundary of zero, so we start with a fixedCount of 1
+                this(new long[8], 1);
+            }
+
+            public Boundaries(long[] boundaries, int fixedCount)
+            {
+                init(boundaries, fixedCount);
+            }
+
+            void init(long[] boundaries, int fixedCount)
+            {
+                this.boundaries = boundaries;
+                this.fixedCount = fixedCount;
+            }
+
+            public void addCandidate(long candidate)
+            {
+                // we make sure we have room before adding another element, so that we can share the addCandidate logic statically
+                boundaries = ensureCapacity(boundaries, fixedCount);
+                fixedCount = addCandidate(boundaries, fixedCount, candidate);
+            }
+
+            private static int addCandidate(long[] boundaries, int fixedCount, long candidate)
+            {
+                // check how far we are from the last fixed boundary
+                long delta = candidate - boundaries[fixedCount - 1];
+                assert delta >= 0;
+                if (delta != 0)
+                {
+                    if (delta <= MAX_SEGMENT_SIZE)
+                        // overwrite the unfixed (potential) boundary if the resultant segment would still be mmappable
+                        boundaries[fixedCount] = candidate;
+                    else if (boundaries[fixedCount] == 0)
+                        // or, if it is not initialised, we cannot make an mmapped segment here, so this is the fixed boundary
+                        boundaries[fixedCount++] = candidate;
+                    else
+                        // otherwise, fix the prior boundary and initialise our unfixed boundary
+                        boundaries[++fixedCount] = candidate;
+                }
+                return fixedCount;
+            }
+
+            // ensures there is room for another fixed boundary AND an unfixed candidate boundary, i.e. fixedCount + 2 items
+            private static long[] ensureCapacity(long[] boundaries, int fixedCount)
+            {
+                if (fixedCount + 1 >= boundaries.length)
+                    return Arrays.copyOf(boundaries, boundaries.length * 2);
+                return boundaries;
+            }
+
+            void clear()
+            {
+                fixedCount = 1;
+                Arrays.fill(boundaries, 0);
+            }
+
+            // returns the fixed boundaries, truncated to a correctly sized long[]
+            public long[] truncate()
+            {
+                return Arrays.copyOf(boundaries, fixedCount);
+            }
 
-        // offset of the open segment (first segment begins at 0).
-        private long currentStart = 0;
+            // returns the finished boundaries for the provided length, truncated to a correctly sized long[]
+            public long[] finish(long length, boolean isFinal)
+            {
+                assert length > 0;
+                // ensure there's room for the length to be added
+                boundaries = ensureCapacity(boundaries, fixedCount);
+
+                // clone our current contents, so we don't corrupt them
+                int fixedCount = this.fixedCount;
+                long[] boundaries = this.boundaries.clone();
+
+                // if we're finishing early, our length may be before some of our boundaries,
+                // so walk backwards until our boundaries are <= length
+                while (boundaries[fixedCount - 1] >= length)
+                    boundaries[fixedCount--] = 0;
+                if (boundaries[fixedCount] >= length)
+                    boundaries[fixedCount] = 0;
+
+                // add our length as a boundary
+                fixedCount = addCandidate(boundaries, fixedCount, length);
+
+                // if we have any unfixed boundary at the end, it's now fixed, since we're done
+                if (boundaries[fixedCount] != 0)
+                    fixedCount++;
+
+                boundaries = Arrays.copyOf(boundaries, fixedCount);
+                if (isFinal)
+                {
+                    // if this is the final one, save it
+                    this.boundaries = boundaries;
+                    this.fixedCount = fixedCount;
+                }
+                return boundaries;
+            }
+        }
 
-        // current length of the open segment.
-        // used to allow merging multiple too-large-to-mmap segments, into a single buffered segment.
-        private long currentSize = 0;
+        private final Boundaries boundaries = new Boundaries();
 
         public Builder()
         {
             super();
-            boundaries = new ArrayList<>();
-            boundaries.add(0L);
         }
 
-        public void addPotentialBoundary(long boundary)
+        public long[] boundaries()
         {
-            if (boundary - currentStart <= MAX_SEGMENT_SIZE)
-            {
-                // boundary fits into current segment: expand it
-                currentSize = boundary - currentStart;
-                return;
-            }
+            return boundaries.truncate();
+        }
 
-            // close the current segment to try and make room for the boundary
-            if (currentSize > 0)
-            {
-                currentStart += currentSize;
-                boundaries.add(currentStart);
-            }
-            currentSize = boundary - currentStart;
+        // indicates if we may need to repair the mmapped file boundaries. this is a cheap check to see if there
+        // are any spans larger than an mmap segment size, which should be rare to occur in practice.
+        boolean mayNeedRepair(String path)
+        {
+            // old boundaries were created without the length, so add it as a candidate
+            long length = new File(path).length();
+            boundaries.addCandidate(length);
+            long[] boundaries = this.boundaries.truncate();
 
-            // if we couldn't make room, the boundary needs its own segment
-            if (currentSize > MAX_SEGMENT_SIZE)
+            long prev = 0;
+            for (long boundary : boundaries)
             {
-                currentStart = boundary;
-                boundaries.add(currentStart);
-                currentSize = 0;
+                if (boundary - prev > MAX_SEGMENT_SIZE)
+                    return true;
+                prev = boundary;
             }
+            return false;
+        }
+
+        public void addPotentialBoundary(long boundary)
+        {
+            boundaries.addCandidate(boundary);
         }
 
         public SegmentedFile complete(String path, long overrideLength, boolean isFinal)
@@ -188,10 +362,10 @@ public class MmappedSegmentedFile extends SegmentedFile
             assert !isFinal || overrideLength <= 0;
             long length = overrideLength > 0 ? overrideLength : new File(path).length();
             // create the segments
-            return new MmappedSegmentedFile(path, length, createSegments(path, length));
+            return new MmappedSegmentedFile(path, length, createSegments(path, length, isFinal));
         }
 
-        private Segment[] createSegments(String path, long length)
+        private Segment[] createSegments(String path, long length, boolean isFinal)
         {
             RandomAccessFile raf;
             try
@@ -203,27 +377,17 @@ public class MmappedSegmentedFile extends SegmentedFile
                 throw new RuntimeException(e);
             }
 
-            // if we're early finishing a range that doesn't span multiple segments, but the finished file now does,
-            // we remove these from the end (we loop incase somehow this spans multiple segments, but that would
-            // be a loco dataset
-            while (length < boundaries.get(boundaries.size() - 1))
-                boundaries.remove(boundaries.size() -1);
-
-            // add a sentinel value == length
-            List<Long> boundaries = new ArrayList<>(this.boundaries);
-            if (length != boundaries.get(boundaries.size() - 1))
-                boundaries.add(length);
-
+            long[] boundaries = this.boundaries.finish(length, isFinal);
 
-            int segcount = boundaries.size() - 1;
+            int segcount = boundaries.length - 1;
             Segment[] segments = new Segment[segcount];
 
             try
             {
                 for (int i = 0; i < segcount; i++)
                 {
-                    long start = boundaries.get(i);
-                    long size = boundaries.get(i + 1) - start;
+                    long start = boundaries[i];
+                    long size = boundaries[i + 1] - start;
                     MappedByteBuffer segment = size <= MAX_SEGMENT_SIZE
                                                ? raf.getChannel().map(FileChannel.MapMode.READ_ONLY, start, size)
                                                : null;
@@ -245,9 +409,10 @@ public class MmappedSegmentedFile extends SegmentedFile
         public void serializeBounds(DataOutput out) throws IOException
         {
             super.serializeBounds(out);
-            out.writeInt(boundaries.size());
-            for (long position: boundaries)
-                out.writeLong(position);
+            long[] boundaries = this.boundaries.truncate();
+            out.writeInt(boundaries.length);
+            for (long boundary : boundaries)
+                out.writeLong(boundary);
         }
 
         @Override
@@ -256,12 +421,11 @@ public class MmappedSegmentedFile extends SegmentedFile
             super.deserializeBounds(in);
 
             int size = in.readInt();
-            List<Long> temp = new ArrayList<>(size);
-            
+            long[] boundaries = new long[size];
             for (int i = 0; i < size; i++)
-                temp.add(in.readLong());
+                boundaries[i] = in.readLong();
 
-            boundaries = temp;
+            this.boundaries.init(boundaries, size);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/src/java/org/apache/cassandra/io/util/SegmentedFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/SegmentedFile.java b/src/java/org/apache/cassandra/io/util/SegmentedFile.java
index c65ecbf..23454bc 100644
--- a/src/java/org/apache/cassandra/io/util/SegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/SegmentedFile.java
@@ -23,6 +23,7 @@ import java.io.File;
 import java.io.IOException;
 import java.nio.MappedByteBuffer;
 import java.util.Iterator;
+import java.util.List;
 import java.util.NoSuchElementException;
 
 import com.google.common.util.concurrent.RateLimiter;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c37562e3/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
----------------------------------------------------------------------
diff --git a/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java b/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
new file mode 100644
index 0000000..e17c6a7
--- /dev/null
+++ b/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
@@ -0,0 +1,322 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied.  See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+package org.apache.cassandra.io.sstable;
+
+import java.io.*;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Random;
+
+import com.google.common.io.Files;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.MmappedSegmentedFile;
+import org.apache.cassandra.io.util.MmappedSegmentedFile.Builder.Boundaries;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class LongSegmentedFileBoundaryTest
+{
+    @BeforeClass
+    public static void setup() throws Exception
+    {
+        SchemaLoader.cleanupAndLeaveDirs();
+        Keyspace.setInitialized();
+        StorageService.instance.initServer();
+    }
+
+    @AfterClass
+    public static void tearDown()
+    {
+        Config.setClientMode(false);
+    }
+
+    @Test
+    public void testRandomBoundaries()
+    {
+        long[] candidates = new long[1 + (1 << 16)];
+        int[] indexesToCheck = new int[1 << 8];
+        Random random = new Random();
+
+        for (int run = 0; run < 100; run++)
+        {
+
+            long seed = random.nextLong();
+            random.setSeed(seed);
+            System.out.println("Seed: " + seed);
+
+            // at least 1Ki, and as many as 256Ki, boundaries
+            int candidateCount = (1 + random.nextInt(candidates.length >> 10)) << 10;
+            generateBoundaries(random, candidateCount, candidates, indexesToCheck);
+
+            Boundaries builder = new Boundaries();
+            int nextIndexToCheck = indexesToCheck[0];
+            int checkCount = 0;
+            System.out.printf("[0..%d)", candidateCount);
+            for (int i = 1; i < candidateCount - 1; i++)
+            {
+                if (i == nextIndexToCheck)
+                {
+                    if (checkCount % 20 == 0)
+                        System.out.printf(" %d", i);
+                    // grow number of samples logarithmically; work will still increase superlinearly, as size of dataset grows linearly
+                    int sampleCount = 1 << (31 - Integer.numberOfLeadingZeros(++checkCount));
+                    checkBoundarySample(random, candidates, i, sampleCount, builder);
+                    // select out next index to check (there may be dups, so skip them)
+                    while ((nextIndexToCheck = checkCount == indexesToCheck.length ? candidateCount : indexesToCheck[checkCount]) == i)
+                        checkCount++;
+                }
+
+                builder.addCandidate(candidates[i]);
+            }
+            System.out.println();
+            checkBoundaries(candidates, candidateCount - 1, builder, candidates[candidateCount - 1]);
+            Assert.assertEquals(candidateCount, nextIndexToCheck);
+        }
+    }
+
+    private static void generateBoundaries(Random random, int candidateCount, long[] candidates, int[] indexesToCheck)
+    {
+        // average averageBoundarySize is 4MiB, max 4GiB, min 4KiB
+        long averageBoundarySize = (4L << 10) * random.nextInt(1 << 20);
+        long prev = 0;
+        for (int i = 1 ; i < candidateCount ; i++)
+            candidates[i] = prev += Math.max(1, averageBoundarySize + (random.nextGaussian() * averageBoundarySize));
+
+        // generate indexes we will corroborate our behaviour on
+        for (int i = 0 ; i < indexesToCheck.length ; i++)
+            indexesToCheck[i] = 1 + random.nextInt(candidateCount - 2);
+        Arrays.sort(indexesToCheck);
+    }
+
+    private static void checkBoundarySample(Random random, long[] candidates, int candidateCount, int sampleCount, Boundaries builder)
+    {
+        for (int i = 0 ; i < sampleCount ; i++)
+        {
+            // pick a number exponentially less likely to be near the beginning, since we test that area earlier
+            int position = 0 ;
+            while (position <= 0)
+                position = candidateCount / (Integer.lowestOneBit(random.nextInt()));
+            long upperBound = candidates[position];
+            long lowerBound = random.nextBoolean() ? (rand(random, 0, upperBound) / (Integer.lowestOneBit(random.nextInt())))
+                                                   : candidates[Math.max(0, position - random.nextInt(64))];
+            long length = rand(random, lowerBound, upperBound);
+            checkBoundaries(candidates, candidateCount, builder, length);
+        }
+        checkBoundaries(candidates, candidateCount, builder, candidates[candidateCount]);
+    }
+
+    private static long rand(Random random, long lowerBound, long upperBound)
+    {
+        if (upperBound == lowerBound)
+            return upperBound;
+        return lowerBound + ((random.nextLong() & Long.MAX_VALUE) % (upperBound - lowerBound));
+    }
+
+    private static void checkBoundaries(long[] candidates, int candidateCount, Boundaries builder, long length)
+    {
+        if (length == 0)
+            return;
+
+        long[] boundaries = new long[(int) (10 + 2 * (length / Integer.MAX_VALUE))];
+        int count = 1;
+        int prev = 0;
+        while (true)
+        {
+            int p = candidates[prev + 1] - boundaries[count - 1] >= Integer.MAX_VALUE
+                    ? prev + 1
+                    : Arrays.binarySearch(candidates, prev, candidateCount, boundaries[count - 1] + Integer.MAX_VALUE);
+            if (p < 0) p = -2 -p;
+            if (p >= candidateCount - 1 || candidates[p] >= length)
+                break;
+            boundaries[count++] = candidates[p];
+            if (candidates[p + 1] >= length)
+                break;
+            prev = p;
+        }
+        if (candidates[candidateCount - 1] < length && length - boundaries[count - 1] >= Integer.MAX_VALUE)
+            boundaries[count++] = candidates[candidateCount - 1];
+        boundaries[count++] = length;
+        final long[] canon = Arrays.copyOf(boundaries, count);
+        final long[] check = builder.finish(length, false);
+        if (!Arrays.equals(canon, check))
+            Assert.assertTrue("\n" + Arrays.toString(canon) + "\n" + Arrays.toString(check), Arrays.equals(canon, check));
+    }
+
+    @Test
+    public void testBoundariesAndRepairSmall() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1, 1 << 16);
+    }
+
+    @Test
+    public void testBoundariesAndRepairMedium() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1, 1 << 20);
+    }
+
+    @Test
+    public void testBoundariesAndRepairLarge() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1, 100 << 20);
+    }
+
+    @Test
+    public void testBoundariesAndRepairHuge() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1, Integer.MAX_VALUE - 1024);
+    }
+
+    @Test
+    public void testBoundariesAndRepairTooHuge() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1, Integer.MAX_VALUE);
+    }
+
+    @Test
+    public void testBoundariesAndRepairHugeIndex() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1 << 7, 1 << 15);
+    }
+
+    @Test
+    public void testBoundariesAndRepairReallyHugeIndex() throws InvalidRequestException, IOException
+    {
+        testBoundariesAndRepair(1 << 14, 1 << 15);
+    }
+
+    private void testBoundariesAndRepair(int rows, int rowSize) throws InvalidRequestException, IOException
+    {
+        String KS = "cql_keyspace";
+        String TABLE = "table1";
+
+        File tempdir = Files.createTempDir();
+        try
+        {
+            Assert.assertTrue(DatabaseDescriptor.getColumnIndexSize() < rowSize);
+            Assert.assertTrue(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap);
+            Assert.assertTrue(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap);
+            Assert.assertTrue(StorageService.getPartitioner() instanceof ByteOrderedPartitioner);
+            File dataDir = new File(tempdir.getAbsolutePath() + File.separator + KS + File.separator + TABLE);
+            Assert.assertTrue(dataDir.mkdirs());
+
+            String schema = "CREATE TABLE cql_keyspace.table" + (rows > 1 ? "2" : "1") + " (k bigint, v1 blob, v2 blob, v3 blob, v4 blob, v5 blob, PRIMARY KEY (k" + (rows > 1 ? ", v1" : "") + ")) WITH compression = { 'sstable_compression':'' };";
+            String insert = "INSERT INTO cql_keyspace.table" + (rows > 1 ? "2" : "1") + " (k, v1, v2, v3, v4, v5) VALUES (?, ?, ?, ?, ?, ?)";
+
+            CQLSSTableWriter.Builder builder = CQLSSTableWriter.builder()
+                                                      .inDirectory(dataDir)
+                                                      .forTable(schema)
+                                                      .withPartitioner(StorageService.getPartitioner())
+                                                      .using(insert)
+                                                      .sorted();
+            CQLSSTableWriter writer = builder.build();
+
+            // write 8Gb of decorated keys
+            ByteBuffer[] value = new ByteBuffer[rows];
+            for (int row = 0 ; row < rows ; row++)
+            {
+                // if we're using clustering columns, the clustering key is replicated across every other column
+                value[row] = ByteBuffer.allocate(rowSize / (rows > 1 ? 8 : 5));
+                value[row].putInt(0, row);
+            }
+            long targetSize = 8L << 30;
+            long dk = 0;
+            long size = 0;
+            long dkSize = rowSize * rows;
+            while (size < targetSize)
+            {
+                for (int row = 0 ; row < rows ; row++)
+                    writer.addRow(dk, value[row], value[row], value[row], value[row], value[row]);
+                size += dkSize;
+                dk++;
+            }
+
+            Descriptor descriptor = writer.getCurrentDescriptor().asType(Descriptor.Type.FINAL);
+            writer.close();
+
+            // open (and close) the reader so that the summary file is created
+            SSTableReader reader = SSTableReader.open(descriptor);
+            reader.selfRef().release();
+
+            // then check the boundaries are reasonable, and corrupt them
+            checkThenCorruptBoundaries(descriptor, rows * rowSize < Integer.MAX_VALUE);
+
+            // then check that reopening corrects the corruption
+            reader = SSTableReader.open(descriptor);
+            reader.selfRef().release();
+            checkThenCorruptBoundaries(descriptor, rows * rowSize < Integer.MAX_VALUE);
+        }
+        finally
+        {
+            FileUtils.deleteRecursive(tempdir);
+        }
+    }
+
+    private static void checkThenCorruptBoundaries(Descriptor descriptor, boolean expectDataMmappable) throws IOException
+    {
+        File summaryFile = new File(descriptor.filenameFor(Component.SUMMARY));
+        DataInputStream iStream = new DataInputStream(new FileInputStream(summaryFile));
+        IndexSummary indexSummary = IndexSummary.serializer.deserialize(iStream, StorageService.getPartitioner(), true, CFMetaData.DEFAULT_MIN_INDEX_INTERVAL, CFMetaData.DEFAULT_MAX_INDEX_INTERVAL);
+        ByteBuffer first = ByteBufferUtil.readWithLength(iStream);
+        ByteBuffer last = ByteBufferUtil.readWithLength(iStream);
+        MmappedSegmentedFile.Builder ibuilder = new MmappedSegmentedFile.Builder();
+        MmappedSegmentedFile.Builder dbuilder = new MmappedSegmentedFile.Builder();
+        ibuilder.deserializeBounds(iStream);
+        dbuilder.deserializeBounds(iStream);
+        iStream.close();
+        // index file cannot generally be non-mmappable, as index entries cannot be larger than MAX_SEGMENT_SIZE (due to promotedSize being encoded as an int)
+        assertBoundaries(descriptor.filenameFor(Component.PRIMARY_INDEX), true, ibuilder.boundaries());
+        assertBoundaries(descriptor.filenameFor(Component.DATA), expectDataMmappable, dbuilder.boundaries());
+
+        DataOutputStreamPlus oStream = new DataOutputStreamPlus(new FileOutputStream(summaryFile));
+        IndexSummary.serializer.serialize(indexSummary, oStream, true);
+        ByteBufferUtil.writeWithLength(first, oStream);
+        ByteBufferUtil.writeWithLength(last, oStream);
+        oStream.writeInt(1);
+        oStream.writeLong(new File(descriptor.filenameFor(Component.PRIMARY_INDEX)).length());
+        oStream.writeLong(new File(descriptor.filenameFor(Component.DATA)).length());
+        oStream.close();
+    }
+
+    private static void assertBoundaries(String path, boolean expectMmappable, long[] boundaries)
+    {
+        long length = new File(path).length();
+        long prev = boundaries[0];
+        for (int i = 1 ; i <= boundaries.length && prev < length ; i++)
+        {
+            long boundary = i == boundaries.length ? length : boundaries[i];
+            Assert.assertEquals(String.format("[%d, %d), %d of %d", boundary, prev, i, boundaries.length),
+                                expectMmappable, boundary - prev <= Integer.MAX_VALUE);
+            prev = boundary;
+        }
+    }
+
+}


[15/16] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by be...@apache.org.
Merge branch 'cassandra-2.2' into cassandra-3.0


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

Branch: refs/heads/trunk
Commit: e3d58448b6cadef7be060bb35996cd86d9ec9d59
Parents: a039b7d 25de92e
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Wed Sep 30 19:47:39 2015 +0100
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Wed Sep 30 19:47:39 2015 +0100

----------------------------------------------------------------------

----------------------------------------------------------------------



[08/16] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/util/ByteBufferDataInput.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/util/ByteBufferDataInput.java
index bf926e9,0000000..2f00687
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/io/util/ByteBufferDataInput.java
+++ b/src/java/org/apache/cassandra/io/util/ByteBufferDataInput.java
@@@ -1,171 -1,0 +1,177 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.io.util;
 +
 +import java.io.*;
 +import java.nio.ByteBuffer;
 +
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +
 +public class ByteBufferDataInput extends AbstractDataInput implements FileDataInput, DataInput
 +{
 +    private final ByteBuffer buffer;
 +    private final String filename;
 +    private final long segmentOffset;
 +    private int position;
 +
 +    public ByteBufferDataInput(ByteBuffer buffer, String filename, long segmentOffset, int position)
 +    {
 +        assert buffer != null;
 +        this.buffer = buffer;
 +        this.filename = filename;
 +        this.segmentOffset = segmentOffset;
 +        this.position = position;
 +    }
 +
 +    // Only use when we know the seek in within the mapped segment. Throws an
 +    // IOException otherwise.
 +    public void seek(long pos) throws IOException
 +    {
 +        long inSegmentPos = pos - segmentOffset;
-         if (inSegmentPos < 0 || inSegmentPos > buffer.capacity())
++        if (!contains(pos))
 +            throw new IOException(String.format("Seek position %d is not within mmap segment (seg offs: %d, length: %d)", pos, segmentOffset, buffer.capacity()));
 +
 +        position = (int) inSegmentPos;
 +    }
 +
++    public boolean contains(long pos)
++    {
++        long inSegmentPos = pos - segmentOffset;
++        return inSegmentPos >= 0 && inSegmentPos < buffer.capacity();
++    }
++
 +    public long getFilePointer()
 +    {
 +        return segmentOffset + position;
 +    }
 +
 +    public long getPosition()
 +    {
 +        return segmentOffset + position;
 +    }
 +
 +    public long getPositionLimit()
 +    {
 +        return segmentOffset + buffer.capacity();
 +    }
 +
 +    @Override
 +    public boolean markSupported()
 +    {
 +        return false;
 +    }
 +
 +    public void reset(FileMark mark) throws IOException
 +    {
 +        assert mark instanceof MappedFileDataInputMark;
 +        position = ((MappedFileDataInputMark) mark).position;
 +    }
 +
 +    public FileMark mark()
 +    {
 +        return new MappedFileDataInputMark(position);
 +    }
 +
 +    public long bytesPastMark(FileMark mark)
 +    {
 +        assert mark instanceof MappedFileDataInputMark;
 +        assert position >= ((MappedFileDataInputMark) mark).position;
 +        return position - ((MappedFileDataInputMark) mark).position;
 +    }
 +
 +    public boolean isEOF() throws IOException
 +    {
 +        return position == buffer.capacity();
 +    }
 +
 +    public long bytesRemaining() throws IOException
 +    {
 +        return buffer.capacity() - position;
 +    }
 +
 +    public String getPath()
 +    {
 +        return filename;
 +    }
 +
 +    public int read() throws IOException
 +    {
 +        if (isEOF())
 +            return -1;
 +        return buffer.get(position++) & 0xFF;
 +    }
 +
 +    /**
 +     * Does the same thing as <code>readFully</code> do but without copying data (thread safe)
 +     * @param length length of the bytes to read
 +     * @return buffer with portion of file content
 +     * @throws IOException on any fail of I/O operation
 +     */
 +    public ByteBuffer readBytes(int length) throws IOException
 +    {
 +        int remaining = buffer.remaining() - position;
 +        if (length > remaining)
 +            throw new IOException(String.format("mmap segment underflow; remaining is %d but %d requested",
 +                                                remaining, length));
 +
 +        if (length == 0)
 +            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
 +
 +        ByteBuffer bytes = buffer.duplicate();
 +        bytes.position(buffer.position() + position).limit(buffer.position() + position + length);
 +        position += length;
 +
 +        // we have to copy the data in case we unreference the underlying sstable.  See CASSANDRA-3179
 +        ByteBuffer clone = ByteBuffer.allocate(bytes.remaining());
 +        clone.put(bytes);
 +        clone.flip();
 +        return clone;
 +    }
 +
 +    @Override
 +    public final void readFully(byte[] bytes) throws IOException
 +    {
 +        ByteBufferUtil.arrayCopy(buffer, buffer.position() + position, bytes, 0, bytes.length);
 +        position += bytes.length;
 +    }
 +
 +    @Override
 +    public final void readFully(byte[] bytes, int offset, int count) throws IOException
 +    {
 +        ByteBufferUtil.arrayCopy(buffer, buffer.position() + position, bytes, offset, count);
 +        position += count;
 +    }
 +
 +    private static class MappedFileDataInputMark implements FileMark
 +    {
 +        int position;
 +
 +        MappedFileDataInputMark(int position)
 +        {
 +            this.position = position;
 +        }
 +    }
 +
 +    @Override
 +    public String toString() {
 +        return getClass().getSimpleName() + "(" +
 +               "filename='" + filename + "'" +
 +               ", position=" + position +
 +               ")";
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
index a8fae9f,623f65a..808b5ad
--- a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
@@@ -27,6 -28,13 +28,12 @@@ import com.google.common.annotations.Vi
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
+ import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.db.RowIndexEntry;
 -import org.apache.cassandra.io.FSReadError;
+ import org.apache.cassandra.io.sstable.Component;
+ import org.apache.cassandra.io.sstable.Descriptor;
+ import org.apache.cassandra.io.sstable.IndexSummary;
+ import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.utils.JVMStabilityInspector;
  
  public class MmappedSegmentedFile extends SegmentedFile
@@@ -135,6 -141,73 +142,74 @@@
          }
      }
  
+     // see CASSANDRA-10357
+     public static boolean maybeRepair(CFMetaData metadata, Descriptor descriptor, IndexSummary indexSummary, SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
+     {
+         boolean mayNeedRepair = false;
+         if (ibuilder instanceof Builder)
+             mayNeedRepair = ((Builder) ibuilder).mayNeedRepair(descriptor.filenameFor(Component.PRIMARY_INDEX));
+         if (dbuilder instanceof Builder)
+             mayNeedRepair |= ((Builder) dbuilder).mayNeedRepair(descriptor.filenameFor(Component.DATA));
+ 
+         if (mayNeedRepair)
+             forceRepair(metadata, descriptor, indexSummary, ibuilder, dbuilder);
+         return mayNeedRepair;
+     }
+ 
+     // if one of the index/data files have boundaries larger than we can mmap, and they were written by a version that did not guarantee correct boundaries were saved,
+     // rebuild the boundaries and save them again
+     private static void forceRepair(CFMetaData metadata, Descriptor descriptor, IndexSummary indexSummary, SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
+     {
+         if (ibuilder instanceof Builder)
+             ((Builder) ibuilder).boundaries.clear();
+         if (dbuilder instanceof Builder)
+             ((Builder) dbuilder).boundaries.clear();
+ 
++        RowIndexEntry.IndexSerializer rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata);
+         try (RandomAccessFile raf = new RandomAccessFile(descriptor.filenameFor(Component.PRIMARY_INDEX), "r");)
+         {
+             long iprev = 0, dprev = 0;
+             for (int i = 0; i < indexSummary.size(); i++)
+             {
+                 // first read the position in the summary, and read the corresponding position in the data file
+                 long icur = indexSummary.getPosition(i);
+                 raf.seek(icur);
+                 ByteBufferUtil.readWithShortLength(raf);
 -                RowIndexEntry rie = metadata.comparator.rowIndexEntrySerializer().deserialize(raf, descriptor.version);
++                RowIndexEntry rie = rowIndexEntrySerializer.deserialize(raf, descriptor.version);
+                 long dcur = rie.position;
+ 
+                 // if these positions are small enough to map out a segment from the prior version (i.e. less than 2Gb),
+                 // just add these as a boundary and proceed to the next index summary record; most scenarios will be
+                 // served by this, keeping the cost of rebuild to a minimum.
+ 
+                 if (Math.max(icur - iprev , dcur - dprev) > MAX_SEGMENT_SIZE)
+                 {
+                     // otherwise, loop over its index block, providing each RIE as a potential boundary for both files
+                     raf.seek(iprev);
+                     while (raf.getFilePointer() < icur)
+                     {
+                         // add the position of this record in the index file as an index file boundary
+                         ibuilder.addPotentialBoundary(raf.getFilePointer());
+                         // then read the RIE, and add its data file position as a boundary for the data file
+                         ByteBufferUtil.readWithShortLength(raf);
 -                        rie = metadata.comparator.rowIndexEntrySerializer().deserialize(raf, descriptor.version);
++                        rie = rowIndexEntrySerializer.deserialize(raf, descriptor.version);
+                         dbuilder.addPotentialBoundary(rie.position);
+                     }
+                 }
+ 
+                 ibuilder.addPotentialBoundary(icur);
+                 dbuilder.addPotentialBoundary(dcur);
+ 
+                 iprev = icur;
+                 dprev = dcur;
+             }
+         }
+         catch (IOException e)
+         {
+             logger.error("Failed to recalculate boundaries for {}; mmap access may degrade to buffered for this file", descriptor);
+         }
+     }
+ 
      /**
       * Overrides the default behaviour to create segments of a maximum size.
       */
@@@ -153,68 -326,83 +328,58 @@@
          public Builder()
          {
              super();
-             boundaries = new ArrayList<>();
-             boundaries.add(0L);
          }
  
-         public void addPotentialBoundary(long boundary)
+         public long[] boundaries()
          {
-             if (boundary - currentStart <= MAX_SEGMENT_SIZE)
-             {
-                 // boundary fits into current segment: expand it
-                 currentSize = boundary - currentStart;
-                 return;
-             }
+             return boundaries.truncate();
+         }
  
-             // close the current segment to try and make room for the boundary
-             if (currentSize > 0)
-             {
-                 currentStart += currentSize;
-                 boundaries.add(currentStart);
-             }
-             currentSize = boundary - currentStart;
+         // indicates if we may need to repair the mmapped file boundaries. this is a cheap check to see if there
+         // are any spans larger than an mmap segment size, which should be rare to occur in practice.
+         boolean mayNeedRepair(String path)
+         {
+             // old boundaries were created without the length, so add it as a candidate
+             long length = new File(path).length();
+             boundaries.addCandidate(length);
+             long[] boundaries = this.boundaries.truncate();
  
-             // if we couldn't make room, the boundary needs its own segment
-             if (currentSize > MAX_SEGMENT_SIZE)
+             long prev = 0;
+             for (long boundary : boundaries)
              {
-                 currentStart = boundary;
-                 boundaries.add(currentStart);
-                 currentSize = 0;
+                 if (boundary - prev > MAX_SEGMENT_SIZE)
+                     return true;
+                 prev = boundary;
              }
+             return false;
+         }
+ 
+         public void addPotentialBoundary(long boundary)
+         {
+             boundaries.addCandidate(boundary);
          }
  
 -        public SegmentedFile complete(String path, long overrideLength, boolean isFinal)
 +        public SegmentedFile complete(ChannelProxy channel, long overrideLength)
          {
 -            assert !isFinal || overrideLength <= 0;
 -            long length = overrideLength > 0 ? overrideLength : new File(path).length();
 +            long length = overrideLength > 0 ? overrideLength : channel.size();
              // create the segments
-             return new MmappedSegmentedFile(channel, length, createSegments(channel, length));
 -            return new MmappedSegmentedFile(path, length, createSegments(path, length, isFinal));
--        }
 -
 -        private Segment[] createSegments(String path, long length, boolean isFinal)
 -        {
 -            RandomAccessFile raf;
 -            try
 -            {
 -                raf = new RandomAccessFile(path, "r");
 -            }
 -            catch (IOException e)
 -            {
 -                throw new RuntimeException(e);
 -            }
  
-         private Segment[] createSegments(ChannelProxy channel, long length)
-         {
-             // if we're early finishing a range that doesn't span multiple segments, but the finished file now does,
-             // we remove these from the end (we loop incase somehow this spans multiple segments, but that would
-             // be a loco dataset
-             while (length < boundaries.get(boundaries.size() - 1))
-                 boundaries.remove(boundaries.size() -1);
- 
-             // add a sentinel value == length
-             List<Long> boundaries = new ArrayList<>(this.boundaries);
-             if (length != boundaries.get(boundaries.size() - 1))
-                 boundaries.add(length);
- 
-             int segcount = boundaries.size() - 1;
 -            long[] boundaries = this.boundaries.finish(length, isFinal);
++            long[] boundaries = this.boundaries.finish(length, overrideLength <= 0);
+ 
+             int segcount = boundaries.length - 1;
              Segment[] segments = new Segment[segcount];
+ 
 -            try
 -            {
 -                for (int i = 0; i < segcount; i++)
 -                {
 -                    long start = boundaries[i];
 -                    long size = boundaries[i + 1] - start;
 -                    MappedByteBuffer segment = size <= MAX_SEGMENT_SIZE
 -                                               ? raf.getChannel().map(FileChannel.MapMode.READ_ONLY, start, size)
 -                                               : null;
 -                    segments[i] = new Segment(start, segment);
 -                }
 -            }
 -            catch (IOException e)
 -            {
 -                throw new FSReadError(e, path);
 -            }
 -            finally
 +            for (int i = 0; i < segcount; i++)
              {
-                 long start = boundaries.get(i);
-                 long size = boundaries.get(i + 1) - start;
 -                FileUtils.closeQuietly(raf);
++                long start = boundaries[i];
++                long size = boundaries[i + 1] - start;
 +                MappedByteBuffer segment = size <= MAX_SEGMENT_SIZE
 +                                           ? channel.map(FileChannel.MapMode.READ_ONLY, start, size)
 +                                           : null;
 +                segments[i] = new Segment(start, segment);
              }
--            return segments;
++
++            return new MmappedSegmentedFile(channel, length, segments);
          }
  
          @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/util/SegmentedFile.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/util/SegmentedFile.java
index 66898c6,23454bc..30707d8
--- a/src/java/org/apache/cassandra/io/util/SegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/SegmentedFile.java
@@@ -23,9 -23,9 +23,10 @@@ import java.io.File
  import java.io.IOException;
  import java.nio.MappedByteBuffer;
  import java.util.Iterator;
+ import java.util.List;
  import java.util.NoSuchElementException;
  
 +import com.google.common.base.Throwables;
  import com.google.common.util.concurrent.RateLimiter;
  
  import org.apache.cassandra.config.Config;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
----------------------------------------------------------------------
diff --cc test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
index 0000000,e17c6a7..4913b32
mode 000000,100644..100644
--- a/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
+++ b/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
@@@ -1,0 -1,322 +1,324 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ package org.apache.cassandra.io.sstable;
+ 
+ import java.io.*;
+ import java.nio.ByteBuffer;
+ import java.util.Arrays;
+ import java.util.Random;
+ 
+ import com.google.common.io.Files;
+ import org.junit.AfterClass;
+ import org.junit.BeforeClass;
+ import org.junit.Test;
+ 
+ import junit.framework.Assert;
+ import org.apache.cassandra.SchemaLoader;
+ import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.config.Config;
+ import org.apache.cassandra.config.DatabaseDescriptor;
+ import org.apache.cassandra.db.Keyspace;
+ import org.apache.cassandra.dht.ByteOrderedPartitioner;
+ import org.apache.cassandra.exceptions.InvalidRequestException;
++import org.apache.cassandra.io.sstable.format.SSTableReader;
+ import org.apache.cassandra.io.util.DataOutputStreamPlus;
+ import org.apache.cassandra.io.util.FileUtils;
+ import org.apache.cassandra.io.util.MmappedSegmentedFile;
+ import org.apache.cassandra.io.util.MmappedSegmentedFile.Builder.Boundaries;
++import org.apache.cassandra.io.util.WrappedDataOutputStreamPlus;
+ import org.apache.cassandra.service.StorageService;
+ import org.apache.cassandra.utils.ByteBufferUtil;
+ 
+ public class LongSegmentedFileBoundaryTest
+ {
+     @BeforeClass
+     public static void setup() throws Exception
+     {
+         SchemaLoader.cleanupAndLeaveDirs();
+         Keyspace.setInitialized();
+         StorageService.instance.initServer();
+     }
+ 
+     @AfterClass
+     public static void tearDown()
+     {
+         Config.setClientMode(false);
+     }
+ 
+     @Test
+     public void testRandomBoundaries()
+     {
+         long[] candidates = new long[1 + (1 << 16)];
+         int[] indexesToCheck = new int[1 << 8];
+         Random random = new Random();
+ 
+         for (int run = 0; run < 100; run++)
+         {
+ 
+             long seed = random.nextLong();
+             random.setSeed(seed);
+             System.out.println("Seed: " + seed);
+ 
+             // at least 1Ki, and as many as 256Ki, boundaries
+             int candidateCount = (1 + random.nextInt(candidates.length >> 10)) << 10;
+             generateBoundaries(random, candidateCount, candidates, indexesToCheck);
+ 
+             Boundaries builder = new Boundaries();
+             int nextIndexToCheck = indexesToCheck[0];
+             int checkCount = 0;
+             System.out.printf("[0..%d)", candidateCount);
+             for (int i = 1; i < candidateCount - 1; i++)
+             {
+                 if (i == nextIndexToCheck)
+                 {
+                     if (checkCount % 20 == 0)
+                         System.out.printf(" %d", i);
+                     // grow number of samples logarithmically; work will still increase superlinearly, as size of dataset grows linearly
+                     int sampleCount = 1 << (31 - Integer.numberOfLeadingZeros(++checkCount));
+                     checkBoundarySample(random, candidates, i, sampleCount, builder);
+                     // select out next index to check (there may be dups, so skip them)
+                     while ((nextIndexToCheck = checkCount == indexesToCheck.length ? candidateCount : indexesToCheck[checkCount]) == i)
+                         checkCount++;
+                 }
+ 
+                 builder.addCandidate(candidates[i]);
+             }
+             System.out.println();
+             checkBoundaries(candidates, candidateCount - 1, builder, candidates[candidateCount - 1]);
+             Assert.assertEquals(candidateCount, nextIndexToCheck);
+         }
+     }
+ 
+     private static void generateBoundaries(Random random, int candidateCount, long[] candidates, int[] indexesToCheck)
+     {
+         // average averageBoundarySize is 4MiB, max 4GiB, min 4KiB
+         long averageBoundarySize = (4L << 10) * random.nextInt(1 << 20);
+         long prev = 0;
+         for (int i = 1 ; i < candidateCount ; i++)
+             candidates[i] = prev += Math.max(1, averageBoundarySize + (random.nextGaussian() * averageBoundarySize));
+ 
+         // generate indexes we will corroborate our behaviour on
+         for (int i = 0 ; i < indexesToCheck.length ; i++)
+             indexesToCheck[i] = 1 + random.nextInt(candidateCount - 2);
+         Arrays.sort(indexesToCheck);
+     }
+ 
+     private static void checkBoundarySample(Random random, long[] candidates, int candidateCount, int sampleCount, Boundaries builder)
+     {
+         for (int i = 0 ; i < sampleCount ; i++)
+         {
+             // pick a number exponentially less likely to be near the beginning, since we test that area earlier
+             int position = 0 ;
+             while (position <= 0)
+                 position = candidateCount / (Integer.lowestOneBit(random.nextInt()));
+             long upperBound = candidates[position];
+             long lowerBound = random.nextBoolean() ? (rand(random, 0, upperBound) / (Integer.lowestOneBit(random.nextInt())))
+                                                    : candidates[Math.max(0, position - random.nextInt(64))];
+             long length = rand(random, lowerBound, upperBound);
+             checkBoundaries(candidates, candidateCount, builder, length);
+         }
+         checkBoundaries(candidates, candidateCount, builder, candidates[candidateCount]);
+     }
+ 
+     private static long rand(Random random, long lowerBound, long upperBound)
+     {
+         if (upperBound == lowerBound)
+             return upperBound;
+         return lowerBound + ((random.nextLong() & Long.MAX_VALUE) % (upperBound - lowerBound));
+     }
+ 
+     private static void checkBoundaries(long[] candidates, int candidateCount, Boundaries builder, long length)
+     {
+         if (length == 0)
+             return;
+ 
+         long[] boundaries = new long[(int) (10 + 2 * (length / Integer.MAX_VALUE))];
+         int count = 1;
+         int prev = 0;
+         while (true)
+         {
+             int p = candidates[prev + 1] - boundaries[count - 1] >= Integer.MAX_VALUE
+                     ? prev + 1
+                     : Arrays.binarySearch(candidates, prev, candidateCount, boundaries[count - 1] + Integer.MAX_VALUE);
+             if (p < 0) p = -2 -p;
+             if (p >= candidateCount - 1 || candidates[p] >= length)
+                 break;
+             boundaries[count++] = candidates[p];
+             if (candidates[p + 1] >= length)
+                 break;
+             prev = p;
+         }
+         if (candidates[candidateCount - 1] < length && length - boundaries[count - 1] >= Integer.MAX_VALUE)
+             boundaries[count++] = candidates[candidateCount - 1];
+         boundaries[count++] = length;
+         final long[] canon = Arrays.copyOf(boundaries, count);
+         final long[] check = builder.finish(length, false);
+         if (!Arrays.equals(canon, check))
+             Assert.assertTrue("\n" + Arrays.toString(canon) + "\n" + Arrays.toString(check), Arrays.equals(canon, check));
+     }
+ 
+     @Test
+     public void testBoundariesAndRepairSmall() throws InvalidRequestException, IOException
+     {
+         testBoundariesAndRepair(1, 1 << 16);
+     }
+ 
+     @Test
+     public void testBoundariesAndRepairMedium() throws InvalidRequestException, IOException
+     {
+         testBoundariesAndRepair(1, 1 << 20);
+     }
+ 
+     @Test
+     public void testBoundariesAndRepairLarge() throws InvalidRequestException, IOException
+     {
+         testBoundariesAndRepair(1, 100 << 20);
+     }
+ 
+     @Test
+     public void testBoundariesAndRepairHuge() throws InvalidRequestException, IOException
+     {
+         testBoundariesAndRepair(1, Integer.MAX_VALUE - 1024);
+     }
+ 
+     @Test
+     public void testBoundariesAndRepairTooHuge() throws InvalidRequestException, IOException
+     {
+         testBoundariesAndRepair(1, Integer.MAX_VALUE);
+     }
+ 
+     @Test
+     public void testBoundariesAndRepairHugeIndex() throws InvalidRequestException, IOException
+     {
+         testBoundariesAndRepair(1 << 7, 1 << 15);
+     }
+ 
+     @Test
+     public void testBoundariesAndRepairReallyHugeIndex() throws InvalidRequestException, IOException
+     {
+         testBoundariesAndRepair(1 << 14, 1 << 15);
+     }
+ 
+     private void testBoundariesAndRepair(int rows, int rowSize) throws InvalidRequestException, IOException
+     {
+         String KS = "cql_keyspace";
+         String TABLE = "table1";
+ 
+         File tempdir = Files.createTempDir();
+         try
+         {
+             Assert.assertTrue(DatabaseDescriptor.getColumnIndexSize() < rowSize);
+             Assert.assertTrue(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap);
+             Assert.assertTrue(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap);
+             Assert.assertTrue(StorageService.getPartitioner() instanceof ByteOrderedPartitioner);
+             File dataDir = new File(tempdir.getAbsolutePath() + File.separator + KS + File.separator + TABLE);
+             Assert.assertTrue(dataDir.mkdirs());
+ 
+             String schema = "CREATE TABLE cql_keyspace.table" + (rows > 1 ? "2" : "1") + " (k bigint, v1 blob, v2 blob, v3 blob, v4 blob, v5 blob, PRIMARY KEY (k" + (rows > 1 ? ", v1" : "") + ")) WITH compression = { 'sstable_compression':'' };";
+             String insert = "INSERT INTO cql_keyspace.table" + (rows > 1 ? "2" : "1") + " (k, v1, v2, v3, v4, v5) VALUES (?, ?, ?, ?, ?, ?)";
+ 
+             CQLSSTableWriter.Builder builder = CQLSSTableWriter.builder()
+                                                       .inDirectory(dataDir)
+                                                       .forTable(schema)
+                                                       .withPartitioner(StorageService.getPartitioner())
+                                                       .using(insert)
+                                                       .sorted();
+             CQLSSTableWriter writer = builder.build();
+ 
+             // write 8Gb of decorated keys
+             ByteBuffer[] value = new ByteBuffer[rows];
+             for (int row = 0 ; row < rows ; row++)
+             {
+                 // if we're using clustering columns, the clustering key is replicated across every other column
+                 value[row] = ByteBuffer.allocate(rowSize / (rows > 1 ? 8 : 5));
+                 value[row].putInt(0, row);
+             }
+             long targetSize = 8L << 30;
+             long dk = 0;
+             long size = 0;
+             long dkSize = rowSize * rows;
+             while (size < targetSize)
+             {
+                 for (int row = 0 ; row < rows ; row++)
+                     writer.addRow(dk, value[row], value[row], value[row], value[row], value[row]);
+                 size += dkSize;
+                 dk++;
+             }
+ 
+             Descriptor descriptor = writer.getCurrentDescriptor().asType(Descriptor.Type.FINAL);
+             writer.close();
+ 
+             // open (and close) the reader so that the summary file is created
+             SSTableReader reader = SSTableReader.open(descriptor);
+             reader.selfRef().release();
+ 
+             // then check the boundaries are reasonable, and corrupt them
+             checkThenCorruptBoundaries(descriptor, rows * rowSize < Integer.MAX_VALUE);
+ 
+             // then check that reopening corrects the corruption
+             reader = SSTableReader.open(descriptor);
+             reader.selfRef().release();
+             checkThenCorruptBoundaries(descriptor, rows * rowSize < Integer.MAX_VALUE);
+         }
+         finally
+         {
+             FileUtils.deleteRecursive(tempdir);
+         }
+     }
+ 
+     private static void checkThenCorruptBoundaries(Descriptor descriptor, boolean expectDataMmappable) throws IOException
+     {
+         File summaryFile = new File(descriptor.filenameFor(Component.SUMMARY));
+         DataInputStream iStream = new DataInputStream(new FileInputStream(summaryFile));
+         IndexSummary indexSummary = IndexSummary.serializer.deserialize(iStream, StorageService.getPartitioner(), true, CFMetaData.DEFAULT_MIN_INDEX_INTERVAL, CFMetaData.DEFAULT_MAX_INDEX_INTERVAL);
+         ByteBuffer first = ByteBufferUtil.readWithLength(iStream);
+         ByteBuffer last = ByteBufferUtil.readWithLength(iStream);
+         MmappedSegmentedFile.Builder ibuilder = new MmappedSegmentedFile.Builder();
+         MmappedSegmentedFile.Builder dbuilder = new MmappedSegmentedFile.Builder();
+         ibuilder.deserializeBounds(iStream);
+         dbuilder.deserializeBounds(iStream);
+         iStream.close();
+         // index file cannot generally be non-mmappable, as index entries cannot be larger than MAX_SEGMENT_SIZE (due to promotedSize being encoded as an int)
+         assertBoundaries(descriptor.filenameFor(Component.PRIMARY_INDEX), true, ibuilder.boundaries());
+         assertBoundaries(descriptor.filenameFor(Component.DATA), expectDataMmappable, dbuilder.boundaries());
+ 
 -        DataOutputStreamPlus oStream = new DataOutputStreamPlus(new FileOutputStream(summaryFile));
++        DataOutputStreamPlus oStream = new WrappedDataOutputStreamPlus(new FileOutputStream(summaryFile));
+         IndexSummary.serializer.serialize(indexSummary, oStream, true);
+         ByteBufferUtil.writeWithLength(first, oStream);
+         ByteBufferUtil.writeWithLength(last, oStream);
+         oStream.writeInt(1);
+         oStream.writeLong(new File(descriptor.filenameFor(Component.PRIMARY_INDEX)).length());
+         oStream.writeLong(new File(descriptor.filenameFor(Component.DATA)).length());
+         oStream.close();
+     }
+ 
+     private static void assertBoundaries(String path, boolean expectMmappable, long[] boundaries)
+     {
+         long length = new File(path).length();
+         long prev = boundaries[0];
+         for (int i = 1 ; i <= boundaries.length && prev < length ; i++)
+         {
+             long boundary = i == boundaries.length ? length : boundaries[i];
+             Assert.assertEquals(String.format("[%d, %d), %d of %d", boundary, prev, i, boundaries.length),
+                                 expectMmappable, boundary - prev <= Integer.MAX_VALUE);
+             prev = boundary;
+         }
+     }
+ 
+ }


[13/16] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by be...@apache.org.
Merge branch 'cassandra-2.1' into cassandra-2.2

Conflicts:
	src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
	src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
	src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java


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

Branch: refs/heads/trunk
Commit: 25de92e321604626d6c098233082904832c07814
Parents: 7452b20 c37562e
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Wed Sep 30 19:47:14 2015 +0100
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Wed Sep 30 19:47:14 2015 +0100

----------------------------------------------------------------------
 .../io/sstable/AbstractSSTableSimpleWriter.java |   2 +
 .../cassandra/io/sstable/CQLSSTableWriter.java  |  15 +
 .../io/sstable/SSTableSimpleUnsortedWriter.java |   6 +
 .../io/sstable/SSTableSimpleWriter.java         |  13 +
 .../io/sstable/format/SSTableReader.java        |  17 +
 .../cassandra/io/util/ByteBufferDataInput.java  |   8 +-
 .../cassandra/io/util/MmappedSegmentedFile.java | 275 ++++++++++++----
 .../apache/cassandra/io/util/SegmentedFile.java |   1 +
 .../sstable/LongSegmentedFileBoundaryTest.java  | 324 +++++++++++++++++++
 9 files changed, 605 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
index 4181ed0,c364171..8873f88
--- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
@@@ -370,20 -376,22 +380,25 @@@ public class CQLSSTableWriter implement
              }
          }
  
+         CFMetaData metadata()
+         {
+             return schema;
+         }
+ 
          /**
 -         * Adds the specified column family to the specified keyspace.
 +         * Creates the keyspace with the specified table.
           *
 -         * @param ksm the keyspace meta data
 -         * @param cfm the column family meta data
 +         * @param the table the table that must be created.
           */
 -        private static void addTableToKeyspace(KSMetaData ksm, CFMetaData cfm)
 +        private static void createKeyspaceWithTable(CFMetaData table)
          {
 -            ksm = KSMetaData.cloneWith(ksm, Iterables.concat(ksm.cfMetaData().values(), Collections.singleton(cfm)));
 -            Schema.instance.load(cfm);
 -            Schema.instance.setKeyspaceDefinition(ksm);
 +            KSMetaData ksm;
 +            ksm = KSMetaData.newKeyspace(table.ksName,
 +                                         AbstractReplicationStrategy.getClass("org.apache.cassandra.locator.SimpleStrategy"),
 +                                         ImmutableMap.of("replication_factor", "1"),
 +                                         true,
 +                                         Collections.singleton(table));
 +            Schema.instance.load(ksm);
          }
  
          /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
index 4bb75bc,25ec354..534e77b
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
@@@ -143,7 -140,13 +143,13 @@@ public class SSTableSimpleUnsortedWrite
          return previous;
      }
  
+     public Descriptor getCurrentDescriptor()
+     {
+         // can be implemented, but isn't necessary
+         throw new UnsupportedOperationException();
+     }
+ 
 -    protected ColumnFamily createColumnFamily() throws IOException
 +    protected ColumnFamily createColumnFamily()
      {
          return ArrayBackedSortedColumns.factory.create(metadata);
      }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25de92e3/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
index 2601d6d,23da501..f81e57d
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
@@@ -26,7 -24,6 +26,8 @@@ import org.apache.cassandra.db.*
  import org.apache.cassandra.db.marshal.AbstractType;
  import org.apache.cassandra.dht.IPartitioner;
  import org.apache.cassandra.io.FSError;
++import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.io.sstable.format.SSTableWriter;
  
  /**
   * A SSTable writer that assumes rows are in (partitioner) sorted order.
@@@ -71,6 -65,13 +72,13 @@@ public class SSTableSimpleWriter extend
          writer = getWriter();
      }
  
+     SSTableReader closeAndOpenReader()
+     {
+         if (currentKey != null)
+             writeRow(currentKey, columnFamily);
 -        return writer.closeAndOpenReader();
++        return writer.finish(true);
+     }
+ 
      public void close()
      {
          try