Don't regenerate bloomfilter and summaries on startup

Patch by Kurt Greaves; Reviewed by Chris Lohfink for CASSANDRA-11163, 
CASSANDRA-14166


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

Branch: refs/heads/trunk
Commit: 58e6c55e17d6b75328b065e093d51c08b8844004
Parents: 2c15098
Author: kurt <k...@instaclustr.com>
Authored: Wed Jan 31 04:50:24 2018 +0000
Committer: Mick Semb Wever <m...@apache.org>
Committed: Mon Mar 12 19:17:36 2018 +1100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 NEWS.txt                                        |  2 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  4 +-
 .../io/sstable/format/SSTableReader.java        | 97 ++++++++++++++------
 .../cassandra/io/sstable/SSTableReaderTest.java | 92 ++++++++++++++++++-
 5 files changed, 161 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/58e6c55e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 9c6a853..1d1a07a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.17
+ * Don't regenerate bloomfilter and summaries on startup (CASSANDRA-11163)
  * Fix NPE when performing comparison against a null frozen in LWT 
(CASSANDRA-14087)
  * Log when SSTables are deleted (CASSANDRA-14302)
  * Fix batch commitlog sync regression (CASSANDRA-14292)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/58e6c55e/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 64de28a..c06030e 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -50,6 +50,8 @@ Upgrading
     - Materialized view users upgrading from 3.0.15 or later that have 
performed range movements (join, decommission, move, etc),
       should run repair on the base tables, and subsequently on the views to 
ensure data affected by CASSANDRA-14251 is correctly
       propagated to all replicas.
+    - Changes to bloom_filter_fp_chance will no longer take effect on existing 
sstables when the node is restarted. Only
+      compactions/upgradesstables regenerates bloom filters and Summaries 
sstable components. See CASSANDRA-11163
 
 3.0.16
 =====

