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