http://git-wip-us.apache.org/repos/asf/cassandra/blob/58e6c55e/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java 
b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index b5946bb..14e06b0 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -1752,8 +1752,8 @@ public class ColumnFamilyStore implements 
ColumnFamilyStoreMBean
                 {
                     if (logger.isTraceEnabled())
                         logger.trace("using snapshot sstable {}", 
entries.getKey());
-                    // open without tracking hotness
-                    sstable = SSTableReader.open(entries.getKey(), 
entries.getValue(), metadata, true, false);
+                    // open offline so we don't modify components or track 
hotness.
+                    sstable = SSTableReader.open(entries.getKey(), 
entries.getValue(), metadata, true, true);
                     refs.tryRef(sstable);
                     // release the self ref as we never add the snapshot 
sstable to DataTracker where it is otherwise released
                     sstable.selfRef().release();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/58e6c55e/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java 
b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index 2c94e45..c66fd8c 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -362,19 +362,19 @@ public abstract class SSTableReader extends SSTable 
implements SelfRefCounted<SS
 
     public static SSTableReader open(Descriptor descriptor, Set<Component> 
components, CFMetaData metadata) throws IOException
     {
-        return open(descriptor, components, metadata, true, true);
+        return open(descriptor, components, metadata, true, false);
     }
 
     // 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, false, false); // do 
not track hotness
+        return open(descriptor, components, cfs.metadata, false, true);
     }
 
     // use only for offline or "Standalone" operations
     public static SSTableReader openNoValidation(Descriptor descriptor, 
CFMetaData metadata) throws IOException
     {
-        return open(descriptor, componentsFor(descriptor), metadata, false, 
false); // do not track hotness
+        return open(descriptor, componentsFor(descriptor), metadata, false, 
true);
     }
 
     /**
@@ -435,11 +435,22 @@ public abstract class SSTableReader extends SSTable 
implements SelfRefCounted<SS
         }
     }
 
+    /**
+     * Open an SSTable for reading
+     * @param descriptor SSTable to open
+     * @param components Components included with this SSTable
+     * @param metadata for this SSTables CF
+     * @param validate Check SSTable for corruption (limited)
+     * @param isOffline Whether we are opening this SSTable "offline", for 
example from an external tool or not for inclusion in queries (validations)
+     *                  This stops regenerating BF + Summaries and also 
disables tracking of hotness for the SSTable.
+     * @return {@link SSTableReader}
+     * @throws IOException
+     */
     public static SSTableReader open(Descriptor descriptor,
-                                      Set<Component> components,
-                                      CFMetaData metadata,
-                                      boolean validate,
-                                      boolean trackHotness) throws IOException
+                                     Set<Component> components,
+                                     CFMetaData metadata,
+                                     boolean validate,
+                                     boolean isOffline) throws IOException
     {
         // Minimum components without which we can't do anything
         assert components.contains(Component.DATA) : "Data component is 
missing for sstable " + descriptor;
@@ -488,10 +499,10 @@ public abstract class SSTableReader extends SSTable 
implements SelfRefCounted<SS
         {
             // load index and filter
             long start = System.nanoTime();
-            sstable.load(validationMetadata);
+            sstable.load(validationMetadata, isOffline);
             logger.trace("INDEX LOAD TIME for {}: {} ms.", descriptor, 
TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
 
-            sstable.setup(trackHotness);
+            sstable.setup(!isOffline); // Don't track hotness if we're offline.
             if (validate)
                 sstable.validate();
 
@@ -681,34 +692,37 @@ public abstract class SSTableReader extends SSTable 
implements SelfRefCounted<SS
         return keyCache != null;
     }
 
-    private void load(ValidationMetadata validation) throws IOException
+    /**
+     * See {@link #load(boolean, boolean)}
+     * @param validation Metadata for SSTable being loaded
+     * @param isOffline Whether the SSTable is being loaded by an offline tool 
(sstabledump, scrub, etc)
+     * @throws IOException
+     */
+    private void load(ValidationMetadata validation, boolean isOffline) throws 
IOException
     {
         if (metadata.params.bloomFilterFpChance == 1.0)
         {
             // bf is disabled.
-            load(false, true);
+            load(false, !isOffline);
             bf = FilterFactory.AlwaysPresent;
         }
-        else if (!components.contains(Component.PRIMARY_INDEX))
+        else if (!components.contains(Component.PRIMARY_INDEX)) // What 
happens if filter component and primary index is missing?
         {
             // avoid any reading of the missing primary index component.
             // this should only happen during StandaloneScrubber
-            load(false, false);
+            load(false, !isOffline);
         }
         else if (!components.contains(Component.FILTER) || validation == null)
         {
             // bf is enabled, but filter component is missing.
-            load(true, true);
-        }
-        else if (validation.bloomFilterFPChance != 
metadata.params.bloomFilterFpChance)
-        {
-            // bf fp chance in sstable metadata and it has changed since 
compaction.
-            load(true, true);
+            load(!isOffline, !isOffline);
+            if (isOffline)
+                bf = FilterFactory.AlwaysPresent;
         }
         else
         {
             // bf is enabled and fp chance matches the currently configured 
value.
-            load(false, true);
+            load(false, !isOffline);
             loadBloomFilter(descriptor.version.hasOldBfHashOrder());
         }
     }
@@ -727,7 +741,8 @@ public abstract class SSTableReader extends SSTable 
implements SelfRefCounted<SS
     }
 
     /**
-     * Loads ifile, dfile and indexSummary, and optionally recreates the bloom 
filter.
+     * Loads ifile, dfile and indexSummary, and optionally recreates and 
persists the bloom filter.
+     * @param recreateBloomFilter Recreate the bloomfilter.
      * @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
      */
@@ -737,20 +752,22 @@ public abstract class SSTableReader extends SSTable 
implements SelfRefCounted<SS
             SegmentedFile.Builder dbuilder = 
SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode(), compression))
         {
             boolean summaryLoaded = loadSummary(ibuilder, dbuilder);
-            boolean builtSummary = false;
-            if (recreateBloomFilter || !summaryLoaded)
-            {
+            boolean buildSummary = !summaryLoaded || recreateBloomFilter;
+            if (buildSummary)
                 buildSummary(recreateBloomFilter, ibuilder, dbuilder, 
summaryLoaded, Downsampling.BASE_SAMPLING_LEVEL);
-                builtSummary = true;
-            }
 
             if (components.contains(Component.PRIMARY_INDEX))
                 ifile = ibuilder.buildIndex(descriptor, indexSummary);
 
             dfile = dbuilder.buildData(descriptor, sstableMetadata);
 
-            if (saveSummaryIfCreated && builtSummary)
-                saveSummary(ibuilder, dbuilder);
+            if (buildSummary)
+            {
+                if (saveSummaryIfCreated)
+                    saveSummary(ibuilder, dbuilder);
+                if (recreateBloomFilter)
+                    saveBloomFilter();
+            }
         }
         catch (Throwable t)
         { // Because the tidier has not been set-up yet in 
SSTableReader.open(), we must release the files in case of error
@@ -927,6 +944,30 @@ public abstract class SSTableReader extends SSTable 
implements SelfRefCounted<SS
         }
     }
 
+    public void saveBloomFilter()
+    {
+        saveBloomFilter(this.descriptor, bf);
+    }
+
+    public static void saveBloomFilter(Descriptor descriptor, IFilter filter)
+    {
+        File filterFile = new File(descriptor.filenameFor(Component.FILTER));
+        try (DataOutputStreamPlus stream = new 
BufferedDataOutputStreamPlus(new FileOutputStream(filterFile)))
+        {
+            FilterFactory.serialize(filter, stream);
+            stream.flush();
+        }
+        catch (IOException e)
+        {
+            logger.trace("Cannot save SSTable bloomfilter: ", e);
+
+            // corrupted hence delete it and let it load it now.
+            if (filterFile.exists())
+                FileUtils.deleteWithConfirm(filterFile);
+        }
+
+    }
+
     public void setReplaced()
     {
         synchronized (tidy.global)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/58e6c55e/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java 
b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
index 8eefd29..50519b2 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
@@ -17,8 +17,11 @@
  */
 package org.apache.cassandra.io.sstable;
 
+import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Path;
 import java.util.*;
 import java.util.concurrent.*;
 
@@ -47,13 +50,14 @@ import org.apache.cassandra.index.Index;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.FileDataInput;
 import org.apache.cassandra.io.util.MmappedRegions;
-import org.apache.cassandra.io.util.MmappedSegmentedFile;
 import org.apache.cassandra.io.util.SegmentedFile;
 import org.apache.cassandra.schema.CachingParams;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FilterFactory;
 import org.apache.cassandra.utils.Pair;
+import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -354,10 +358,10 @@ public class SSTableReaderTest
 
 
             new RowUpdateBuilder(store.metadata, timestamp, key.getKey())
-                .clustering("col")
-                .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
-                .build()
-                .applyUnsafe();
+            .clustering("col")
+            .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
+            .build()
+            .applyUnsafe();
 
         }
         store.forceBlockingFlush();
@@ -371,6 +375,84 @@ public class SSTableReaderTest
         Assert.assertArrayEquals(ByteBufferUtil.getArray(firstKey.getKey()), 
target.getIndexSummaryKey(0));
         assert target.first.equals(firstKey);
         assert target.last.equals(lastKey);
+
+        executeInternal(String.format("ALTER TABLE \"%s\".\"%s\" WITH 
bloom_filter_fp_chance = 0.3", ks, cf));
+
+        File summaryFile = new File(desc.filenameFor(Component.SUMMARY));
+        Path bloomPath = new File(desc.filenameFor(Component.FILTER)).toPath();
+        Path summaryPath = summaryFile.toPath();
+
+        long bloomModified = Files.getLastModifiedTime(bloomPath).toMillis();
+        long summaryModified = 
Files.getLastModifiedTime(summaryPath).toMillis();
+
+        Thread.sleep(TimeUnit.MILLISECONDS.toMillis(10)); // sleep to ensure 
modified time will be different
+
+        // Offline tests
+        // check that bloomfilter/summary ARE NOT regenerated
+        target = SSTableReader.openNoValidation(desc, store.metadata);
+
+        assertEquals(bloomModified, 
Files.getLastModifiedTime(bloomPath).toMillis());
+        assertEquals(summaryModified, 
Files.getLastModifiedTime(summaryPath).toMillis());
+
+        target.selfRef().release();
+
+        // check that bloomfilter/summary ARE NOT regenerated and 
BF=AlwaysPresent when filter component is missing
+        Set<Component> components = SSTable.discoverComponentsFor(desc);
+        components.remove(Component.FILTER);
+        target = SSTableReader.openNoValidation(desc, components, store);
+
+        assertEquals(bloomModified, 
Files.getLastModifiedTime(bloomPath).toMillis());
+        assertEquals(summaryModified, 
Files.getLastModifiedTime(summaryPath).toMillis());
+        assertEquals(FilterFactory.AlwaysPresent, target.getBloomFilter());
+
+        target.selfRef().release();
+
+        // #### online tests ####
+        // check that summary & bloomfilter are not regenerated when SSTable 
is opened and BFFP has been changed
+        target = SSTableReader.open(desc, store.metadata);
+
+        assertEquals(bloomModified, 
Files.getLastModifiedTime(bloomPath).toMillis());
+        assertEquals(summaryModified, 
Files.getLastModifiedTime(summaryPath).toMillis());
+
+        target.selfRef().release();
+
+        // check that bloomfilter is recreated when it doesn't exist and this 
causes the summary to be recreated
+        components = SSTable.discoverComponentsFor(desc);
+        components.remove(Component.FILTER);
+
+        target = SSTableReader.open(desc, components, store.metadata);
+
+        assertTrue("Bloomfilter was not recreated", bloomModified < 
Files.getLastModifiedTime(bloomPath).toMillis());
+        assertTrue("Summary was not recreated", summaryModified < 
Files.getLastModifiedTime(summaryPath).toMillis());
+
+        target.selfRef().release();
+
+        // check that only the summary is regenerated when it is deleted
+        components.add(Component.FILTER);
+        summaryModified = Files.getLastModifiedTime(summaryPath).toMillis();
+        summaryFile.delete();
+
+        Thread.sleep(TimeUnit.MILLISECONDS.toMillis(10)); // sleep to ensure 
modified time will be different
+        bloomModified = Files.getLastModifiedTime(bloomPath).toMillis();
+
+        target = SSTableReader.open(desc, components, store.metadata);
+
+        assertEquals(bloomModified, 
Files.getLastModifiedTime(bloomPath).toMillis());
+        assertTrue("Summary was not recreated", summaryModified < 
Files.getLastModifiedTime(summaryPath).toMillis());
+
+        target.selfRef().release();
+
+        // check that summary and bloomfilter is not recreated when the INDEX 
is missing
+        components.add(Component.SUMMARY);
+        components.remove(Component.PRIMARY_INDEX);
+
+        summaryModified = Files.getLastModifiedTime(summaryPath).toMillis();
+        target = SSTableReader.open(desc, components, store.metadata, false, 
false);
+
+        Thread.sleep(TimeUnit.MILLISECONDS.toMillis(10)); // sleep to ensure 
modified time will be different
+        assertEquals(bloomModified, 
Files.getLastModifiedTime(bloomPath).toMillis());
+        assertEquals(summaryModified, 
Files.getLastModifiedTime(summaryPath).toMillis());
+
         target.selfRef().release();
     }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to