You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2023/01/04 12:48:35 UTC

[GitHub] [cassandra] jacek-lewandowski opened a new pull request, #2064: CASSANDRA-17056: CEP-17

jacek-lewandowski opened a new pull request, #2064:
URL: https://github.com/apache/cassandra/pull/2064

   Co-authored-by: @jacek-lewandowski 
   Co-authored-by: @blambov 
   
   The [Cassandra Jira](https://issues.apache.org/jira/projects/CASSANDRA/issues/CASSANDRA-17056)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1071806982


##########
src/java/org/apache/cassandra/utils/Throwables.java:
##########
@@ -189,13 +189,19 @@ public static Throwable perform(Throwable accumulate, String filePath, FileOpTyp
     }
 
     public static void closeAndAddSuppressed(@Nonnull Throwable t, AutoCloseable... closeables)
+    {
+        closeAndAddSuppressed(t, Arrays.asList(closeables));
+    }
+
+    public static void closeAndAddSuppressed(@Nonnull Throwable t, Iterable<AutoCloseable> closeables)
     {
         Preconditions.checkNotNull(t);
         for (AutoCloseable closeable : closeables)
         {
             try
             {
-                closeable.close();
+                if (closeable != null)

Review Comment:
   This is a new method. `FileUtils.close` just throws while here we are adding suppressed exceptions. Actually this works very similar to try..catch with resources, where only non-null resources are closed and no NPE is thrown. If you think I should move it to `FileUtils` I'm ok with that.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1072101239


##########
src/java/org/apache/cassandra/db/compaction/CompactionManager.java:
##########
@@ -1890,22 +1890,17 @@ public void run()
         return executor.submitIfRunning(runnable, "cache write");
     }
 
-    public List<SSTableReader> runIndexSummaryRedistribution(IndexSummaryRedistribution redistribution) throws IOException
+    public <T, E extends Throwable> T runWithActiveCompactions(Holder holder, ThrowingSupplier<T, E> callable) throws E

Review Comment:
   ok



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1080961790


##########
src/java/org/apache/cassandra/config/CassandraRelevantProperties.java:
##########
@@ -331,7 +331,9 @@
      *
      * If only keyspaces are specified, mutations for all tables in such keyspace will be replayed
      * */
-    COMMIT_LOG_REPLAY_LIST("cassandra.replayList", null)
+    COMMIT_LOG_REPLAY_LIST("cassandra.replayList", null),
+
+    SSTABLE_FORMAT_DEFAULT("cassandra.sstable.format.default", "BIG")

Review Comment:
   heh... I've tried that, it creates some weird static initialization (dis)order problems



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081296044


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java:
##########
@@ -36,6 +39,8 @@
     SSTableWriter.Factory getWriterFactory();
     SSTableReader.Factory getReaderFactory();
 
+    Set<Component> supportedComponents();

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081338660


##########
src/java/org/apache/cassandra/io/sstable/Descriptor.java:
##########
@@ -423,6 +425,19 @@ public boolean isCompatible()
         return version.isCompatible();
     }
 
+    public Set<Component> discoverComponents()
+    {
+        Set<Component.Type> knownTypes = Sets.difference(Component.TYPES, Collections.singleton(Component.Type.CUSTOM));
+        Set<Component> components = Sets.newHashSetWithExpectedSize(knownTypes.size());
+        for (Component.Type componentType : knownTypes)
+        {
+            Component component = new Component(componentType);

Review Comment:
   Nit: Move the declaration of `singletons` after the components -- this avoids making people wonder how it's populated.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081328896


##########
src/java/org/apache/cassandra/io/sstable/format/FilterComponent.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileInputStreamPlus;
+import org.apache.cassandra.io.util.FileOutputStreamPlus;
+import org.apache.cassandra.utils.BloomFilterSerializer;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+
+public class FilterComponent
+{
+    private static final Logger logger = LoggerFactory.getLogger(FilterComponent.class);
+
+    private FilterComponent()
+    {
+    }
+
+    /**
+     * Load bloom filter from Filter.db file.
+     */
+    public static IFilter load(Descriptor descriptor) throws IOException
+    {
+        File filterFile = descriptor.fileFor(Component.FILTER);
+
+        if (!filterFile.exists())
+            return null;
+
+        if (filterFile.length() == 0)
+            return FilterFactory.AlwaysPresent;
+
+        try (FileInputStreamPlus stream = descriptor.fileFor(Component.FILTER).newInputStream())
+        {
+            return BloomFilterSerializer.forVersion(descriptor.version.hasOldBfFormat()).deserialize(stream);
+        }
+        catch (IOException ex)
+        {
+            throw new IOException("Failed to load Bloom filter for SSTable: " + descriptor.baseFilename(), ex);
+        }
+    }
+
+    public static void save(IFilter filter, Descriptor descriptor) throws IOException
+    {
+        File filterFile = descriptor.fileFor(Component.FILTER);
+        try (FileOutputStreamPlus stream = filterFile.newOutputStream(File.WriteMode.OVERWRITE))
+        {
+            filter.serialize(stream, descriptor.version.hasOldBfFormat());
+            stream.flush();
+            stream.sync(); // is it needed if we close the file right after that?

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081342432


##########
src/java/org/apache/cassandra/io/sstable/format/FilterComponent.java:
##########
@@ -90,4 +97,63 @@ public static void saveOrDeleteCorrupted(Descriptor descriptor, IFilter filter)
             throw ex;
         }
     }
+
+    /**
+     * Optionally loads a Bloom filter. If the filter is not needed (FP chance is neglectable), it sets
+     * {@link AlwaysPresentFilter} as a filter in the builder. If the filter is expected to be recreated for various
+     * reasons, it leaves it {@code null} (unchanged). Otherwise, it attempts to load the filter, and if it succeeds,
+     * it is set in the builder. If a filter fails to load, it is left {@code null} (unchanged) meaning that it should
+     * be rebuilt.
+     */
+    public static IFilter maybeLoadBloomFilter(Descriptor descriptor, Set<Component> components, TableMetadata metadata, ValidationMetadata validationMetadata)
+    {
+        double currentFPChance = validationMetadata != null ? validationMetadata.bloomFilterFPChance : Double.NaN;
+        double desiredFPChance = metadata.params.bloomFilterFpChance;
+
+        IFilter filter = null;
+        if (!shouldUseBloomFilter(desiredFPChance))
+        {
+            if (logger.isTraceEnabled())
+                logger.trace("Bloom filter for {} will not be loaded because fpChance={} is neglectable", descriptor, desiredFPChance);
+
+            return FilterFactory.AlwaysPresent;
+        }
+        else if (!components.contains(Component.FILTER) || Double.isNaN(currentFPChance))
+        {
+            if (logger.isTraceEnabled())
+                logger.trace("Bloom filter for {} will not be loaded because filter component is missing or sstable lacks validation metadata", descriptor);
+
+            return null;
+        }
+        else if (!isFPChanceDiffNeglectable(desiredFPChance, currentFPChance) && rebuildFilterOnFPChanceChange)
+        {
+            if (logger.isTraceEnabled())
+                logger.trace("Bloom filter for {} will not be loaded because fpChance has changed from {} to {} and the filter should be recreated", descriptor, currentFPChance, desiredFPChance);
+
+            return null;
+        }
+
+        try
+        {
+            filter = load(descriptor);
+            if (filter == null || filter instanceof AlwaysPresentFilter)
+                logger.info("Bloom filter for {} is missing or invalid", descriptor);
+        }
+        catch (IOException ex)
+        {
+            logger.info("Bloom filter for " + descriptor + " could not be deserialized", ex);
+        }
+
+        return filter;
+    }
+
+    static boolean shouldUseBloomFilter(double fpChance)
+    {
+        return !(Math.abs(1 - fpChance) <= filterFPChanceTolerance);
+    }
+
+    static boolean isFPChanceDiffNeglectable(double fpChance1, double fpChance2)

Review Comment:
   Nit: there's one "neglectable" left in the comment.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1090239389


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -1415,7 +1418,7 @@ public boolean mayContainAssumingKeyIsInRange(DecoratedKey key)
     protected static final class InstanceTidier implements Tidy
     {
         private final Descriptor descriptor;
-        private final TableId tableId;
+        private final Owner owner;

Review Comment:
   I don't know TBH. But since the owner is passed explicitly when the sstable instance is built, I would find it quite weird that sstable could lost such reference. Especially that the building code doesn't necessarily need to pass an owner, it is allowed to pass null. This is just my gut feeling, and I'm not strongly opposed to change that.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1094784473


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java:
##########
@@ -115,38 +123,75 @@ default boolean delete(Descriptor desc)
 
     FileTime getLastModifiedTime(Descriptor desc);
 
-    enum Type
+    default void setup(Map<String, String> options)
+    {
+    }
+
+    class Type
     {
-        //The original sstable format
-        BIG("big", BigFormat.instance);
+        private final static ImmutableList<Type> types;
+
+        static
+        {
+            Map<String, Supplier<SSTableFormat<?, ?>>> factories = DatabaseDescriptor.getSSTableFormatFactories();
+            List<Type> typesList = new ArrayList<>(factories.size());
+            factories.forEach((key, factory) -> typesList.add(new Type(typesList.size(), key.toLowerCase(), factory.get())));
+            types = ImmutableList.copyOf(typesList);
+        }
 
+        public final int ordinal;
         public final SSTableFormat<?, ?> info;
         public final String name;
+        private final static Type[] typesArray0 = new Type[0];
+
+        private static Type currentType;
 
         public static Type current()
         {
-            return CassandraRelevantProperties.SSTABLE_FORMAT_DEFAULT.getEnum(true, Type.class);
+            if (currentType != null)
+                return currentType;
+
+            String key = CassandraRelevantProperties.SSTABLE_FORMAT_DEFAULT.getString();
+            try
+            {
+                Type type = validate(key);
+                currentType = type;
+                return type;
+            }
+            catch (RuntimeException ex)
+            {
+                throw new ConfigurationException("SSTable format " + key + " is not registered. Registered formats are: " + types);
+            }
         }
 
-        Type(String name, SSTableFormat<?, ?> info)
+        private Type(int ordinal, String name, SSTableFormat<?, ?> info)
         {
             //Since format comes right after generation
             //we disallow formats with numeric names
             assert !CharMatcher.digit().matchesAllOf(name);
-
+            this.ordinal = ordinal;
             this.name = name;
             this.info = info;
         }
 
         public static Type validate(String name)
         {
-            for (Type valid : Type.values())
-            {
-                if (valid.name.equalsIgnoreCase(name))
-                    return valid;
-            }
+            for (int i = 0; i < types.size(); i++)

Review Comment:
   I just made the validation when initializing the configuration so that non-lowercased names are not accepted



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java:
##########
@@ -115,38 +123,75 @@ default boolean delete(Descriptor desc)
 
     FileTime getLastModifiedTime(Descriptor desc);
 
-    enum Type
+    default void setup(Map<String, String> options)
+    {
+    }
+
+    class Type
     {
-        //The original sstable format
-        BIG("big", BigFormat.instance);
+        private final static ImmutableList<Type> types;
+
+        static
+        {
+            Map<String, Supplier<SSTableFormat<?, ?>>> factories = DatabaseDescriptor.getSSTableFormatFactories();
+            List<Type> typesList = new ArrayList<>(factories.size());
+            factories.forEach((key, factory) -> typesList.add(new Type(typesList.size(), key.toLowerCase(), factory.get())));
+            types = ImmutableList.copyOf(typesList);
+        }
 
+        public final int ordinal;
         public final SSTableFormat<?, ?> info;
         public final String name;
+        private final static Type[] typesArray0 = new Type[0];
+
+        private static Type currentType;
 
         public static Type current()
         {
-            return CassandraRelevantProperties.SSTABLE_FORMAT_DEFAULT.getEnum(true, Type.class);
+            if (currentType != null)
+                return currentType;
+
+            String key = CassandraRelevantProperties.SSTABLE_FORMAT_DEFAULT.getString();
+            try
+            {
+                Type type = validate(key);
+                currentType = type;
+                return type;
+            }
+            catch (RuntimeException ex)
+            {
+                throw new ConfigurationException("SSTable format " + key + " is not registered. Registered formats are: " + types);
+            }
         }
 
-        Type(String name, SSTableFormat<?, ?> info)
+        private Type(int ordinal, String name, SSTableFormat<?, ?> info)
         {
             //Since format comes right after generation
             //we disallow formats with numeric names
             assert !CharMatcher.digit().matchesAllOf(name);
-
+            this.ordinal = ordinal;
             this.name = name;
             this.info = info;
         }
 
         public static Type validate(String name)
         {
-            for (Type valid : Type.values())
-            {
-                if (valid.name.equalsIgnoreCase(name))
-                    return valid;
-            }
+            for (int i = 0; i < types.size(); i++)

Review Comment:
   https://github.com/apache/cassandra/pull/2064/commits/80ef51bf6a43b9925949fa9e74c384dae0a5d6e9



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java:
##########
@@ -115,38 +123,75 @@ default boolean delete(Descriptor desc)
 
     FileTime getLastModifiedTime(Descriptor desc);
 
-    enum Type
+    default void setup(Map<String, String> options)
+    {
+    }
+
+    class Type
     {
-        //The original sstable format
-        BIG("big", BigFormat.instance);
+        private final static ImmutableList<Type> types;
+
+        static
+        {
+            Map<String, Supplier<SSTableFormat<?, ?>>> factories = DatabaseDescriptor.getSSTableFormatFactories();
+            List<Type> typesList = new ArrayList<>(factories.size());
+            factories.forEach((key, factory) -> typesList.add(new Type(typesList.size(), key.toLowerCase(), factory.get())));
+            types = ImmutableList.copyOf(typesList);
+        }
 
+        public final int ordinal;
         public final SSTableFormat<?, ?> info;
         public final String name;
+        private final static Type[] typesArray0 = new Type[0];
+
+        private static Type currentType;
 
         public static Type current()
         {
-            return CassandraRelevantProperties.SSTABLE_FORMAT_DEFAULT.getEnum(true, Type.class);
+            if (currentType != null)
+                return currentType;
+
+            String key = CassandraRelevantProperties.SSTABLE_FORMAT_DEFAULT.getString();

Review Comment:
   https://github.com/apache/cassandra/pull/2064/commits/80ef51bf6a43b9925949fa9e74c384dae0a5d6e9



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski merged pull request #2064: CASSANDRA-17056: CEP-17

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski merged PR #2064:
URL: https://github.com/apache/cassandra/pull/2064


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1071939141


##########
src/java/org/apache/cassandra/db/compaction/Scrubber.java:
##########
@@ -564,20 +565,27 @@ public ScrubResult(Scrubber scrubber)
     /**
      * During 2.x migration, under some circumstances rows might have gotten duplicated.
      * Merging iterator merges rows with same clustering.
-     *
+     * <p>
      * For more details, refer to CASSANDRA-12144.
      */
-    private static class RowMergingSSTableIterator extends WrappingUnfilteredRowIterator
+    private static class RowMergingSSTableIterator extends UnmodifiableIterator<Unfiltered> implements WrappingUnfilteredRowIterator

Review Comment:
   Yes, we can supply a default implementation in the interface just like it is done in `UnmodifiableIterator`, and not have to descend from the latter.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1072064724


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -808,32 +809,12 @@ 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);
+            tidy.runOnClose = () -> {

Review Comment:
   I must disagree. To my knowledge lambda does not include a reference to the enclosing instance unless explicitly `this` or some member is used. I've verified that, also see this answer https://stackoverflow.com/a/35636556



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1082405062


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -808,32 +809,12 @@ 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);
+            tidy.runOnClose = () -> {

Review Comment:
   fee2c251d370fd61ca436bc16169c1d1774ec01c



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081292027


##########
src/java/org/apache/cassandra/db/compaction/Scrubber.java:
##########
@@ -742,6 +750,12 @@ public FixNegativeLocalDeletionTimeIterator(UnfilteredRowIterator iterator, Outp
             this.negativeLocalExpirationTimeMetrics = negativeLocalDeletionInfoMetrics;
         }
 
+        @Override
+        public UnfilteredRowIterator wrapped()
+        {
+            return iterator;
+        }
+

Review Comment:
   looks good



##########
src/java/org/apache/cassandra/db/compaction/Scrubber.java:
##########
@@ -564,20 +565,27 @@ public ScrubResult(Scrubber scrubber)
     /**
      * During 2.x migration, under some circumstances rows might have gotten duplicated.
      * Merging iterator merges rows with same clustering.
-     *
+     * <p>
      * For more details, refer to CASSANDRA-12144.
      */
-    private static class RowMergingSSTableIterator extends WrappingUnfilteredRowIterator
+    private static class RowMergingSSTableIterator extends UnmodifiableIterator<Unfiltered> implements WrappingUnfilteredRowIterator

Review Comment:
   looks good



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java:
##########
@@ -36,6 +36,9 @@
     SSTableWriter.Factory getWriterFactory();
     SSTableReader.Factory getReaderFactory();
 
+    boolean isKeyCacheSupported();

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1082406344


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -1973,17 +1970,36 @@ public void run()
                     if (logger.isTraceEnabled())
                         logger.trace("Async instance tidier for {}, after barrier", descriptor);
 
-                    if (bf != null)
-                        bf.close();
-                    if (summary != null)
-                        summary.close();
-                    if (runOnClose != null)
+                    Throwable exceptions = null;
+                    if (runOnClose != null) try
+                    {
                         runOnClose.run();
-                    if (dfile != null)
-                        dfile.close();
-                    if (ifile != null)
-                        ifile.close();
-                    globalRef.release();
+                    }
+                    catch (RuntimeException | Error ex)
+                    {
+                        logger.error("Failed to run on-close listeners for sstable " + descriptor.baseFilename(), ex);
+                        exceptions = ex;
+                    }
+
+                    Throwable closeExceptions = Throwables.close(null, closeables);

Review Comment:
   I like that alternative idea



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1083873012


##########
src/java/org/apache/cassandra/utils/Throwables.java:
##########
@@ -189,13 +189,19 @@ public static Throwable perform(Throwable accumulate, String filePath, FileOpTyp
     }
 
     public static void closeAndAddSuppressed(@Nonnull Throwable t, AutoCloseable... closeables)
+    {
+        closeAndAddSuppressed(t, Arrays.asList(closeables));
+    }
+
+    public static void closeAndAddSuppressed(@Nonnull Throwable t, Iterable<AutoCloseable> closeables)
     {
         Preconditions.checkNotNull(t);
         for (AutoCloseable closeable : closeables)
         {
             try
             {
-                closeable.close();
+                if (closeable != null)

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1093258263


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java:
##########
@@ -115,38 +123,75 @@ default boolean delete(Descriptor desc)
 
     FileTime getLastModifiedTime(Descriptor desc);
 
-    enum Type
+    default void setup(Map<String, String> options)
+    {
+    }
+
+    class Type
     {
-        //The original sstable format
-        BIG("big", BigFormat.instance);
+        private final static ImmutableList<Type> types;
+
+        static
+        {
+            Map<String, Supplier<SSTableFormat<?, ?>>> factories = DatabaseDescriptor.getSSTableFormatFactories();
+            List<Type> typesList = new ArrayList<>(factories.size());
+            factories.forEach((key, factory) -> typesList.add(new Type(typesList.size(), key.toLowerCase(), factory.get())));
+            types = ImmutableList.copyOf(typesList);
+        }
 
+        public final int ordinal;
         public final SSTableFormat<?, ?> info;
         public final String name;
+        private final static Type[] typesArray0 = new Type[0];
+
+        private static Type currentType;
 
         public static Type current()
         {
-            return CassandraRelevantProperties.SSTABLE_FORMAT_DEFAULT.getEnum(true, Type.class);
+            if (currentType != null)
+                return currentType;
+
+            String key = CassandraRelevantProperties.SSTABLE_FORMAT_DEFAULT.getString();
+            try
+            {
+                Type type = validate(key);
+                currentType = type;
+                return type;
+            }
+            catch (RuntimeException ex)
+            {
+                throw new ConfigurationException("SSTable format " + key + " is not registered. Registered formats are: " + types);
+            }
         }
 
-        Type(String name, SSTableFormat<?, ?> info)
+        private Type(int ordinal, String name, SSTableFormat<?, ?> info)
         {
             //Since format comes right after generation
             //we disallow formats with numeric names
             assert !CharMatcher.digit().matchesAllOf(name);
-
+            this.ordinal = ordinal;
             this.name = name;
             this.info = info;
         }
 
         public static Type validate(String name)
         {
-            for (Type valid : Type.values())
-            {
-                if (valid.name.equalsIgnoreCase(name))
-                    return valid;
-            }
+            for (int i = 0; i < types.size(); i++)

Review Comment:
   ok



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1092023198


##########
src/java/org/apache/cassandra/io/sstable/SimpleGaugeProvider.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.util.Objects;
+import java.util.function.Function;
+
+import com.google.common.collect.Iterables;
+
+import com.codahale.metrics.Gauge;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.lifecycle.SSTableSet;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+
+public class SimpleGaugeProvider<T extends Number, R extends SSTableReader> extends GaugeProvider<T>
+{
+    private final Function<SSTableReader, R> mapper;
+    private final Function<Iterable<R>, T> combiner;
+
+    public SimpleGaugeProvider(Function<SSTableReader, R> mapper, String name, Function<Iterable<R>, T> combiner)
+    {
+        super(name);
+        this.mapper = mapper;
+        this.combiner = combiner;
+    }
+
+    @Override
+    public Gauge<T> getTableGauge(ColumnFamilyStore cfs)
+    {
+        return () -> combine(cfs.getLiveSSTables());
+    }
+
+    @Override
+    public Gauge<T> getKeyspaceGauge(Keyspace keyspace)
+    {
+        return () -> combine(getAllReaders(keyspace));
+    }
+
+    @Override
+    public Gauge<T> getGlobalGauge()
+    {
+        return () -> combine(Iterables.concat(Iterables.transform(Keyspace.all(), SimpleGaugeProvider::getAllReaders)));
+    }
+
+    private T combine(Iterable<SSTableReader> allReaders)
+    {
+        Iterable<R> readers = Iterables.filter(Iterables.transform(allReaders, mapper::apply), Objects::nonNull);
+        return combiner.apply(readers);
+    }
+
+    private static Iterable<SSTableReader> getAllReaders(Keyspace keyspace)
+    {
+        return Iterables.concat(Iterables.transform(keyspace.getColumnFamilyStores(), cfs -> cfs.getSSTables(SSTableSet.LIVE)));

Review Comment:
   Nit: for consistency, can this also use `cfs.getLiveSSTables()`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1072164958


##########
src/java/org/apache/cassandra/io/sstable/format/FilterComponent.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileInputStreamPlus;
+import org.apache.cassandra.io.util.FileOutputStreamPlus;
+import org.apache.cassandra.utils.BloomFilterSerializer;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+
+public class FilterComponent
+{
+    private static final Logger logger = LoggerFactory.getLogger(FilterComponent.class);
+
+    private FilterComponent()
+    {
+    }
+
+    /**
+     * Load bloom filter from Filter.db file.
+     */
+    public static IFilter load(Descriptor descriptor) throws IOException
+    {
+        File filterFile = descriptor.fileFor(Component.FILTER);
+
+        if (!filterFile.exists())
+            return null;
+
+        if (filterFile.length() == 0)
+            return FilterFactory.AlwaysPresent;
+
+        try (FileInputStreamPlus stream = descriptor.fileFor(Component.FILTER).newInputStream())
+        {
+            return BloomFilterSerializer.forVersion(descriptor.version.hasOldBfFormat()).deserialize(stream);
+        }
+        catch (IOException ex)
+        {
+            throw new IOException("Failed to load Bloom filter for SSTable: " + descriptor.baseFilename(), ex);
+        }
+    }
+
+    public static void save(IFilter filter, Descriptor descriptor) throws IOException
+    {
+        File filterFile = descriptor.fileFor(Component.FILTER);
+        try (FileOutputStreamPlus stream = filterFile.newOutputStream(File.WriteMode.OVERWRITE))
+        {
+            filter.serialize(stream, descriptor.version.hasOldBfFormat());
+            stream.flush();
+            stream.sync(); // is it needed if we close the file right after that?
+        }
+        catch (IOException ex)
+        {
+            throw new IOException("Failed to save Bloom filter for SSTable: " + descriptor.baseFilename(), ex);
+        }
+    }
+
+    public static void saveOrDeleteCorrupted(Descriptor descriptor, IFilter filter) throws IOException

Review Comment:
   I'm bad on naming, will change



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#issuecomment-1381989273

   One other thing I think we should include in this is breaking the dependency loop between `SSTableReader` and `ColumnFamilyStore`. `SSTableReader` is a low-level class and should not know how its owner works, and even less about how to get hold of its owner via `Schema`.
   
   If necessary, we can introduce an `SSTableReader.Owner` interface (similar to `Memtable.Owner`) to expose only functionality it needs.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1072027365


##########
src/java/org/apache/cassandra/schema/Schema.java:
##########
@@ -224,6 +225,15 @@ public ColumnFamilyStore getColumnFamilyStoreInstance(TableId id)
                : null;
     }
 
+    public ColumnFamilyStore getColumnFamilyStoreInstance(TableMetadata metadata)
+    {
+        ColumnFamilyStore cfs = getColumnFamilyStoreInstance(metadata.id);
+        if (cfs == null || !metadata.isIndex())

Review Comment:
   no, if it is null we want to return
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081429923


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java:
##########
@@ -217,7 +218,7 @@ public void setOpenResult(boolean openResult)
     /**
      * Open the resultant SSTableReader before it has been fully written

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081529860


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java:
##########
@@ -395,12 +393,13 @@ void load(boolean recreateBloomFilter,
                   StatsMetadata statsMetadata,
                   Set<Component> components) throws IOException
         {
-            try (FileHandle.Builder ibuilder = new FileHandle.Builder(descriptor.filenameFor(Component.PRIMARY_INDEX))
-                                               .mmapped(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap)
-                                               .withChunkCache(ChunkCache.instance);
-                 FileHandle.Builder dbuilder = new FileHandle.Builder(descriptor.filenameFor(Component.DATA)).mmapped(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap)
-                                                                                                             .withChunkCache(ChunkCache.instance))
+            try
             {
+                FileHandle.Builder ibuilder = new FileHandle.Builder(descriptor.fileFor(Component.PRIMARY_INDEX))

Review Comment:
   Can we also remove `lastRegions` for now, as it can be leaked or released while still in use?
   
   We need to separately figure out how to reuse memmappings for the index and data readers -- perhaps give the builder some `ReuseableRegions` to use, to be owned and closed by the writer.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081498581


##########
src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriterBuilder.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.big;
+
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.sstable.DataComponent;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SortedTableWriterBuilder;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.Throwables;
+
+public class BigTableWriterBuilder extends SortedTableWriterBuilder<RowIndexEntry, BigFormatPartitionWriter, BigTableWriter, BigTableWriterBuilder>
+{
+    private RowIndexEntry.IndexSerializer rowIndexEntrySerializer;
+    private BigTableWriter.IndexWriter indexWriter;
+    private SequentialWriter dataWriter;
+    private BigFormatPartitionWriter partitionWriter;
+
+    public BigTableWriterBuilder(Descriptor descriptor)
+    {
+        super(descriptor);
+    }
+
+    @Override
+    public SequentialWriter getDataWriter()
+    {
+        return dataWriter;
+    }
+
+    @Override
+    public BigFormatPartitionWriter getPartitionWriter()
+    {
+        return partitionWriter;
+    }
+
+    public RowIndexEntry.IndexSerializer getRowIndexEntrySerializer()
+    {
+        return rowIndexEntrySerializer;
+    }
+
+    public BigTableWriter.IndexWriter getIndexWriter()
+    {
+        return indexWriter;
+    }
+
+    @Override
+    protected BigTableWriter buildInternal(LifecycleNewTracker lifecycleNewTracker)
+    {
+        try
+        {
+            rowIndexEntrySerializer = new RowIndexEntry.Serializer(descriptor.version, getSerializationHeader());
+            dataWriter = DataComponent.buildWriter(descriptor,

Review Comment:
   Well, usually builders are so that end users don't use long argument lists; the internals can, and in this case it makes better sense that they do. If necessary, the builder can be made a static inner class so that the relevant constructor is private.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1090298045


##########
src/java/org/apache/cassandra/io/util/MmappedRegions.java:
##########
@@ -140,6 +147,25 @@ public void extend(long length)
         copy = new State(state);
     }
 
+    public void extend(CompressionMetadata compressionMetadata)
+    {
+        assert !isCopy() : "Copies cannot be extended";
+
+        if (compressionMetadata.dataLength <= state.length)

Review Comment:
   https://github.com/apache/cassandra/pull/2064/commits/5022c14d3583666f936c3467b4594bade7fbd360



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1093272853


##########
src/java/org/apache/cassandra/io/sstable/Component.java:
##########
@@ -41,55 +39,76 @@
 {
     public static final char separator = '-';
 
-    final static ImmutableSet<Type> TYPES = Type.all;
+    @SuppressWarnings("rawtypes")
+    public static Set<Component> getSingletonsFor(Class<? extends SSTableFormat> format)
+    {
+        return singletons.stream().filter(c -> c.type.formatClass.isAssignableFrom(format)).collect(Collectors.toSet());
+    }
 
     /**
      * WARNING: Be careful while changing the names or string representation of the enum
      * members. Streaming code depends on the names during streaming (Ref: CASSANDRA-14556).
      */
-    public static class Type
+    public final static class Type
     {
-        private final static List<Type> singletonsCollector = new ArrayList<>(11);
+        private final static CopyOnWriteArrayList<Type> typesCollector = new CopyOnWriteArrayList<>();
         // the base data for an sstable: the remaining components can be regenerated
         // based on the data component
-        public static final Type DATA = new Type("DATA", "Data.db", name -> Component.DATA);
+        public static final Type DATA = new Type("DATA", "Data.db", name -> Component.DATA, true);
         // file to hold information about uncompressed data length, chunk offsets etc.
-        public static final Type COMPRESSION_INFO = new Type("COMPRESSION_INFO", "CompressionInfo.db", name -> Component.COMPRESSION_INFO);
+        public static final Type COMPRESSION_INFO = new Type("COMPRESSION_INFO", "CompressionInfo.db", name -> Component.COMPRESSION_INFO, true);
         // statistical metadata about the content of the sstable
-        public static final Type STATS = new Type("STATS", "Statistics.db", name -> Component.STATS);
+        public static final Type STATS = new Type("STATS", "Statistics.db", name -> Component.STATS, true);
         // serialized bloom filter for the row keys in the sstable
-        public static final Type FILTER = new Type("FILTER", "Filter.db", name -> Component.FILTER);
+        public static final Type FILTER = new Type("FILTER", "Filter.db", name -> Component.FILTER, true);
         // holds CRC32 checksum of the data file
-        public static final Type DIGEST = new Type("DIGEST","Digest.crc32", name -> Component.DIGEST);
-        // holds the CRC32 for chunks in an a uncompressed file.
-        public static final Type CRC = new Type("CRC","CRC.db", name -> Component.CRC);
+        public static final Type DIGEST = new Type("DIGEST","Digest.crc32", name -> Component.DIGEST, true);
+        // holds the CRC32 for chunks in an uncompressed file.
+        public static final Type CRC = new Type("CRC","CRC.db", name -> Component.CRC, true);
         // table of contents, stores the list of all components for the sstable
-        public static final Type TOC = new Type("TOC","TOC.txt", name -> Component.TOC);
+        public static final Type TOC = new Type("TOC","TOC.txt", name -> Component.TOC, true);
         // built-in secondary index (may be multiple per sstable)
-        public static final Type SECONDARY_INDEX = new Type("SECONDARY_INDEX","SI_.*.db", name -> new Component(Type.SECONDARY_INDEX, name));
+        public static final Type SECONDARY_INDEX = new Type("SECONDARY_INDEX","SI_.*.db", name -> new Component(Type.SECONDARY_INDEX, name), false);
         // custom component, used by e.g. custom compaction strategy
-        public static final Type CUSTOM = new Type("CUSTOM",null, name -> new Component(Type.CUSTOM, name));
+        public static final Type CUSTOM = new Type("CUSTOM",null, name -> new Component(Type.CUSTOM, name), false);
 
-        public static final ImmutableSet<Type> all = ImmutableSet.copyOf(singletonsCollector);
+        public static final List<Type> all = Collections.unmodifiableList(typesCollector);
 
         public final int id;
         public final String name;
         public final String repr;
-        private Function<String, Component> componentFactory;
+        private final Function<String, Component> componentFactory;
+        public final boolean isSingleton;
+
+        @SuppressWarnings("rawtypes")
+        public final Class<? extends SSTableFormat> formatClass;
+
+        private Type(String name, String repr, Function<String, Component> componentFactory, boolean isSingleton)
+        {
+            this(name, repr, componentFactory, isSingleton, SSTableFormat.class);
+        }
 
-        protected Type(String name, String repr, Function<String, Component> componentFactory)
+        @SuppressWarnings("rawtypes")
+        public Type(String name, String repr, Function<String, Component> componentFactory, boolean isSingleton, Class<? extends SSTableFormat> formatClass)
         {
             this.name = name;
             this.repr = repr;
-            this.id = singletonsCollector.size();
+            this.id = typesCollector.size();
             this.componentFactory = componentFactory;
-            singletonsCollector.add(this);
+            this.isSingleton = isSingleton;
+            this.formatClass = formatClass;
+            synchronized (typesCollector)
+            {
+                if (typesCollector.stream().anyMatch(t -> Objects.equal(t.name, name) && t.formatClass == formatClass || t.formatClass.isAssignableFrom(formatClass) || formatClass.isAssignableFrom(t.formatClass)))

Review Comment:
   Obviously `t.formatClass == formatClass` is redundant...
   
   But to the point, I was thinking once about having something like intermediate abstract format, something like `SortedTableFormat` (and `SortedTableReader`) which would correspond to `SortedTableWriter`. Some components could be pushed down from the top level and I would use that extended verification as a sanity check.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1072034116


##########
src/java/org/apache/cassandra/schema/Schema.java:
##########
@@ -224,6 +225,15 @@ public ColumnFamilyStore getColumnFamilyStoreInstance(TableId id)
                : null;
     }
 
+    public ColumnFamilyStore getColumnFamilyStoreInstance(TableMetadata metadata)

Review Comment:
   actually, after applying your fixes this method is not needed any longer; so I will just add some comment to the original method



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1072183800


##########
src/java/org/apache/cassandra/utils/BloomFilterSerializer.java:
##########
@@ -17,47 +17,66 @@
  */
 package org.apache.cassandra.utils;
 
-import java.io.DataInput;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.OutputStream;
 
 import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IGenericSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.utils.obs.IBitSet;
 import org.apache.cassandra.utils.obs.OffHeapBitSet;
 
-public final class BloomFilterSerializer
+public final class BloomFilterSerializer<I extends InputStream & DataInputPlus, O extends OutputStream & DataOutputPlus> implements IGenericSerializer<BloomFilter, I, O>

Review Comment:
   long story short, because `FileInputStreamPlus` is not a subclass of `DataInputStreamPlus`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1072270442


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -808,32 +809,12 @@ 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);
+            tidy.runOnClose = () -> {

Review Comment:
   [This](https://issues.apache.org/jira/browse/CASSANDRA-9423?focusedCommentId=14602771&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-14602771) is the original problem stemming from this. AFAIR Stefania later fought the same problem with lambdas (where `this` was included in the closure even when it was not required), but I may be wrong, or things may have changed.
   
   If you have verified that leak detection works correctly for all supported JDK versions, I'll be very happy to accept that this is one less problem we have to worry about.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1072081121


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -907,43 +907,18 @@ public SSTableReader cloneWithNewStart(DecoratedKey newStart, final Runnable run
             // TODO: merge with caller's firstKeyBeyond() work,to save time
             if (newStart.compareTo(first) > 0)
             {
-                final long dataStart = getPosition(newStart, Operator.EQ);
-                final long indexStart = getIndexScanPosition(newStart);
-                this.tidy.runOnClose = new DropPageCache(dfile, dataStart, ifile, indexStart, runOnClose);
+                Map<FileHandle, Long> handleAndPositions = new LinkedHashMap<>(2);
+                if (dfile != null)
+                    handleAndPositions.put(dfile, getPosition(newStart, Operator.EQ));
+                if (ifile != null)
+                    handleAndPositions.put(ifile, getIndexScanPosition(newStart));
+                runOnClose(() -> handleAndPositions.forEach(FileHandle::dropPageCache));

Review Comment:
   I hope this will convince you:
   
   ```java
           class SomeUnserializableClass {
               public String a = "x";
               public byte[] serialize(String s) {
                   return SerializationUtils.serialize((Supplier<String> & Serializable) () -> {
                       return s + "#" + s + a;
                   });
               }
           }
   
           SomeUnserializableClass serializer = new SomeUnserializableClass();
           byte[] serialized = serializer.serialize("abc");
           Supplier<String> deseraizlied = SerializationUtils.deserialize(serialized);
           assertEquals("abc#abc", deseraizlied.get());
   ```
   
   this test indeed does not work because of `NotSerializableException` - we try to access `a` in lambda, which requires serialization of `SomeUnserializableClass`. 
   
   When `a` is removed, so that we just have:
   
   ```java
   return s + "#" + s;
   ```
   
   in lambda, test tests passes.
   
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1072343960


##########
src/java/org/apache/cassandra/io/sstable/format/FilterComponent.java:
##########
@@ -90,4 +97,63 @@ public static void saveOrDeleteCorrupted(Descriptor descriptor, IFilter filter)
             throw ex;
         }
     }
+
+    /**
+     * Optionally loads a Bloom filter. If the filter is not needed (FP chance is neglectable), it sets
+     * {@link AlwaysPresentFilter} as a filter in the builder. If the filter is expected to be recreated for various
+     * reasons, it leaves it {@code null} (unchanged). Otherwise, it attempts to load the filter, and if it succeeds,
+     * it is set in the builder. If a filter fails to load, it is left {@code null} (unchanged) meaning that it should

Review Comment:
   This comment needs adjusting &mdash; the filter is returned rather than set in the builder.



##########
src/java/org/apache/cassandra/io/sstable/format/FilterComponent.java:
##########
@@ -90,4 +97,63 @@ public static void saveOrDeleteCorrupted(Descriptor descriptor, IFilter filter)
             throw ex;
         }
     }
+
+    /**
+     * Optionally loads a Bloom filter. If the filter is not needed (FP chance is neglectable), it sets
+     * {@link AlwaysPresentFilter} as a filter in the builder. If the filter is expected to be recreated for various
+     * reasons, it leaves it {@code null} (unchanged). Otherwise, it attempts to load the filter, and if it succeeds,
+     * it is set in the builder. If a filter fails to load, it is left {@code null} (unchanged) meaning that it should
+     * be rebuilt.
+     */
+    public static IFilter maybeLoadBloomFilter(Descriptor descriptor, Set<Component> components, TableMetadata metadata, ValidationMetadata validationMetadata)
+    {
+        double currentFPChance = validationMetadata != null ? validationMetadata.bloomFilterFPChance : Double.NaN;
+        double desiredFPChance = metadata.params.bloomFilterFpChance;
+
+        IFilter filter = null;
+        if (!shouldUseBloomFilter(desiredFPChance))
+        {
+            if (logger.isTraceEnabled())
+                logger.trace("Bloom filter for {} will not be loaded because fpChance={} is neglectable", descriptor, desiredFPChance);
+
+            return FilterFactory.AlwaysPresent;
+        }
+        else if (!components.contains(Component.FILTER) || Double.isNaN(currentFPChance))
+        {
+            if (logger.isTraceEnabled())
+                logger.trace("Bloom filter for {} will not be loaded because filter component is missing or sstable lacks validation metadata", descriptor);
+
+            return null;
+        }
+        else if (!isFPChanceDiffNeglectable(desiredFPChance, currentFPChance) && rebuildFilterOnFPChanceChange)
+        {
+            if (logger.isTraceEnabled())
+                logger.trace("Bloom filter for {} will not be loaded because fpChance has changed from {} to {} and the filter should be recreated", descriptor, currentFPChance, desiredFPChance);
+
+            return null;
+        }
+
+        try
+        {
+            filter = load(descriptor);
+            if (filter == null || filter instanceof AlwaysPresentFilter)
+                logger.info("Bloom filter for {} is missing or invalid", descriptor);
+        }
+        catch (IOException ex)
+        {
+            logger.info("Bloom filter for " + descriptor + " could not be deserialized", ex);
+        }
+
+        return filter;
+    }
+
+    static boolean shouldUseBloomFilter(double fpChance)
+    {
+        return !(Math.abs(1 - fpChance) <= filterFPChanceTolerance);
+    }
+
+    static boolean isFPChanceDiffNeglectable(double fpChance1, double fpChance2)

Review Comment:
   Nit: We should use "negligible" instead of "neglectable" (archaic form).



##########
src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java:
##########
@@ -43,11 +43,11 @@
  * for on-close (i.e. when all references expire) that drops the page cache prior to that key position
  *
  * hard-links are created for each partially written sstable so that readers opened against them continue to work past
- * the rename of the temporary file, which is deleted once all readers against the hard-link have been closed.
+ * renaming of the temporary file, which is deleted once all readers against the hard-link have been closed.
  * If for any reason the writer is rolled over, we immediately rename and fully expose the completed file in the Tracker.
  *
- * On abort we restore the original lower bounds to the existing readers and delete any temporary files we had in progress,
- * but leave any hard-links in place for the readers we opened to cleanup when they're finished as we would had we finished
+ * On abort, we restore the original lower bounds to the existing readers and delete any temporary files we had in progress,
+ * but leave any hard-links in place for the readers we opened to clean-up when they're finished as we would have finished

Review Comment:
   This does not have the same meaning, "had we" in this case stands for "if we had"



##########
src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriterBuilder.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.big;
+
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.sstable.DataComponent;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SortedTableWriterBuilder;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.Throwables;
+
+public class BigTableWriterBuilder extends SortedTableWriterBuilder<RowIndexEntry, BigFormatPartitionWriter, BigTableWriter, BigTableWriterBuilder>
+{
+    private RowIndexEntry.IndexSerializer rowIndexEntrySerializer;
+    private BigTableWriter.IndexWriter indexWriter;
+    private SequentialWriter dataWriter;
+    private BigFormatPartitionWriter partitionWriter;
+
+    public BigTableWriterBuilder(Descriptor descriptor)
+    {
+        super(descriptor);
+    }
+
+    @Override
+    public SequentialWriter getDataWriter()
+    {
+        return dataWriter;
+    }
+
+    @Override
+    public BigFormatPartitionWriter getPartitionWriter()
+    {
+        return partitionWriter;
+    }
+
+    public RowIndexEntry.IndexSerializer getRowIndexEntrySerializer()
+    {
+        return rowIndexEntrySerializer;
+    }
+
+    public BigTableWriter.IndexWriter getIndexWriter()
+    {
+        return indexWriter;
+    }
+
+    @Override
+    protected BigTableWriter buildInternal(LifecycleNewTracker lifecycleNewTracker)
+    {
+        try
+        {
+            rowIndexEntrySerializer = new RowIndexEntry.Serializer(descriptor.version, getSerializationHeader());
+            dataWriter = DataComponent.buildWriter(descriptor,

Review Comment:
   Setting these fields here and grabbing them via the getter is somewhat unsafe; e.g. if the builder is reused we could mix instances up.
   
   Can't we just pass anything we construct as arguments to the `BigTableWriter` constructor?



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java:
##########
@@ -217,7 +218,7 @@ public void setOpenResult(boolean openResult)
     /**
      * Open the resultant SSTableReader before it has been fully written

Review Comment:
   Doc should explain the consumer.



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -1983,39 +1380,36 @@ public void addTo(Ref.IdentityCollection identities)
     {
         identities.add(this);
         identities.add(tidy.globalRef);
-        dfile.addTo(identities);
-        ifile.addTo(identities);
-        bf.addTo(identities);
-        indexSummary.addTo(identities);
-
+        tidy.closeables.forEach(c -> {
+            if (c instanceof SharedCloseable)
+                ((SharedCloseable) c).addTo(identities);
+        });
     }
 
     public boolean maybePresent(DecoratedKey key)

Review Comment:
   Could this be renamed to `mayBePresent` or better still `couldContain`? The current spelling implies "present" is used as a verb, i.e. that this is a function that changes some state.



##########
src/java/org/apache/cassandra/io/sstable/format/SortedTableWriter.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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.IOException;
+import java.nio.BufferOverflowException;
+import java.util.Set;
+import java.util.function.Consumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionPurger;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.guardrails.Guardrails;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.db.rows.ComplexColumnData;
+import org.apache.cassandra.db.rows.PartitionSerializationException;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundaryMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.compress.CompressedSequentialWriter;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.SchemaConstants;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public abstract class SortedTableWriter<P extends SortedTablePartitionWriter, RIE extends AbstractRowIndexEntry> extends SSTableWriter<RIE>
+{
+    private final static Logger logger = LoggerFactory.getLogger(SortedTableWriter.class);
+
+    protected final SequentialWriter dataWriter;
+    protected final P partitionWriter;
+    private final FileHandle.Builder dataFileBuilder = new FileHandle.Builder(descriptor.fileFor(Component.DATA));
+    private DecoratedKey lastWrittenKey;
+    private DataPosition dataMark;
+    private long lastEarlyOpenLength;
+
+    public SortedTableWriter(SortedTableWriterBuilder<RIE, P, ?, ?> builder, LifecycleNewTracker lifecycleNewTracker)
+    {
+        super(builder, lifecycleNewTracker);
+        checkNotNull(builder.getDataWriter());
+        checkNotNull(builder.getPartitionWriter());
+
+        this.dataWriter = builder.getDataWriter();
+        this.partitionWriter = builder.getPartitionWriter();
+    }
+
+    /**
+     * Appends partition data to this writer.
+     *
+     * @param partition the partition to write
+     * @return the created index entry if something was written, that is if {@code iterator}
+     * wasn't empty, {@code null} otherwise.
+     * @throws FSWriteError if write to the dataFile fails
+     */
+    @Override
+    public final RIE append(UnfilteredRowIterator partition)
+    {
+        if (partition.isEmpty())
+            return null;
+
+        try
+        {
+            if (!verifyPartition(partition.partitionKey()))
+                return null;
+
+            startPartition(partition.partitionKey(), partition.partitionLevelDeletion());
+
+            RIE indexEntry;
+            if (header.hasStatic())
+                addStaticRow(partition.partitionKey(), partition.staticRow());
+
+            while (partition.hasNext())
+                addUnfiltered(partition.partitionKey(), partition.next());
+
+            indexEntry = endPartition(partition.partitionKey(), partition.partitionLevelDeletion());
+
+            return indexEntry;
+        }
+        catch (BufferOverflowException boe)
+        {
+            throw new PartitionSerializationException(partition, boe);
+        }
+        catch (IOException e)
+        {
+            throw new FSWriteError(e, getFilename());
+        }
+    }
+
+    private boolean verifyPartition(DecoratedKey key)
+    {
+        assert key != null : "Keys must not be null"; // empty keys ARE allowed b/c of indexed column values
+
+        if (key.getKey().remaining() > FBUtilities.MAX_UNSIGNED_SHORT)
+        {
+            logger.error("Key size {} exceeds maximum of {}, skipping row", key.getKey().remaining(), FBUtilities.MAX_UNSIGNED_SHORT);
+            return false;
+        }
+
+        if (lastWrittenKey != null && lastWrittenKey.compareTo(key) >= 0)
+            throw new RuntimeException(String.format("Last written key %s >= current key %s, writing into %s", lastWrittenKey, key, getFilename()));
+
+        return true;
+    }
+
+    private void startPartition(DecoratedKey key, DeletionTime partitionLevelDeletion) throws IOException
+    {
+        partitionWriter.start(key, partitionLevelDeletion);
+        metadataCollector.update(partitionLevelDeletion);
+
+        onStartPartition(key);
+    }
+
+    private void addStaticRow(DecoratedKey key, Row row) throws IOException
+    {
+        guardCollectionSize(key, row);
+
+        partitionWriter.addStaticRow(row);
+        if (!row.isEmpty())
+            Rows.collectStats(row, metadataCollector);
+
+        onStaticRow(row);
+    }
+
+    private void addUnfiltered(DecoratedKey key, Unfiltered unfiltered) throws IOException
+    {
+        if (unfiltered.isRow())
+        {
+            addRow(key, (Row) unfiltered);
+        }
+        else
+        {
+            assert unfiltered.isRangeTombstoneMarker();
+            addRangeTomstoneMarker((RangeTombstoneMarker) unfiltered);
+        }
+    }
+
+    private void addRow(DecoratedKey key, Row row) throws IOException
+    {
+        guardCollectionSize(key, row);
+
+        partitionWriter.addUnfiltered(row);
+        metadataCollector.updateClusteringValues(row.clustering());
+        Rows.collectStats(row, metadataCollector);
+
+        onRow(row);
+    }
+
+    private void addRangeTomstoneMarker(RangeTombstoneMarker marker) throws IOException
+    {
+        partitionWriter.addUnfiltered(marker);
+
+        metadataCollector.updateClusteringValues(marker.clustering());
+        if (marker.isBoundary())
+        {
+            RangeTombstoneBoundaryMarker bm = (RangeTombstoneBoundaryMarker) marker;
+            metadataCollector.update(bm.endDeletionTime());
+            metadataCollector.update(bm.startDeletionTime());
+        }
+        else
+        {
+            metadataCollector.update(((RangeTombstoneBoundMarker) marker).deletionTime());
+        }
+
+        onRangeTombstoneMarker(marker);
+    }
+
+    private RIE endPartition(DecoratedKey key, DeletionTime partitionLevelDeletion) throws IOException
+    {
+        long finishResult = partitionWriter.finish();
+
+        long endPosition = dataWriter.position();
+        long rowSize = endPosition - partitionWriter.getInitialPosition();
+        maybeLogLargePartitionWarning(key, rowSize);
+        maybeLogManyTombstonesWarning(key, metadataCollector.totalTombstones);
+        metadataCollector.addPartitionSizeInBytes(rowSize);
+        metadataCollector.addKey(key.getKey());
+        metadataCollector.addCellPerPartitionCount();
+
+        lastWrittenKey = key;
+        last = lastWrittenKey;
+        if (first == null)
+            first = lastWrittenKey;
+
+        if (logger.isTraceEnabled())
+            logger.trace("wrote {} at {}", key, endPosition);
+
+        return createRowIndexEntry(key, partitionLevelDeletion, finishResult);
+    }
+
+    protected void onStartPartition(DecoratedKey key)
+    {
+        notifyObservers(o -> o.startPartition(key, partitionWriter.getInitialPosition(), partitionWriter.getInitialPosition()));
+    }
+
+    protected void onStaticRow(Row row)
+    {
+        notifyObservers(o -> o.staticRow(row));
+    }
+
+    protected void onRow(Row row)
+    {
+        notifyObservers(o -> o.nextUnfilteredCluster(row));
+    }
+
+    protected void onRangeTombstoneMarker(RangeTombstoneMarker marker)
+    {
+        notifyObservers(o -> o.nextUnfilteredCluster(marker));
+    }
+
+    protected abstract RIE createRowIndexEntry(DecoratedKey key, DeletionTime partitionLevelDeletion, long finishResult) throws IOException;
+
+    protected final void notifyObservers(Consumer<SSTableFlushObserver> action)
+    {
+        if (observers != null && !observers.isEmpty())
+            observers.forEach(action);
+    }
+
+    @Override
+    public void mark()
+    {
+        dataMark = dataWriter.mark();
+    }
+
+    @Override
+    public void resetAndTruncate()
+    {
+        dataWriter.resetAndTruncate(dataMark);
+        partitionWriter.reset();
+    }
+
+    @Override
+    public long getFilePointer()
+    {
+        return dataWriter.position();
+    }
+
+    @Override
+    public long getOnDiskFilePointer()
+    {
+        return dataWriter.getOnDiskFilePointer();
+    }
+
+    @Override
+    public long getEstimatedOnDiskBytesWritten()
+    {
+        return dataWriter.getEstimatedOnDiskBytesWritten();
+    }
+
+    protected FileHandle openDataFile(long lengthOverride, StatsMetadata statsMetadata)
+    {
+        int dataBufferSize = ioOptions.diskOptimizationStrategy.bufferSize(statsMetadata.estimatedPartitionSize.percentile(ioOptions.diskOptimizationEstimatePercentile));
+
+        FileHandle dataFile = dataFileBuilder.mmapped(ioOptions.defaultDiskAccessMode == Config.DiskAccessMode.mmap)

Review Comment:
   This should be `mmapped(ioOptions.defaultDiskAccessMode)`, after which the `mmapped(boolean)` version can be removed.



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java:
##########
@@ -103,4 +105,11 @@ public static Type validate(String name)
     {
         List<GaugeProvider<?, ?>> getGaugeProviders();
     }
+
+    interface SSTableReaderFactory<R extends SSTableReader, B extends SSTableReaderBuilder<R, B>>
+    {
+        SSTableReaderBuilder<R, B> builder(Descriptor descriptor);
+
+        SSTableReaderLoadingBuilder<R, B> builder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components);

Review Comment:
   The difference between these two is very unclear. Could you add comments or rename?



##########
src/java/org/apache/cassandra/io/sstable/format/SortedTableWriter.java:
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.IOException;
+import java.nio.BufferOverflowException;
+import java.util.Collection;
+import java.util.Set;
+import java.util.function.Consumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.rows.PartitionSerializationException;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundaryMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.TimeUUID;
+
+public abstract class SortedTableWriter<P extends SortedTablePartitionWriter, RIE extends AbstractRowIndexEntry> extends SSTableWriter<RIE>
+{
+    private final static Logger logger = LoggerFactory.getLogger(SortedTableWriter.class);
+
+    protected SequentialWriter dataWriter;
+    protected P partitionWriter;
+    private DecoratedKey lastWrittenKey;
+    private DataPosition dataMark;
+
+    public SortedTableWriter(Descriptor descriptor,
+                             long keyCount,
+                             long repairedAt,
+                             TimeUUID pendingRepair,
+                             boolean isTransient,
+                             TableMetadataRef metadata,
+                             MetadataCollector metadataCollector,
+                             SerializationHeader header,
+                             Collection<SSTableFlushObserver> observers,
+                             Set<Component> components)
+    {
+        super(descriptor, keyCount, repairedAt, pendingRepair, isTransient, metadata, metadataCollector, header, observers, components);
+    }
+
+    /**
+     * Appends partition data to this writer.
+     *
+     * @param partition the partition to write
+     * @return the created index entry if something was written, that is if {@code iterator}
+     * wasn't empty, {@code null} otherwise.
+     * @throws FSWriteError if write to the dataFile fails
+     */
+    @Override
+    public final RIE append(UnfilteredRowIterator partition)

Review Comment:
   There's only one use of the returned index entry, and it does not require the precise type. Can we make this return `AbstractRowIndexEntry` and do away with the parameter?



##########
src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java:
##########
@@ -153,183 +153,228 @@ private CompressionParams compressionFor(final OperationType opType)
 
     public void mark()
     {
-        dataMark = dataFile.mark();
-        iwriter.mark();
+        dataMark = dataWriter.mark();
+        indexWriter.mark();
     }
 
     public void resetAndTruncate()
     {
-        dataFile.resetAndTruncate(dataMark);
-        iwriter.resetAndTruncate();
-    }
-
-    /**
-     * Perform sanity checks on @param decoratedKey and @return the position in the data file before any data is written
-     */
-    protected long beforeAppend(DecoratedKey decoratedKey)
-    {
-        assert decoratedKey != null : "Keys must not be null"; // empty keys ARE allowed b/c of indexed column values
-        if (lastWrittenKey != null && lastWrittenKey.compareTo(decoratedKey) >= 0)
-            throw new RuntimeException("Last written key " + lastWrittenKey + " >= current key " + decoratedKey + " writing into " + getFilename());
-        return (lastWrittenKey == null) ? 0 : dataFile.position();
-    }
-
-    private void afterAppend(DecoratedKey decoratedKey, long dataEnd, RowIndexEntry index, ByteBuffer indexInfo) throws IOException
-    {
-        metadataCollector.addKey(decoratedKey.getKey());
-        lastWrittenKey = decoratedKey;
-        last = lastWrittenKey;
-        if (first == null)
-            first = lastWrittenKey;
-
-        if (logger.isTraceEnabled())
-            logger.trace("wrote {} at {}", decoratedKey, dataEnd);
-        iwriter.append(decoratedKey, index, dataEnd, indexInfo);
+        dataWriter.resetAndTruncate(dataMark);
+        indexWriter.resetAndTruncate();
     }
 
     /**
      * Appends partition data to this writer.
      *
-     * @param iterator the partition to write
+     * @param partition the partition to write
      * @return the created index entry if something was written, that is if {@code iterator}
      * wasn't empty, {@code null} otherwise.
-     *
-     * @throws FSWriteError if a write to the dataFile fails
+     * @throws FSWriteError if write to the dataFile fails
      */
-    public RowIndexEntry append(UnfilteredRowIterator iterator)
+    @Override
+    public final RowIndexEntry append(UnfilteredRowIterator partition)
     {
-        DecoratedKey key = iterator.partitionKey();
-
-        if (key.getKey().remaining() > FBUtilities.MAX_UNSIGNED_SHORT)
-        {
-            logger.error("Key size {} exceeds maximum of {}, skipping row", key.getKey().remaining(), FBUtilities.MAX_UNSIGNED_SHORT);
-            return null;
-        }
-
-        if (iterator.isEmpty())
+        if (partition.isEmpty())
             return null;
 
-        long startPosition = beforeAppend(key);
-        observers.forEach((o) -> o.startPartition(key, columnIndexWriter.getInitialPosition(), iwriter.indexFile.position()));
+        try
+        {
+            if (!verifyPartition(partition.partitionKey()))
+                return null;
 
-        //Reuse the writer for each row
-        columnIndexWriter.reset(DatabaseDescriptor.getColumnIndexCacheSize(), DatabaseDescriptor.getColumnIndexSize());
+            startPartition(partition.partitionKey(), partition.partitionLevelDeletion());
 
-        try (UnfilteredRowIterator collecting = Transformation.apply(iterator, new StatsCollector(metadataCollector)))
-        {
-            columnIndexWriter.buildRowIndex(collecting);
+            RowIndexEntry indexEntry;
+            if (header.hasStatic())
+                addStaticRow(partition.partitionKey(), partition.staticRow());
 
-            // afterAppend() writes the partition key before the first RowIndexEntry - so we have to add it's
-            // serialized size to the index-writer position
-            long indexFilePosition = ByteBufferUtil.serializedSizeWithShortLength(key.getKey()) + iwriter.indexFile.position();
+            while (partition.hasNext())
+                addUnfiltered(partition.partitionKey(), partition.next());
 
-            RowIndexEntry entry = RowIndexEntry.create(startPosition, indexFilePosition,
-                                                       collecting.partitionLevelDeletion(),
-                                                       columnIndexWriter.headerLength,
-                                                       columnIndexWriter.columnIndexCount,
-                                                       columnIndexWriter.indexInfoSerializedSize(),
-                                                       columnIndexWriter.indexSamples(),
-                                                       columnIndexWriter.offsets(),
-                                                       rowIndexEntrySerializer.indexInfoSerializer());
+            indexEntry = endPartition(partition.partitionKey(), partition.partitionLevelDeletion());
 
-            long endPosition = dataFile.position();
-            long rowSize = endPosition - startPosition;
-            maybeLogLargePartitionWarning(key, rowSize);
-            maybeLogManyTombstonesWarning(key, metadataCollector.totalTombstones);
-            metadataCollector.addPartitionSizeInBytes(rowSize);
-            afterAppend(key, endPosition, entry, columnIndexWriter.buffer());
-            return entry;
+            return indexEntry;
         }
         catch (BufferOverflowException boe)
         {
-            throw new PartitionSerializationException(iterator, boe);
+            throw new PartitionSerializationException(partition, boe);
         }
         catch (IOException e)
         {
-            throw new FSWriteError(e, dataFile.getPath());
+            throw new FSWriteError(e, getFilename());
         }
     }
 
-    private void maybeLogLargePartitionWarning(DecoratedKey key, long rowSize)
+    private boolean verifyPartition(DecoratedKey key)
     {
-        if (rowSize > DatabaseDescriptor.getCompactionLargePartitionWarningThreshold())
+        assert key != null : "Keys must not be null"; // empty keys ARE allowed b/c of indexed column values
+
+        if (key.getKey().remaining() > FBUtilities.MAX_UNSIGNED_SHORT)
         {
-            String keyString = metadata().partitionKeyType.getString(key.getKey());
-            logger.warn("Writing large partition {}/{}:{} ({}) to sstable {}", metadata.keyspace, metadata.name, keyString, FBUtilities.prettyPrintMemory(rowSize), getFilename());
+            logger.error("Key size {} exceeds maximum of {}, skipping row", key.getKey().remaining(), FBUtilities.MAX_UNSIGNED_SHORT);
+            return false;
         }
+
+        if (lastWrittenKey != null && lastWrittenKey.compareTo(key) >= 0)
+            throw new RuntimeException(String.format("Last written key %s >= current key %s, writing into %s", lastWrittenKey, key, getFilename()));
+
+        return true;
     }
 
-    private void maybeLogManyTombstonesWarning(DecoratedKey key, int tombstoneCount)
+    private void startPartition(DecoratedKey key, DeletionTime partitionLevelDeletion) throws IOException
     {
-        if (tombstoneCount > DatabaseDescriptor.getCompactionTombstoneWarningThreshold())
-        {
-            String keyString = metadata().partitionKeyType.getString(key.getKey());
-            logger.warn("Writing {} tombstones to {}/{}:{} in sstable {}", tombstoneCount, metadata.keyspace, metadata.name, keyString, getFilename());
-        }
+        partitionWriter.start(key, partitionLevelDeletion);
+        metadataCollector.update(partitionLevelDeletion);
+
+        onStartPartition(key);
     }
 
-    private static class StatsCollector extends Transformation
+    private void addStaticRow(DecoratedKey key, Row row) throws IOException
     {
-        private final MetadataCollector collector;
-        private int cellCount;
+        guardCollectionSize(metadata(), key, row);
+
+        partitionWriter.addStaticRow(row);
+        if (!row.isEmpty())
+            Rows.collectStats(row, metadataCollector);
 
-        StatsCollector(MetadataCollector collector)
+        onStaticRow(row);
+    }
+
+    private void addUnfiltered(DecoratedKey key, Unfiltered unfiltered) throws IOException
+    {
+        if (unfiltered.isRow())
         {
-            this.collector = collector;
+            addRow(key, (Row) unfiltered);
         }
-
-        @Override
-        public Row applyToStatic(Row row)
+        else
         {
-            if (!row.isEmpty())
-                cellCount += Rows.collectStats(row, collector);
-            return row;
+            assert unfiltered.isRangeTombstoneMarker();

Review Comment:
   Nit: I would remove this -- the cast will check anyway, and this is a rather hot path.



##########
src/java/org/apache/cassandra/config/CassandraRelevantProperties.java:
##########
@@ -331,7 +331,9 @@
      *
      * If only keyspaces are specified, mutations for all tables in such keyspace will be replayed
      * */
-    COMMIT_LOG_REPLAY_LIST("cassandra.replayList", null)
+    COMMIT_LOG_REPLAY_LIST("cassandra.replayList", null),
+
+    SSTABLE_FORMAT_DEFAULT("cassandra.sstable.format.default", "BIG")

Review Comment:
   Nit: I personally prefer the default to be given in context, i.e. in the format class (which will make the new `getEnum` unnecessary).



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableFlushObserver.java:
##########
@@ -57,4 +58,6 @@
      * Called when all data is written to the file and it's ready to be finished up.
      */
     void complete();
+
+    void staticRow(Row staticRow);

Review Comment:
   This is best placed after `startPartition` and should have JavaDoc.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081198496


##########
src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriterBuilder.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.big;
+
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.sstable.DataComponent;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SortedTableWriterBuilder;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.Throwables;
+
+public class BigTableWriterBuilder extends SortedTableWriterBuilder<RowIndexEntry, BigFormatPartitionWriter, BigTableWriter, BigTableWriterBuilder>
+{
+    private RowIndexEntry.IndexSerializer rowIndexEntrySerializer;
+    private BigTableWriter.IndexWriter indexWriter;
+    private SequentialWriter dataWriter;
+    private BigFormatPartitionWriter partitionWriter;
+
+    public BigTableWriterBuilder(Descriptor descriptor)
+    {
+        super(descriptor);
+    }
+
+    @Override
+    public SequentialWriter getDataWriter()
+    {
+        return dataWriter;
+    }
+
+    @Override
+    public BigFormatPartitionWriter getPartitionWriter()
+    {
+        return partitionWriter;
+    }
+
+    public RowIndexEntry.IndexSerializer getRowIndexEntrySerializer()
+    {
+        return rowIndexEntrySerializer;
+    }
+
+    public BigTableWriter.IndexWriter getIndexWriter()
+    {
+        return indexWriter;
+    }
+
+    @Override
+    protected BigTableWriter buildInternal(LifecycleNewTracker lifecycleNewTracker)
+    {
+        try
+        {
+            rowIndexEntrySerializer = new RowIndexEntry.Serializer(descriptor.version, getSerializationHeader());
+            dataWriter = DataComponent.buildWriter(descriptor,

Review Comment:
   I did it that way in order to avoid long arguments lists on constructor. Also having say, data file handle and partition writer passed as separate fields in the constructor would theoretically let pass a handle to a different file than the provided partition writer operates on. As far as I understand, those two parameters are inherently related and passing them always together (in a single builder) is more consistent to me.
   
   The resource fields are cleaned in finally block, so the builder can be safely reused. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1082594688


##########
src/java/org/apache/cassandra/utils/Throwables.java:
##########
@@ -189,13 +189,19 @@ public static Throwable perform(Throwable accumulate, String filePath, FileOpTyp
     }
 
     public static void closeAndAddSuppressed(@Nonnull Throwable t, AutoCloseable... closeables)
+    {
+        closeAndAddSuppressed(t, Arrays.asList(closeables));
+    }
+
+    public static void closeAndAddSuppressed(@Nonnull Throwable t, Iterable<AutoCloseable> closeables)
     {
         Preconditions.checkNotNull(t);
         for (AutoCloseable closeable : closeables)
         {
             try
             {
-                closeable.close();
+                if (closeable != null)

Review Comment:
   9ffff01f36d271686bac16b932b57c7d918a355d as well



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1090298510


##########
src/java/org/apache/cassandra/io/sstable/Component.java:
##########
@@ -35,43 +41,49 @@
 {
     public static final char separator = '-';
 
-    final static EnumSet<Type> TYPES = EnumSet.allOf(Type.class);
+    final static ImmutableSet<Type> TYPES = Type.all;
 
     /**
      * WARNING: Be careful while changing the names or string representation of the enum
      * members. Streaming code depends on the names during streaming (Ref: CASSANDRA-14556).
      */
-    public enum Type
+    public static class Type
     {
+        private final static List<Type> singletonsCollector = new ArrayList<>(11);
         // the base data for an sstable: the remaining components can be regenerated
         // based on the data component
-        DATA("Data.db"),
-        // index of the row keys with pointers to their positions in the data file
-        PRIMARY_INDEX("Index.db"),
-        // serialized bloom filter for the row keys in the sstable
-        FILTER("Filter.db"),
+        public static final Type DATA = new Type("DATA", "Data.db", name -> Component.DATA);
         // file to hold information about uncompressed data length, chunk offsets etc.
-        COMPRESSION_INFO("CompressionInfo.db"),
+        public static final Type COMPRESSION_INFO = new Type("COMPRESSION_INFO", "CompressionInfo.db", name -> Component.COMPRESSION_INFO);
         // statistical metadata about the content of the sstable
-        STATS("Statistics.db"),
+        public static final Type STATS = new Type("STATS", "Statistics.db", name -> Component.STATS);
+        // serialized bloom filter for the row keys in the sstable
+        public static final Type FILTER = new Type("FILTER", "Filter.db", name -> Component.FILTER);
         // holds CRC32 checksum of the data file
-        DIGEST("Digest.crc32"),
+        public static final Type DIGEST = new Type("DIGEST","Digest.crc32", name -> Component.DIGEST);
         // holds the CRC32 for chunks in an a uncompressed file.
-        CRC("CRC.db"),
-        // holds SSTable Index Summary (sampling of Index component)
-        SUMMARY("Summary.db"),
+        public static final Type CRC = new Type("CRC","CRC.db", name -> Component.CRC);
         // table of contents, stores the list of all components for the sstable
-        TOC("TOC.txt"),
+        public static final Type TOC = new Type("TOC","TOC.txt", name -> Component.TOC);
         // built-in secondary index (may be multiple per sstable)
-        SECONDARY_INDEX("SI_.*.db"),
+        public static final Type SECONDARY_INDEX = new Type("SECONDARY_INDEX","SI_.*.db", name -> new Component(Type.SECONDARY_INDEX, name));
         // custom component, used by e.g. custom compaction strategy
-        CUSTOM(null);
+        public static final Type CUSTOM = new Type("CUSTOM",null, name -> new Component(Type.CUSTOM, name));
 
-        final String repr;
+        public static final ImmutableSet<Type> all = ImmutableSet.copyOf(singletonsCollector);

Review Comment:
   https://github.com/apache/cassandra/pull/2064/commits/5689be8c3a242074a29663be46a06c820020416a



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1088788127


##########
src/java/org/apache/cassandra/io/sstable/Component.java:
##########
@@ -35,43 +41,49 @@
 {
     public static final char separator = '-';
 
-    final static EnumSet<Type> TYPES = EnumSet.allOf(Type.class);
+    final static ImmutableSet<Type> TYPES = Type.all;
 
     /**
      * WARNING: Be careful while changing the names or string representation of the enum
      * members. Streaming code depends on the names during streaming (Ref: CASSANDRA-14556).
      */
-    public enum Type
+    public static class Type
     {
+        private final static List<Type> singletonsCollector = new ArrayList<>(11);
         // the base data for an sstable: the remaining components can be regenerated
         // based on the data component
-        DATA("Data.db"),
-        // index of the row keys with pointers to their positions in the data file
-        PRIMARY_INDEX("Index.db"),
-        // serialized bloom filter for the row keys in the sstable
-        FILTER("Filter.db"),
+        public static final Type DATA = new Type("DATA", "Data.db", name -> Component.DATA);
         // file to hold information about uncompressed data length, chunk offsets etc.
-        COMPRESSION_INFO("CompressionInfo.db"),
+        public static final Type COMPRESSION_INFO = new Type("COMPRESSION_INFO", "CompressionInfo.db", name -> Component.COMPRESSION_INFO);
         // statistical metadata about the content of the sstable
-        STATS("Statistics.db"),
+        public static final Type STATS = new Type("STATS", "Statistics.db", name -> Component.STATS);
+        // serialized bloom filter for the row keys in the sstable
+        public static final Type FILTER = new Type("FILTER", "Filter.db", name -> Component.FILTER);
         // holds CRC32 checksum of the data file
-        DIGEST("Digest.crc32"),
+        public static final Type DIGEST = new Type("DIGEST","Digest.crc32", name -> Component.DIGEST);
         // holds the CRC32 for chunks in an a uncompressed file.
-        CRC("CRC.db"),
-        // holds SSTable Index Summary (sampling of Index component)
-        SUMMARY("Summary.db"),
+        public static final Type CRC = new Type("CRC","CRC.db", name -> Component.CRC);
         // table of contents, stores the list of all components for the sstable
-        TOC("TOC.txt"),
+        public static final Type TOC = new Type("TOC","TOC.txt", name -> Component.TOC);
         // built-in secondary index (may be multiple per sstable)
-        SECONDARY_INDEX("SI_.*.db"),
+        public static final Type SECONDARY_INDEX = new Type("SECONDARY_INDEX","SI_.*.db", name -> new Component(Type.SECONDARY_INDEX, name));
         // custom component, used by e.g. custom compaction strategy
-        CUSTOM(null);
+        public static final Type CUSTOM = new Type("CUSTOM",null, name -> new Component(Type.CUSTOM, name));
 
-        final String repr;
+        public static final ImmutableSet<Type> all = ImmutableSet.copyOf(singletonsCollector);

Review Comment:
   This being final and immutable means we can't register a new type, doesn't it?
   
   E.g. the `BigComponentType`s probably aren't here.
   
   Perhaps wrap it an `UnmodifiableSet` like the components?



##########
src/java/org/apache/cassandra/io/util/MmappedRegions.java:
##########
@@ -140,6 +147,25 @@ public void extend(long length)
         copy = new State(state);
     }
 
+    public void extend(CompressionMetadata compressionMetadata)
+    {
+        assert !isCopy() : "Copies cannot be extended";
+
+        if (compressionMetadata.dataLength <= state.length)

Review Comment:
   Nit: there should be a comment why `dataLength` is used here and `compressedFileLength` below. (If I'm not mistaken the data length might be truncated to shorter than the end of the last chunk, while we do want to map the whole chunk to be able to read it).



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -1415,7 +1418,7 @@ public boolean mayContainAssumingKeyIsInRange(DecoratedKey key)
     protected static final class InstanceTidier implements Tidy
     {
         private final Descriptor descriptor;
-        private final TableId tableId;
+        private final Owner owner;

Review Comment:
   Does it make sense for this to be a weak reference? Can the CFS be collected before the sstables?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1094783730


##########
src/java/org/apache/cassandra/service/CacheService.java:
##########
@@ -467,9 +467,15 @@ public Future<Pair<KeyCacheKey, AbstractRowIndexEntry>> deserialize(DataInputPlu
                                                    ? SSTableIdFactory.instance.fromBytes(ByteBufferUtil.readWithShortLength(input))
                                                    : new SequenceBasedSSTableId(generation); // Backwards compatibility for "int based generation sstables"
             int typeOrdinal = input.readByte();
-            if (typeOrdinal < 0 || typeOrdinal >= SSTableFormat.Type.values().length)
-                throw new IOException("Failed to deserialize key of key cache - invalid type ordinal " + typeOrdinal);
-            SSTableFormat.Type type = SSTableFormat.Type.values()[typeOrdinal];
+            SSTableFormat.Type type;
+            try
+            {
+                type = SSTableFormat.Type.getByOrdinal(typeOrdinal);

Review Comment:
   https://github.com/apache/cassandra/pull/2064/commits/80ef51bf6a43b9925949fa9e74c384dae0a5d6e9



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081449080


##########
src/java/org/apache/cassandra/io/sstable/format/FilterComponent.java:
##########
@@ -90,4 +97,63 @@ public static void saveOrDeleteCorrupted(Descriptor descriptor, IFilter filter)
             throw ex;
         }
     }
+
+    /**
+     * Optionally loads a Bloom filter. If the filter is not needed (FP chance is neglectable), it sets
+     * {@link AlwaysPresentFilter} as a filter in the builder. If the filter is expected to be recreated for various
+     * reasons, it leaves it {@code null} (unchanged). Otherwise, it attempts to load the filter, and if it succeeds,
+     * it is set in the builder. If a filter fails to load, it is left {@code null} (unchanged) meaning that it should

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1072169155


##########
src/java/org/apache/cassandra/io/sstable/format/FilterComponent.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileInputStreamPlus;
+import org.apache.cassandra.io.util.FileOutputStreamPlus;
+import org.apache.cassandra.utils.BloomFilterSerializer;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+
+public class FilterComponent
+{
+    private static final Logger logger = LoggerFactory.getLogger(FilterComponent.class);
+
+    private FilterComponent()
+    {
+    }
+
+    /**
+     * Load bloom filter from Filter.db file.
+     */
+    public static IFilter load(Descriptor descriptor) throws IOException
+    {
+        File filterFile = descriptor.fileFor(Component.FILTER);
+
+        if (!filterFile.exists())
+            return null;
+
+        if (filterFile.length() == 0)
+            return FilterFactory.AlwaysPresent;
+
+        try (FileInputStreamPlus stream = descriptor.fileFor(Component.FILTER).newInputStream())
+        {
+            return BloomFilterSerializer.forVersion(descriptor.version.hasOldBfFormat()).deserialize(stream);
+        }
+        catch (IOException ex)
+        {
+            throw new IOException("Failed to load Bloom filter for SSTable: " + descriptor.baseFilename(), ex);
+        }
+    }
+
+    public static void save(IFilter filter, Descriptor descriptor) throws IOException
+    {
+        File filterFile = descriptor.fileFor(Component.FILTER);
+        try (FileOutputStreamPlus stream = filterFile.newOutputStream(File.WriteMode.OVERWRITE))
+        {
+            filter.serialize(stream, descriptor.version.hasOldBfFormat());
+            stream.flush();
+            stream.sync(); // is it needed if we close the file right after that?

Review Comment:
   thanks



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1073503101


##########
src/java/org/apache/cassandra/db/compaction/Scrubber.java:
##########
@@ -564,20 +565,27 @@ public ScrubResult(Scrubber scrubber)
     /**
      * During 2.x migration, under some circumstances rows might have gotten duplicated.
      * Merging iterator merges rows with same clustering.
-     *
+     * <p>
      * For more details, refer to CASSANDRA-12144.
      */
-    private static class RowMergingSSTableIterator extends WrappingUnfilteredRowIterator
+    private static class RowMergingSSTableIterator extends UnmodifiableIterator<Unfiltered> implements WrappingUnfilteredRowIterator

Review Comment:
   We do not have to supply anything, the Java's original implementation of `Iterator.remove` is:
   
   ```
       default void remove() {
           throw new UnsupportedOperationException("remove");
       }
   ```
   
   `UnmodifiableIterator` just makes it final
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1071937289


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java:
##########
@@ -36,6 +39,8 @@
     SSTableWriter.Factory getWriterFactory();
     SSTableReader.Factory getReaderFactory();
 
+    Set<Component> supportedComponents();

Review Comment:
   maybe will change this to just `all`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081059888


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -808,32 +809,12 @@ 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);
+            tidy.runOnClose = () -> {

Review Comment:
   Great.
   
   Could you add comments explaining that lambdas, if certain that they don't reference `this`, are okay?
   
   As a nit, `existing` can be checked immediately, and `tidy.runOnClose` can be set to the argument directly if it's null.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081325365


##########
src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java:
##########
@@ -82,46 +82,46 @@ public void build()
                 PerSSTableIndexWriter indexWriter = SASIIndex.newWriter(keyValidator, sstable.descriptor, indexes, OperationType.COMPACTION);
                 targetDirectory = indexWriter.getDescriptor().directory.path();
 
-                long previousKeyPosition = 0;
-                try (KeyIterator keys = new KeyIterator(sstable.descriptor, cfs.metadata()))
+                try (KeyReader keys = sstable.keyReader())
                 {
-                    while (keys.hasNext())
+                    while (!keys.isExhausted())
                     {
                         if (isStopRequested())
                             throw new CompactionInterruptedException(getCompactionInfo());
 
-                        final DecoratedKey key = keys.next();
-                        final long keyPosition = keys.getKeyPosition();
+                        final DecoratedKey key = sstable.decorateKey(keys.key());
+                        final long keyPosition = keys.keyPositionForSecondaryIndex();
 
-                        indexWriter.startPartition(key, keyPosition);
+                        indexWriter.startPartition(key, keys.dataPosition(), keyPosition);
 
-                        try
-                        {
-                            long position = sstable.getPosition(key, SSTableReader.Operator.EQ);
-                            dataFile.seek(position);
-                            ByteBufferUtil.readWithShortLength(dataFile); // key
+                        long position = sstable.getPosition(key, SSTableReader.Operator.EQ);

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081323501


##########
src/java/org/apache/cassandra/db/compaction/CompactionManager.java:
##########
@@ -1890,22 +1890,17 @@ public void run()
         return executor.submitIfRunning(runnable, "cache write");
     }
 
-    public List<SSTableReader> runIndexSummaryRedistribution(IndexSummaryRedistribution redistribution) throws IOException
+    public <T, E extends Throwable> T runWithActiveCompactions(Holder holder, ThrowingSupplier<T, E> callable) throws E

Review Comment:
   looks good



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -1033,6 +1012,14 @@ protected void closeInternalComponent(AutoCloseable closeable)
         }
     }
 
+    /**
+     * This method is expected to close the components which occupy memory but are not needed when we just want to
+     * stream the components (for example, when SSTable is opened with SSTableLoader). The method should call
+     * {@link #closeInternalComponent(AutoCloseable)} for each such component. Leaving the implementation empty is
+     * valid, but may impact memory usage.
+     */
+    public abstract void releaseComponents();

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081427200


##########
src/java/org/apache/cassandra/io/sstable/format/SortedTableWriter.java:
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.IOException;
+import java.nio.BufferOverflowException;
+import java.util.Collection;
+import java.util.Set;
+import java.util.function.Consumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.rows.PartitionSerializationException;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundaryMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.TimeUUID;
+
+public abstract class SortedTableWriter<P extends SortedTablePartitionWriter, RIE extends AbstractRowIndexEntry> extends SSTableWriter<RIE>
+{
+    private final static Logger logger = LoggerFactory.getLogger(SortedTableWriter.class);
+
+    protected SequentialWriter dataWriter;
+    protected P partitionWriter;
+    private DecoratedKey lastWrittenKey;
+    private DataPosition dataMark;
+
+    public SortedTableWriter(Descriptor descriptor,
+                             long keyCount,
+                             long repairedAt,
+                             TimeUUID pendingRepair,
+                             boolean isTransient,
+                             TableMetadataRef metadata,
+                             MetadataCollector metadataCollector,
+                             SerializationHeader header,
+                             Collection<SSTableFlushObserver> observers,
+                             Set<Component> components)
+    {
+        super(descriptor, keyCount, repairedAt, pendingRepair, isTransient, metadata, metadataCollector, header, observers, components);
+    }
+
+    /**
+     * Appends partition data to this writer.
+     *
+     * @param partition the partition to write
+     * @return the created index entry if something was written, that is if {@code iterator}
+     * wasn't empty, {@code null} otherwise.
+     * @throws FSWriteError if write to the dataFile fails
+     */
+    @Override
+    public final RIE append(UnfilteredRowIterator partition)

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081442820


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -1983,39 +1380,36 @@ public void addTo(Ref.IdentityCollection identities)
     {
         identities.add(this);
         identities.add(tidy.globalRef);
-        dfile.addTo(identities);
-        ifile.addTo(identities);
-        bf.addTo(identities);
-        indexSummary.addTo(identities);
-
+        tidy.closeables.forEach(c -> {
+            if (c instanceof SharedCloseable)
+                ((SharedCloseable) c).addTo(identities);
+        });
     }
 
     public boolean maybePresent(DecoratedKey key)

Review Comment:
   This is the method by which we check if a tombstone needs to be preserved because data may be present in some older sstable. It should return true if the partition can be contained in the sstable, and false only if it definitely isn't contained, and needs to be as fast as possible (it is called for every partition with a tombstone for every older sstable during compaction).
   
   So if outside of range (which should already be checked btw, it may be worthwhile to see if this check needs to be here) or bloom filter is negative, we _can_ return false, and returning true otherwise is fine (acceptable false positive frequency) as long as we do have a bloom filter. If we don't, we need to do more work or tombstones will never be removed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1094294799


##########
src/java/org/apache/cassandra/io/sstable/BloomFilterSupport.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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 org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.utils.IFilter;
+
+public interface BloomFilterSupport<T extends SSTableReader & BloomFilterSupport<T>>
+{
+    IFilter getBloomFilter();
+
+    BloomFilterTracker getBloomFilterTracker();
+
+    T cloneAndReplace(IFilter newBloomFilter);

Review Comment:
   removed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1094297286


##########
src/java/org/apache/cassandra/config/Config.java:
##########
@@ -352,6 +354,8 @@ public MemtableOptions()
 
     public String[] data_file_directories = new String[0];
 
+    public Map<String, SSTableFormatConfig> sstable_formats = ImmutableMap.of("big", new SSTableFormatConfig(BigFormat.class.getName(), Collections.emptyMap()));

Review Comment:
   I didn't know that, thanks



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1072215878


##########
src/java/org/apache/cassandra/io/sstable/Descriptor.java:
##########
@@ -423,6 +425,19 @@ public boolean isCompatible()
         return version.isCompatible();
     }
 
+    public Set<Component> discoverComponents()
+    {
+        Set<Component.Type> knownTypes = Sets.difference(Component.TYPES, Collections.singleton(Component.Type.CUSTOM));
+        Set<Component> components = Sets.newHashSetWithExpectedSize(knownTypes.size());
+        for (Component.Type componentType : knownTypes)
+        {
+            Component component = new Component(componentType);

Review Comment:
   I'll add a singletons set to the `Component`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1072243414


##########
src/java/org/apache/cassandra/io/sstable/SSTable.java:
##########
@@ -155,9 +155,9 @@ public DecoratedKey decorateKey(ByteBuffer key)
      */
     public static DecoratedKey getMinimalKey(DecoratedKey key)

Review Comment:
   actually, no need this method to be in `SSTable`, I'll move it to `DecoratedKey`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1071940428


##########
src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java:
##########
@@ -374,4 +380,20 @@ public ClusteringPrefix<?> getLowerBoundPrefixFromCache(DecoratedKey partitionKe
         }
 
     }
-}
+
+    @Override
+    public IScrubber getScrubber(LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(metadata());

Review Comment:
   I wanted to avoid requiring CFS as an argument but you are right.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1071934622


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java:
##########
@@ -36,6 +39,8 @@
     SSTableWriter.Factory getWriterFactory();
     SSTableReader.Factory getReaderFactory();
 
+    Set<Component> supportedComponents();

Review Comment:
   OTOH, it can be also, something like `written`, or `produced` as this is also all the components that can be created when sstable is written.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081348044


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableFlushObserver.java:
##########
@@ -57,4 +58,6 @@
      * Called when all data is written to the file and it's ready to be finished up.
      */
     void complete();
+
+    void staticRow(Row staticRow);

Review Comment:
   looks good



##########
src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java:
##########
@@ -153,183 +153,228 @@ private CompressionParams compressionFor(final OperationType opType)
 
     public void mark()
     {
-        dataMark = dataFile.mark();
-        iwriter.mark();
+        dataMark = dataWriter.mark();
+        indexWriter.mark();
     }
 
     public void resetAndTruncate()
     {
-        dataFile.resetAndTruncate(dataMark);
-        iwriter.resetAndTruncate();
-    }
-
-    /**
-     * Perform sanity checks on @param decoratedKey and @return the position in the data file before any data is written
-     */
-    protected long beforeAppend(DecoratedKey decoratedKey)
-    {
-        assert decoratedKey != null : "Keys must not be null"; // empty keys ARE allowed b/c of indexed column values
-        if (lastWrittenKey != null && lastWrittenKey.compareTo(decoratedKey) >= 0)
-            throw new RuntimeException("Last written key " + lastWrittenKey + " >= current key " + decoratedKey + " writing into " + getFilename());
-        return (lastWrittenKey == null) ? 0 : dataFile.position();
-    }
-
-    private void afterAppend(DecoratedKey decoratedKey, long dataEnd, RowIndexEntry index, ByteBuffer indexInfo) throws IOException
-    {
-        metadataCollector.addKey(decoratedKey.getKey());
-        lastWrittenKey = decoratedKey;
-        last = lastWrittenKey;
-        if (first == null)
-            first = lastWrittenKey;
-
-        if (logger.isTraceEnabled())
-            logger.trace("wrote {} at {}", decoratedKey, dataEnd);
-        iwriter.append(decoratedKey, index, dataEnd, indexInfo);
+        dataWriter.resetAndTruncate(dataMark);
+        indexWriter.resetAndTruncate();
     }
 
     /**
      * Appends partition data to this writer.
      *
-     * @param iterator the partition to write
+     * @param partition the partition to write
      * @return the created index entry if something was written, that is if {@code iterator}
      * wasn't empty, {@code null} otherwise.
-     *
-     * @throws FSWriteError if a write to the dataFile fails
+     * @throws FSWriteError if write to the dataFile fails
      */
-    public RowIndexEntry append(UnfilteredRowIterator iterator)
+    @Override
+    public final RowIndexEntry append(UnfilteredRowIterator partition)
     {
-        DecoratedKey key = iterator.partitionKey();
-
-        if (key.getKey().remaining() > FBUtilities.MAX_UNSIGNED_SHORT)
-        {
-            logger.error("Key size {} exceeds maximum of {}, skipping row", key.getKey().remaining(), FBUtilities.MAX_UNSIGNED_SHORT);
-            return null;
-        }
-
-        if (iterator.isEmpty())
+        if (partition.isEmpty())
             return null;
 
-        long startPosition = beforeAppend(key);
-        observers.forEach((o) -> o.startPartition(key, columnIndexWriter.getInitialPosition(), iwriter.indexFile.position()));
+        try
+        {
+            if (!verifyPartition(partition.partitionKey()))
+                return null;
 
-        //Reuse the writer for each row
-        columnIndexWriter.reset(DatabaseDescriptor.getColumnIndexCacheSize(), DatabaseDescriptor.getColumnIndexSize());
+            startPartition(partition.partitionKey(), partition.partitionLevelDeletion());
 
-        try (UnfilteredRowIterator collecting = Transformation.apply(iterator, new StatsCollector(metadataCollector)))
-        {
-            columnIndexWriter.buildRowIndex(collecting);
+            RowIndexEntry indexEntry;
+            if (header.hasStatic())
+                addStaticRow(partition.partitionKey(), partition.staticRow());
 
-            // afterAppend() writes the partition key before the first RowIndexEntry - so we have to add it's
-            // serialized size to the index-writer position
-            long indexFilePosition = ByteBufferUtil.serializedSizeWithShortLength(key.getKey()) + iwriter.indexFile.position();
+            while (partition.hasNext())
+                addUnfiltered(partition.partitionKey(), partition.next());
 
-            RowIndexEntry entry = RowIndexEntry.create(startPosition, indexFilePosition,
-                                                       collecting.partitionLevelDeletion(),
-                                                       columnIndexWriter.headerLength,
-                                                       columnIndexWriter.columnIndexCount,
-                                                       columnIndexWriter.indexInfoSerializedSize(),
-                                                       columnIndexWriter.indexSamples(),
-                                                       columnIndexWriter.offsets(),
-                                                       rowIndexEntrySerializer.indexInfoSerializer());
+            indexEntry = endPartition(partition.partitionKey(), partition.partitionLevelDeletion());
 
-            long endPosition = dataFile.position();
-            long rowSize = endPosition - startPosition;
-            maybeLogLargePartitionWarning(key, rowSize);
-            maybeLogManyTombstonesWarning(key, metadataCollector.totalTombstones);
-            metadataCollector.addPartitionSizeInBytes(rowSize);
-            afterAppend(key, endPosition, entry, columnIndexWriter.buffer());
-            return entry;
+            return indexEntry;
         }
         catch (BufferOverflowException boe)
         {
-            throw new PartitionSerializationException(iterator, boe);
+            throw new PartitionSerializationException(partition, boe);
         }
         catch (IOException e)
         {
-            throw new FSWriteError(e, dataFile.getPath());
+            throw new FSWriteError(e, getFilename());
         }
     }
 
-    private void maybeLogLargePartitionWarning(DecoratedKey key, long rowSize)
+    private boolean verifyPartition(DecoratedKey key)
     {
-        if (rowSize > DatabaseDescriptor.getCompactionLargePartitionWarningThreshold())
+        assert key != null : "Keys must not be null"; // empty keys ARE allowed b/c of indexed column values
+
+        if (key.getKey().remaining() > FBUtilities.MAX_UNSIGNED_SHORT)
         {
-            String keyString = metadata().partitionKeyType.getString(key.getKey());
-            logger.warn("Writing large partition {}/{}:{} ({}) to sstable {}", metadata.keyspace, metadata.name, keyString, FBUtilities.prettyPrintMemory(rowSize), getFilename());
+            logger.error("Key size {} exceeds maximum of {}, skipping row", key.getKey().remaining(), FBUtilities.MAX_UNSIGNED_SHORT);
+            return false;
         }
+
+        if (lastWrittenKey != null && lastWrittenKey.compareTo(key) >= 0)
+            throw new RuntimeException(String.format("Last written key %s >= current key %s, writing into %s", lastWrittenKey, key, getFilename()));
+
+        return true;
     }
 
-    private void maybeLogManyTombstonesWarning(DecoratedKey key, int tombstoneCount)
+    private void startPartition(DecoratedKey key, DeletionTime partitionLevelDeletion) throws IOException
     {
-        if (tombstoneCount > DatabaseDescriptor.getCompactionTombstoneWarningThreshold())
-        {
-            String keyString = metadata().partitionKeyType.getString(key.getKey());
-            logger.warn("Writing {} tombstones to {}/{}:{} in sstable {}", tombstoneCount, metadata.keyspace, metadata.name, keyString, getFilename());
-        }
+        partitionWriter.start(key, partitionLevelDeletion);
+        metadataCollector.update(partitionLevelDeletion);
+
+        onStartPartition(key);
     }
 
-    private static class StatsCollector extends Transformation
+    private void addStaticRow(DecoratedKey key, Row row) throws IOException
     {
-        private final MetadataCollector collector;
-        private int cellCount;
+        guardCollectionSize(metadata(), key, row);
+
+        partitionWriter.addStaticRow(row);
+        if (!row.isEmpty())
+            Rows.collectStats(row, metadataCollector);
 
-        StatsCollector(MetadataCollector collector)
+        onStaticRow(row);
+    }
+
+    private void addUnfiltered(DecoratedKey key, Unfiltered unfiltered) throws IOException
+    {
+        if (unfiltered.isRow())
         {
-            this.collector = collector;
+            addRow(key, (Row) unfiltered);
         }
-
-        @Override
-        public Row applyToStatic(Row row)
+        else
         {
-            if (!row.isEmpty())
-                cellCount += Rows.collectStats(row, collector);
-            return row;
+            assert unfiltered.isRangeTombstoneMarker();

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081343762


##########
src/java/org/apache/cassandra/io/sstable/format/FilterComponent.java:
##########
@@ -90,4 +97,63 @@ public static void saveOrDeleteCorrupted(Descriptor descriptor, IFilter filter)
             throw ex;
         }
     }
+
+    /**
+     * Optionally loads a Bloom filter. If the filter is not needed (FP chance is neglectable), it sets
+     * {@link AlwaysPresentFilter} as a filter in the builder. If the filter is expected to be recreated for various
+     * reasons, it leaves it {@code null} (unchanged). Otherwise, it attempts to load the filter, and if it succeeds,
+     * it is set in the builder. If a filter fails to load, it is left {@code null} (unchanged) meaning that it should
+     * be rebuilt.
+     */
+    public static IFilter maybeLoadBloomFilter(Descriptor descriptor, Set<Component> components, TableMetadata metadata, ValidationMetadata validationMetadata)
+    {
+        double currentFPChance = validationMetadata != null ? validationMetadata.bloomFilterFPChance : Double.NaN;
+        double desiredFPChance = metadata.params.bloomFilterFpChance;
+
+        IFilter filter = null;
+        if (!shouldUseBloomFilter(desiredFPChance))
+        {
+            if (logger.isTraceEnabled())
+                logger.trace("Bloom filter for {} will not be loaded because fpChance={} is neglectable", descriptor, desiredFPChance);
+
+            return FilterFactory.AlwaysPresent;
+        }
+        else if (!components.contains(Component.FILTER) || Double.isNaN(currentFPChance))
+        {
+            if (logger.isTraceEnabled())
+                logger.trace("Bloom filter for {} will not be loaded because filter component is missing or sstable lacks validation metadata", descriptor);
+
+            return null;
+        }
+        else if (!isFPChanceDiffNeglectable(desiredFPChance, currentFPChance) && rebuildFilterOnFPChanceChange)
+        {
+            if (logger.isTraceEnabled())
+                logger.trace("Bloom filter for {} will not be loaded because fpChance has changed from {} to {} and the filter should be recreated", descriptor, currentFPChance, desiredFPChance);
+
+            return null;
+        }
+
+        try
+        {
+            filter = load(descriptor);
+            if (filter == null || filter instanceof AlwaysPresentFilter)
+                logger.info("Bloom filter for {} is missing or invalid", descriptor);
+        }
+        catch (IOException ex)
+        {
+            logger.info("Bloom filter for " + descriptor + " could not be deserialized", ex);
+        }
+
+        return filter;
+    }
+
+    static boolean shouldUseBloomFilter(double fpChance)
+    {
+        return !(Math.abs(1 - fpChance) <= filterFPChanceTolerance);
+    }
+
+    static boolean isFPChanceDiffNeglectable(double fpChance1, double fpChance2)

Review Comment:
   ... and also in the trace message.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081320529


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -1973,17 +1970,36 @@ public void run()
                     if (logger.isTraceEnabled())
                         logger.trace("Async instance tidier for {}, after barrier", descriptor);
 
-                    if (bf != null)
-                        bf.close();
-                    if (summary != null)
-                        summary.close();
-                    if (runOnClose != null)
+                    Throwable exceptions = null;
+                    if (runOnClose != null) try
+                    {
                         runOnClose.run();
-                    if (dfile != null)
-                        dfile.close();
-                    if (ifile != null)
-                        ifile.close();
-                    globalRef.release();
+                    }
+                    catch (RuntimeException | Error ex)
+                    {
+                        logger.error("Failed to run on-close listeners for sstable " + descriptor.baseFilename(), ex);
+                        exceptions = ex;
+                    }
+
+                    Throwable closeExceptions = Throwables.close(null, closeables);

Review Comment:
   It's practically the same pattern, since you do have a check for this concrete part throwing exceptions.
   
   The reason for asking is to avoid relaxing `Throwables.close` -- we may lose an error message we care about for something that should not be null.
   
   Alternatively, we can add a `closeIgnoringNulls` method to `Throwables`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1083905332


##########
src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriterBuilder.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.big;
+
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.sstable.DataComponent;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SortedTableWriterBuilder;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.Throwables;
+
+public class BigTableWriterBuilder extends SortedTableWriterBuilder<RowIndexEntry, BigFormatPartitionWriter, BigTableWriter, BigTableWriterBuilder>
+{
+    private RowIndexEntry.IndexSerializer rowIndexEntrySerializer;
+    private BigTableWriter.IndexWriter indexWriter;
+    private SequentialWriter dataWriter;
+    private BigFormatPartitionWriter partitionWriter;
+
+    public BigTableWriterBuilder(Descriptor descriptor)
+    {
+        super(descriptor);
+    }
+
+    @Override
+    public SequentialWriter getDataWriter()
+    {
+        return dataWriter;
+    }
+
+    @Override
+    public BigFormatPartitionWriter getPartitionWriter()
+    {
+        return partitionWriter;
+    }
+
+    public RowIndexEntry.IndexSerializer getRowIndexEntrySerializer()
+    {
+        return rowIndexEntrySerializer;
+    }
+
+    public BigTableWriter.IndexWriter getIndexWriter()
+    {
+        return indexWriter;
+    }
+
+    @Override
+    protected BigTableWriter buildInternal(LifecycleNewTracker lifecycleNewTracker)
+    {
+        try
+        {
+            rowIndexEntrySerializer = new RowIndexEntry.Serializer(descriptor.version, getSerializationHeader());
+            dataWriter = DataComponent.buildWriter(descriptor,

Review Comment:
   This is overdoing it -- making sure that `buildInternal` cannot be called more than once would suffice.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1080955176


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -808,32 +809,12 @@ 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);
+            tidy.runOnClose = () -> {

Review Comment:
   added test cases in `RefCountedTest`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081324331


##########
src/java/org/apache/cassandra/io/sstable/IndexSummary.java:
##########
@@ -352,6 +353,37 @@ public void remove()
         };
     }
 
+    public long getScanPositionFromBinarySearch(PartitionPosition key)

Review Comment:
   looks good



##########
test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java:
##########
@@ -388,10 +387,10 @@ private void testPosition(IndexSummary original, IndexSummary downsampled, List<
     {
         for (DecoratedKey key : keys)
         {
-            long orig = SSTableReader.getIndexScanPositionFromBinarySearchResult(original.binarySearch(key), original);
+            long orig = original.getScanPositionFromBinarySearchResult(original.binarySearch(key));

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081305221


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -808,32 +809,12 @@ 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);
+            tidy.runOnClose = () -> {

Review Comment:
   The comment looks good, and thank you for adding the test.
   
   To make sure we are checking the `runOnClose` lambda (which is now only constructed on the second call), let's call it twice in the test.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1083863675


##########
src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriterBuilder.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.big;
+
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.sstable.DataComponent;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SortedTableWriterBuilder;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.Throwables;
+
+public class BigTableWriterBuilder extends SortedTableWriterBuilder<RowIndexEntry, BigFormatPartitionWriter, BigTableWriter, BigTableWriterBuilder>
+{
+    private RowIndexEntry.IndexSerializer rowIndexEntrySerializer;
+    private BigTableWriter.IndexWriter indexWriter;
+    private SequentialWriter dataWriter;
+    private BigFormatPartitionWriter partitionWriter;
+
+    public BigTableWriterBuilder(Descriptor descriptor)
+    {
+        super(descriptor);
+    }
+
+    @Override
+    public SequentialWriter getDataWriter()
+    {
+        return dataWriter;
+    }
+
+    @Override
+    public BigFormatPartitionWriter getPartitionWriter()
+    {
+        return partitionWriter;
+    }
+
+    public RowIndexEntry.IndexSerializer getRowIndexEntrySerializer()
+    {
+        return rowIndexEntrySerializer;
+    }
+
+    public BigTableWriter.IndexWriter getIndexWriter()
+    {
+        return indexWriter;
+    }
+
+    @Override
+    protected BigTableWriter buildInternal(LifecycleNewTracker lifecycleNewTracker)
+    {
+        try
+        {
+            rowIndexEntrySerializer = new RowIndexEntry.Serializer(descriptor.version, getSerializationHeader());
+            dataWriter = DataComponent.buildWriter(descriptor,

Review Comment:
   We cannot do a private constructor with this approach because we have a hierarchy - some components are consumed by the constructor of `SortedTableWriter` and the others by `BigTableWriter`. Obviously such a constructor in `SortedTableWriter` would have to be available for its descendants. 
   
   I understand your point, though. I know it is not a perfect approach but I also don't like having different approaches for writer and reader. I think I can try to meet your worries by doing it a bit differently - while not retiring passing stuff in a builder, I can refactor the initialization a bit: f932e370396a287c4293c6698dd9590d5258b72b
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081292598


##########
src/java/org/apache/cassandra/io/sstable/AbstractSSTableIterator.java:
##########
@@ -382,31 +406,174 @@ public Unfiltered next()
         protected abstract boolean hasNextInternal() throws IOException;
         protected abstract Unfiltered nextInternal() throws IOException;
 
+        @Override
         public void close() throws IOException
         {
             if (shouldCloseFile && file != null)
                 file.close();
         }
+
+        @Override
+        public String toString()
+        {
+            return file != null ? file.toString() : "null";
+        }    }

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081298378


##########
src/java/org/apache/cassandra/schema/Schema.java:
##########
@@ -224,6 +225,15 @@ public ColumnFamilyStore getColumnFamilyStoreInstance(TableId id)
                : null;
     }
 
+    public ColumnFamilyStore getColumnFamilyStoreInstance(TableMetadata metadata)

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1082767066


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -1983,39 +1380,36 @@ public void addTo(Ref.IdentityCollection identities)
     {
         identities.add(this);
         identities.add(tidy.globalRef);
-        dfile.addTo(identities);
-        ifile.addTo(identities);
-        bf.addTo(identities);
-        indexSummary.addTo(identities);
-
+        tidy.closeables.forEach(c -> {
+            if (c instanceof SharedCloseable)
+                ((SharedCloseable) c).addTo(identities);
+        });
     }
 
     public boolean maybePresent(DecoratedKey key)

Review Comment:
   You are right, it is not needed to check. Though, I don't like the fact that this method does implicit assumption for certain usages.
   
   Btw. thanks you pointing me to check the actual usages. It turned out that there were some redundant calls.
   
   
   8787244fbac13b8b9dee907d8cda0964bf0d047e
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1082767066


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -1983,39 +1380,36 @@ public void addTo(Ref.IdentityCollection identities)
     {
         identities.add(this);
         identities.add(tidy.globalRef);
-        dfile.addTo(identities);
-        ifile.addTo(identities);
-        bf.addTo(identities);
-        indexSummary.addTo(identities);
-
+        tidy.closeables.forEach(c -> {
+            if (c instanceof SharedCloseable)
+                ((SharedCloseable) c).addTo(identities);
+        });
     }
 
     public boolean maybePresent(DecoratedKey key)

Review Comment:
   You are right, it is not needed to check. Though, I don't like the fact that this method does implicit assumption for certain usages.
   
   8787244fbac13b8b9dee907d8cda0964bf0d047e
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081456635


##########
src/java/org/apache/cassandra/io/sstable/format/big/IndexSummaryComponent.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.big;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+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.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileInputStreamPlus;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class IndexSummaryComponent
+{
+    private static final Logger logger = LoggerFactory.getLogger(IndexSummaryComponent.class);
+
+    public final IndexSummary indexSummary;
+    public final DecoratedKey first;
+    public final DecoratedKey last;
+
+    public IndexSummaryComponent(IndexSummary indexSummary, DecoratedKey first, DecoratedKey last)
+    {
+        this.indexSummary = indexSummary;
+        this.first = first;
+        this.last = last;
+    }
+
+    /**
+     * Load index summary, first key and last key from Summary.db file if it exists.
+     * <p>
+     * if loaded index summary has different index interval from current value stored in schema,
+     * then Summary.db file will be deleted and need to be rebuilt.
+     */
+    public static IndexSummaryComponent load(Descriptor descriptor, TableMetadata metadata) throws IOException
+    {
+        File summaryFile = descriptor.fileFor(Component.SUMMARY);
+        if (!summaryFile.exists())
+        {
+            if (logger.isDebugEnabled())
+                logger.debug("SSTable Summary File {} does not exist", summaryFile.absolutePath());
+            return null;
+        }
+
+        IndexSummary summary = null;
+        try (FileInputStreamPlus iStream = summaryFile.newInputStream())
+        {
+            summary = IndexSummary.serializer.deserialize(iStream,
+                                                          metadata.partitioner,
+                                                          metadata.params.minIndexInterval,
+                                                          metadata.params.maxIndexInterval);
+            DecoratedKey first = metadata.partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
+            DecoratedKey last = metadata.partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
+
+            return new IndexSummaryComponent(summary, first, last);
+        }
+        catch (IOException ex)
+        {
+            if (summary != null)
+                summary.close();
+
+            throw new IOException(String.format("Cannot deserialize SSTable %s component: %s", Component.SUMMARY.name, summaryFile), ex);
+        }
+    }
+
+    public static IndexSummaryComponent loadOrDeleteCorrupted(Descriptor descriptor, TableMetadata metadata) throws IOException
+    {
+        try
+        {
+            return load(descriptor, metadata);
+        }
+        catch (IOException ex)
+        {
+            descriptor.fileFor(Component.SUMMARY).deleteIfExists();
+            throw ex;
+        }
+    }
+
+    /**
+     * Save index summary to Summary.db file.
+     */
+    public void save(Descriptor descriptor) throws IOException
+    {
+        File summaryFile = descriptor.fileFor(Component.SUMMARY);
+        if (summaryFile.exists())
+            summaryFile.delete();
+
+        try (DataOutputStreamPlus oStream = summaryFile.newOutputStream(File.WriteMode.OVERWRITE))
+        {
+            IndexSummary.serializer.serialize(indexSummary, oStream);
+            ByteBufferUtil.writeWithLength(first.getKey(), oStream);
+            ByteBufferUtil.writeWithLength(last.getKey(), oStream);
+        }
+        catch (IOException ex)
+        {
+            throw new IOException("Failed to save index summary for SSTable: " + descriptor.baseFilename(), ex);
+        }
+    }
+
+    public void saveOrDeleteCorrupted(Descriptor descriptor) throws IOException

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1093293126


##########
src/java/org/apache/cassandra/io/sstable/Component.java:
##########
@@ -41,55 +39,76 @@
 {
     public static final char separator = '-';
 
-    final static ImmutableSet<Type> TYPES = Type.all;
+    @SuppressWarnings("rawtypes")
+    public static Set<Component> getSingletonsFor(Class<? extends SSTableFormat> format)
+    {
+        return singletons.stream().filter(c -> c.type.formatClass.isAssignableFrom(format)).collect(Collectors.toSet());
+    }
 
     /**
      * WARNING: Be careful while changing the names or string representation of the enum
      * members. Streaming code depends on the names during streaming (Ref: CASSANDRA-14556).
      */
-    public static class Type
+    public final static class Type
     {
-        private final static List<Type> singletonsCollector = new ArrayList<>(11);
+        private final static CopyOnWriteArrayList<Type> typesCollector = new CopyOnWriteArrayList<>();
         // the base data for an sstable: the remaining components can be regenerated
         // based on the data component
-        public static final Type DATA = new Type("DATA", "Data.db", name -> Component.DATA);
+        public static final Type DATA = new Type("DATA", "Data.db", name -> Component.DATA, true);
         // file to hold information about uncompressed data length, chunk offsets etc.
-        public static final Type COMPRESSION_INFO = new Type("COMPRESSION_INFO", "CompressionInfo.db", name -> Component.COMPRESSION_INFO);
+        public static final Type COMPRESSION_INFO = new Type("COMPRESSION_INFO", "CompressionInfo.db", name -> Component.COMPRESSION_INFO, true);
         // statistical metadata about the content of the sstable
-        public static final Type STATS = new Type("STATS", "Statistics.db", name -> Component.STATS);
+        public static final Type STATS = new Type("STATS", "Statistics.db", name -> Component.STATS, true);
         // serialized bloom filter for the row keys in the sstable
-        public static final Type FILTER = new Type("FILTER", "Filter.db", name -> Component.FILTER);
+        public static final Type FILTER = new Type("FILTER", "Filter.db", name -> Component.FILTER, true);
         // holds CRC32 checksum of the data file
-        public static final Type DIGEST = new Type("DIGEST","Digest.crc32", name -> Component.DIGEST);
-        // holds the CRC32 for chunks in an a uncompressed file.
-        public static final Type CRC = new Type("CRC","CRC.db", name -> Component.CRC);
+        public static final Type DIGEST = new Type("DIGEST","Digest.crc32", name -> Component.DIGEST, true);
+        // holds the CRC32 for chunks in an uncompressed file.
+        public static final Type CRC = new Type("CRC","CRC.db", name -> Component.CRC, true);
         // table of contents, stores the list of all components for the sstable
-        public static final Type TOC = new Type("TOC","TOC.txt", name -> Component.TOC);
+        public static final Type TOC = new Type("TOC","TOC.txt", name -> Component.TOC, true);
         // built-in secondary index (may be multiple per sstable)
-        public static final Type SECONDARY_INDEX = new Type("SECONDARY_INDEX","SI_.*.db", name -> new Component(Type.SECONDARY_INDEX, name));
+        public static final Type SECONDARY_INDEX = new Type("SECONDARY_INDEX","SI_.*.db", name -> new Component(Type.SECONDARY_INDEX, name), false);
         // custom component, used by e.g. custom compaction strategy
-        public static final Type CUSTOM = new Type("CUSTOM",null, name -> new Component(Type.CUSTOM, name));
+        public static final Type CUSTOM = new Type("CUSTOM",null, name -> new Component(Type.CUSTOM, name), false);
 
-        public static final ImmutableSet<Type> all = ImmutableSet.copyOf(singletonsCollector);
+        public static final List<Type> all = Collections.unmodifiableList(typesCollector);
 
         public final int id;
         public final String name;
         public final String repr;
-        private Function<String, Component> componentFactory;
+        private final Function<String, Component> componentFactory;
+        public final boolean isSingleton;
+
+        @SuppressWarnings("rawtypes")
+        public final Class<? extends SSTableFormat> formatClass;
+
+        private Type(String name, String repr, Function<String, Component> componentFactory, boolean isSingleton)
+        {
+            this(name, repr, componentFactory, isSingleton, SSTableFormat.class);
+        }
 
-        protected Type(String name, String repr, Function<String, Component> componentFactory)
+        @SuppressWarnings("rawtypes")
+        public Type(String name, String repr, Function<String, Component> componentFactory, boolean isSingleton, Class<? extends SSTableFormat> formatClass)
         {
             this.name = name;
             this.repr = repr;
-            this.id = singletonsCollector.size();
+            this.id = typesCollector.size();
             this.componentFactory = componentFactory;
-            singletonsCollector.add(this);
+            this.isSingleton = isSingleton;
+            this.formatClass = formatClass;
+            synchronized (typesCollector)
+            {
+                if (typesCollector.stream().anyMatch(t -> Objects.equal(t.name, name) && t.formatClass == formatClass || t.formatClass.isAssignableFrom(formatClass) || formatClass.isAssignableFrom(t.formatClass)))

Review Comment:
   I mean that a subclass of `Component.Type` will always fire `t.formatClass.isAssignableFrom(formatClass)` (e.g. for `t = TOC`) and thus the whole condition without checking `Objects.equal(t.name, name)`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1072132298


##########
src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java:
##########
@@ -82,46 +82,46 @@ public void build()
                 PerSSTableIndexWriter indexWriter = SASIIndex.newWriter(keyValidator, sstable.descriptor, indexes, OperationType.COMPACTION);
                 targetDirectory = indexWriter.getDescriptor().directory.path();
 
-                long previousKeyPosition = 0;
-                try (KeyIterator keys = new KeyIterator(sstable.descriptor, cfs.metadata()))
+                try (KeyReader keys = sstable.keyReader())
                 {
-                    while (keys.hasNext())
+                    while (!keys.isExhausted())
                     {
                         if (isStopRequested())
                             throw new CompactionInterruptedException(getCompactionInfo());
 
-                        final DecoratedKey key = keys.next();
-                        final long keyPosition = keys.getKeyPosition();
+                        final DecoratedKey key = sstable.decorateKey(keys.key());
+                        final long keyPosition = keys.keyPositionForSecondaryIndex();
 
-                        indexWriter.startPartition(key, keyPosition);
+                        indexWriter.startPartition(key, keys.dataPosition(), keyPosition);
 
-                        try
-                        {
-                            long position = sstable.getPosition(key, SSTableReader.Operator.EQ);
-                            dataFile.seek(position);
-                            ByteBufferUtil.readWithShortLength(dataFile); // key
+                        long position = sstable.getPosition(key, SSTableReader.Operator.EQ);

Review Comment:
   🤦🏻‍♂️ 
   indeed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1072207523


##########
src/java/org/apache/cassandra/io/sstable/Descriptor.java:
##########
@@ -423,6 +425,19 @@ public boolean isCompatible()
         return version.isCompatible();
     }
 
+    public Set<Component> discoverComponents()
+    {
+        Set<Component.Type> knownTypes = Sets.difference(Component.TYPES, Collections.singleton(Component.Type.CUSTOM));
+        Set<Component> components = Sets.newHashSetWithExpectedSize(knownTypes.size());
+        for (Component.Type componentType : knownTypes)
+        {
+            Component component = new Component(componentType);

Review Comment:
   I've just copied this method, but I suppose it is because of secondary index component, which isn't a singleton. Though, I don't think this method would ever find any secondary index component with such implementation.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1071982503


##########
src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java:
##########
@@ -361,22 +357,27 @@ public SSTableReader openEarly()
             long indexFileLength = new File(descriptor.filenameFor(Component.PRIMARY_INDEX)).length();
             int indexBufferSize = optimizationStrategy.bufferSize(indexFileLength / indexSummary.size());
             ifile = iwriter.builder.bufferSize(indexBufferSize).complete(boundary.indexLength);
-            if (compression)
-                dbuilder.withCompressionMetadata(((CompressedSequentialWriter) dataFile).open(boundary.dataLength));
+            try (FileHandle.Builder dbuilder = new FileHandle.Builder(descriptor.filenameFor(Component.DATA)))

Review Comment:
   Doesn't this disable the only application of the `regions` reuse in the file handle builder?



##########
src/java/org/apache/cassandra/utils/Throwables.java:
##########
@@ -189,13 +189,19 @@ public static Throwable perform(Throwable accumulate, String filePath, FileOpTyp
     }
 
     public static void closeAndAddSuppressed(@Nonnull Throwable t, AutoCloseable... closeables)
+    {
+        closeAndAddSuppressed(t, Arrays.asList(closeables));
+    }
+
+    public static void closeAndAddSuppressed(@Nonnull Throwable t, Iterable<AutoCloseable> closeables)
     {
         Preconditions.checkNotNull(t);
         for (AutoCloseable closeable : closeables)
         {
             try
             {
-                closeable.close();
+                if (closeable != null)

Review Comment:
   I guess the comment applies to the method below then, but if we don't change it there it would be unexpected to skip here &mdash; unless we make the naming very clear about that difference.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1072137813


##########
src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java:
##########
@@ -56,22 +55,23 @@ class SASIIndexBuilder extends SecondaryIndexBuilder
     private final SortedMap<SSTableReader, Map<ColumnMetadata, ColumnIndex>> sstables;
 
     private long bytesProcessed = 0;
-    private final long totalSizeInBytes;
+    private final long totalBytesToProcess;
 
     public SASIIndexBuilder(ColumnFamilyStore cfs, SortedMap<SSTableReader, Map<ColumnMetadata, ColumnIndex>> sstables)
     {
-        long totalIndexBytes = 0;
+        long totalBytesToProcess = 0;
         for (SSTableReader sstable : sstables.keySet())
-            totalIndexBytes += getPrimaryIndexLength(sstable);
+            totalBytesToProcess += sstable.uncompressedLength();

Review Comment:
   I don't know where, the existing usages of `CompactionInfo` look like they already assumed the data size rather than the index size.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1072258870


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java:
##########
@@ -395,12 +393,13 @@ void load(boolean recreateBloomFilter,
                   StatsMetadata statsMetadata,
                   Set<Component> components) throws IOException
         {
-            try (FileHandle.Builder ibuilder = new FileHandle.Builder(descriptor.filenameFor(Component.PRIMARY_INDEX))
-                                               .mmapped(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap)
-                                               .withChunkCache(ChunkCache.instance);
-                 FileHandle.Builder dbuilder = new FileHandle.Builder(descriptor.filenameFor(Component.DATA)).mmapped(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap)
-                                                                                                             .withChunkCache(ChunkCache.instance))
+            try
             {
+                FileHandle.Builder ibuilder = new FileHandle.Builder(descriptor.fileFor(Component.PRIMARY_INDEX))

Review Comment:
   hmm... as I'm looking at this now, I think it needs some fixes - channel proxy is not closed in case of a failure. I also missed `CompressionMetadata`... it is  not closeable. I think maybe could implement `SharedCloseable`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1072236125


##########
src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java:
##########
@@ -361,22 +357,27 @@ public SSTableReader openEarly()
             long indexFileLength = new File(descriptor.filenameFor(Component.PRIMARY_INDEX)).length();
             int indexBufferSize = optimizationStrategy.bufferSize(indexFileLength / indexSummary.size());
             ifile = iwriter.builder.bufferSize(indexBufferSize).complete(boundary.indexLength);
-            if (compression)
-                dbuilder.withCompressionMetadata(((CompressedSequentialWriter) dataFile).open(boundary.dataLength));
+            try (FileHandle.Builder dbuilder = new FileHandle.Builder(descriptor.filenameFor(Component.DATA)))

Review Comment:
   Indeed, it will be fixed in some later commit



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1068100487


##########
src/java/org/apache/cassandra/db/compaction/Scrubber.java:
##########
@@ -564,20 +565,27 @@ public ScrubResult(Scrubber scrubber)
     /**
      * During 2.x migration, under some circumstances rows might have gotten duplicated.
      * Merging iterator merges rows with same clustering.
-     *
+     * <p>
      * For more details, refer to CASSANDRA-12144.
      */
-    private static class RowMergingSSTableIterator extends WrappingUnfilteredRowIterator
+    private static class RowMergingSSTableIterator extends UnmodifiableIterator<Unfiltered> implements WrappingUnfilteredRowIterator

Review Comment:
   Could we not implement `remove` in the interface so that `extends UnmodifiableIterator` is not necessary here and elsewhere?



##########
src/java/org/apache/cassandra/service/CacheService.java:
##########
@@ -439,13 +440,14 @@ public void serialize(KeyCacheKey key, DataOutputPlus out, ColumnFamilyStore cfs
                 out.writeInt(Integer.MIN_VALUE); // backwards compatibility for "int based generation only"
                 ByteBufferUtil.writeWithShortLength(key.desc.id.asBytes(), out);
             }
-            out.writeBoolean(true);
-
-            SerializationHeader header = new SerializationHeader(false, cfs.metadata(), cfs.metadata().regularAndStaticColumns(), EncodingStats.NO_STATS);
-            new RowIndexEntry.Serializer(key.desc.version, header).serializeForCache(entry, out);
+            // format type id is stored so that in case there is no sstable for the key we can figure out which
+            // serializer (of which sstable format) was used and thus as can use the right implemnentation to skip
+            // the unmatched entry
+            out.writeByte(key.desc.formatType.ordinal());

Review Comment:
   Would it be simpler to store a (varint) length here?



##########
src/java/org/apache/cassandra/io/sstable/AbstractSSTableIterator.java:
##########
@@ -382,31 +406,174 @@ public Unfiltered next()
         protected abstract boolean hasNextInternal() throws IOException;
         protected abstract Unfiltered nextInternal() throws IOException;
 
+        @Override
         public void close() throws IOException
         {
             if (shouldCloseFile && file != null)
                 file.close();
         }
+
+        @Override
+        public String toString()
+        {
+            return file != null ? file.toString() : "null";
+        }    }

Review Comment:
   Nit: formatting needs fixing.



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java:
##########
@@ -36,6 +36,9 @@
     SSTableWriter.Factory getWriterFactory();
     SSTableReader.Factory getReaderFactory();
 
+    boolean isKeyCacheSupported();

Review Comment:
   I would call this `cachesKeys` (if the sstable impl takes care of doing the caching calls) or `cacheKeys`/`shouldCacheKeys` (if other code is doing it) -- the alternate implementation can support cached keys, but does not need to.



##########
src/java/org/apache/cassandra/schema/Schema.java:
##########
@@ -224,6 +225,15 @@ public ColumnFamilyStore getColumnFamilyStoreInstance(TableId id)
                : null;
     }
 
+    public ColumnFamilyStore getColumnFamilyStoreInstance(TableMetadata metadata)
+    {
+        ColumnFamilyStore cfs = getColumnFamilyStoreInstance(metadata.id);
+        if (cfs == null || !metadata.isIndex())

Review Comment:
   Shouldn't this be `!=`?



##########
src/java/org/apache/cassandra/utils/Throwables.java:
##########
@@ -189,13 +189,19 @@ public static Throwable perform(Throwable accumulate, String filePath, FileOpTyp
     }
 
     public static void closeAndAddSuppressed(@Nonnull Throwable t, AutoCloseable... closeables)
+    {
+        closeAndAddSuppressed(t, Arrays.asList(closeables));
+    }
+
+    public static void closeAndAddSuppressed(@Nonnull Throwable t, Iterable<AutoCloseable> closeables)
     {
         Preconditions.checkNotNull(t);
         for (AutoCloseable closeable : closeables)
         {
             try
             {
-                closeable.close();
+                if (closeable != null)

Review Comment:
   I'm not sure we want to relax this. Usually `FileUtils.close` is used to close things that may include nulls.



##########
src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java:
##########
@@ -125,6 +130,24 @@ public AbstractRowIndexEntry.KeyCacheValueSerializer<BigTableReader, RowIndexEnt
         return KeyCacheValueSerializer.instance;
     }
 
+    @Override
+    public BigTableReader cast(SSTableReader sstr)
+    {
+        return sstr == null ? null : (BigTableReader) sstr;

Review Comment:
   There is no need for the null check here. Nulls can be cast to any type.



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -808,32 +809,12 @@ 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);
+            tidy.runOnClose = () -> {

Review Comment:
   AFAIR `AndThen` here is necessary because a lambda retains a reference to the reader, which we definitely don't want (see comment on top of the method).
   
   If you revert this change, please add a "do not replace with a lambda..." comment in front of `AndThen` itself.



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java:
##########
@@ -36,6 +39,8 @@
     SSTableWriter.Factory getWriterFactory();
     SSTableReader.Factory getReaderFactory();
 
+    Set<Component> supportedComponents();

Review Comment:
   As below, "supported" is not the right term here. Maybe "included"? (I believe we also have "required" for the mandatory ones?)



##########
src/java/org/apache/cassandra/schema/Schema.java:
##########
@@ -224,6 +225,15 @@ public ColumnFamilyStore getColumnFamilyStoreInstance(TableId id)
                : null;
     }
 
+    public ColumnFamilyStore getColumnFamilyStoreInstance(TableMetadata metadata)

Review Comment:
   The behaviour of this is different from id lookup. Could we give it a different name?



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -1973,17 +1970,36 @@ public void run()
                     if (logger.isTraceEnabled())
                         logger.trace("Async instance tidier for {}, after barrier", descriptor);
 
-                    if (bf != null)
-                        bf.close();
-                    if (summary != null)
-                        summary.close();
-                    if (runOnClose != null)
+                    Throwable exceptions = null;
+                    if (runOnClose != null) try
+                    {
                         runOnClose.run();
-                    if (dfile != null)
-                        dfile.close();
-                    if (ifile != null)
-                        ifile.close();
-                    globalRef.release();
+                    }
+                    catch (RuntimeException | Error ex)
+                    {
+                        logger.error("Failed to run on-close listeners for sstable " + descriptor.baseFilename(), ex);
+                        exceptions = ex;
+                    }
+
+                    Throwable closeExceptions = Throwables.close(null, closeables);

Review Comment:
   Isn't `FileUtils.close` a better match for what is done here?



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -907,43 +907,18 @@ public SSTableReader cloneWithNewStart(DecoratedKey newStart, final Runnable run
             // TODO: merge with caller's firstKeyBeyond() work,to save time
             if (newStart.compareTo(first) > 0)
             {
-                final long dataStart = getPosition(newStart, Operator.EQ);
-                final long indexStart = getIndexScanPosition(newStart);
-                this.tidy.runOnClose = new DropPageCache(dfile, dataStart, ifile, indexStart, runOnClose);
+                Map<FileHandle, Long> handleAndPositions = new LinkedHashMap<>(2);
+                if (dfile != null)
+                    handleAndPositions.put(dfile, getPosition(newStart, Operator.EQ));
+                if (ifile != null)
+                    handleAndPositions.put(ifile, getIndexScanPosition(newStart));
+                runOnClose(() -> handleAndPositions.forEach(FileHandle::dropPageCache));

Review Comment:
   The same comment as in `runOnClose` applies -- this lambda retains a reference to the reader.



##########
src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java:
##########
@@ -374,4 +380,20 @@ public ClusteringPrefix<?> getLowerBoundPrefixFromCache(DecoratedKey partitionKe
         }
 
     }
-}
+
+    @Override
+    public IScrubber getScrubber(LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(metadata());

Review Comment:
   Could we do this without referencing the higher-level CFS and Schema concepts in the sstable reader implementation?
   
   Perhaps implement it in the format and pass the CFS as an argument?



##########
src/java/org/apache/cassandra/db/compaction/Scrubber.java:
##########
@@ -742,6 +750,12 @@ public FixNegativeLocalDeletionTimeIterator(UnfilteredRowIterator iterator, Outp
             this.negativeLocalExpirationTimeMetrics = negativeLocalDeletionInfoMetrics;
         }
 
+        @Override
+        public UnfilteredRowIterator wrapped()
+        {
+            return iterator;
+        }
+

Review Comment:
   Should we not take advantage of the methods implemented by the wrapping interface, i.e. remove the implementations below?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1082602700


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java:
##########
@@ -395,12 +393,13 @@ void load(boolean recreateBloomFilter,
                   StatsMetadata statsMetadata,
                   Set<Component> components) throws IOException
         {
-            try (FileHandle.Builder ibuilder = new FileHandle.Builder(descriptor.filenameFor(Component.PRIMARY_INDEX))
-                                               .mmapped(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap)
-                                               .withChunkCache(ChunkCache.instance);
-                 FileHandle.Builder dbuilder = new FileHandle.Builder(descriptor.filenameFor(Component.DATA)).mmapped(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap)
-                                                                                                             .withChunkCache(ChunkCache.instance))
+            try
             {
+                FileHandle.Builder ibuilder = new FileHandle.Builder(descriptor.fileFor(Component.PRIMARY_INDEX))

Review Comment:
   I can do that, though I'm pretty convenient about safety of the current approach; On the other hand, I must agree with you that it will be more consistent and elegant to move the management of that outside.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1082688742


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java:
##########
@@ -395,12 +393,13 @@ void load(boolean recreateBloomFilter,
                   StatsMetadata statsMetadata,
                   Set<Component> components) throws IOException
         {
-            try (FileHandle.Builder ibuilder = new FileHandle.Builder(descriptor.filenameFor(Component.PRIMARY_INDEX))
-                                               .mmapped(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap)
-                                               .withChunkCache(ChunkCache.instance);
-                 FileHandle.Builder dbuilder = new FileHandle.Builder(descriptor.filenameFor(Component.DATA)).mmapped(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap)
-                                                                                                             .withChunkCache(ChunkCache.instance))
+            try
             {
+                FileHandle.Builder ibuilder = new FileHandle.Builder(descriptor.fileFor(Component.PRIMARY_INDEX))

Review Comment:
   What do you think about instead of managing it in a builder, creating a method in `FileHandle`, something like `duplicateWithExtendedLength(CompressionMetadata, long)`, which retrieves regions from the current rebufferer if applicable (or we could keep mmapped  regions as a field in `FileHandle`)?
   
   This way, it should be super clear - we create a copy of regions when we really copy a file handle



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081345032


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java:
##########
@@ -103,4 +105,11 @@ public static Type validate(String name)
     {
         List<GaugeProvider<?, ?>> getGaugeProviders();
     }
+
+    interface SSTableReaderFactory<R extends SSTableReader, B extends SSTableReaderBuilder<R, B>>
+    {
+        SSTableReaderBuilder<R, B> builder(Descriptor descriptor);
+
+        SSTableReaderLoadingBuilder<R, B> builder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components);

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081327417


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -1796,9 +1654,33 @@ public static void resetTidying()
         GlobalTidy.lookup.clear();
     }
 
-    public static abstract class Factory
+    public interface Factory<T extends SSTableReader>

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081462402


##########
src/java/org/apache/cassandra/io/sstable/SSTable.java:
##########
@@ -155,9 +155,9 @@ public DecoratedKey decorateKey(ByteBuffer key)
      */
     public static DecoratedKey getMinimalKey(DecoratedKey key)

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1095556618


##########
src/java/org/apache/cassandra/io/sstable/BloomFilterMetrics.java:
##########
@@ -89,7 +89,7 @@ protected R map(SSTableReader r)
     /**
      * False positive ratio of bloom filter from last read
      */
-    public final GaugeProvider<Double> recentBloomFilterFalseRatio = newGaugeProvider("BloomFilterFalseRatio", readers -> {
+    public final GaugeProvider<Double> recentBloomFilterFalseRatio = newGaugeProvider("RecentBloomFilterFalseRatio", readers -> {
         long falsePositiveCount = 0L;
         long truePositiveCount = 0L;
         long trueNegativeCount = 0L;

Review Comment:
   The calls below (`getFalsePositiveCount` etc) should be changed to their recent versions.



##########
src/java/org/apache/cassandra/io/sstable/keycache/KeyCacheSupport.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.keycache;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import org.apache.cassandra.cache.KeyCacheKey;
+import org.apache.cassandra.db.ClusteringPrefix;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.filter.ClusteringIndexFilter;
+import org.apache.cassandra.io.sstable.format.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.DataInputPlus;
+
+public interface KeyCacheSupport<T extends SSTableReader & KeyCacheSupport<T>>
+{
+    @Nullable
+    KeyCache getKeyCache();
+
+    /**
+     * Should quickly get a lower bound prefix from cache only if everything is already availabe in memory and does not
+     * need to be loaded from disk.
+     */
+    @Nullable
+    ClusteringPrefix<?> getLowerBoundPrefixFromCache(DecoratedKey partitionKey, ClusteringIndexFilter filter);
+
+    /**
+     * @return Number of key cache hit
+     */
+    default long getKeyCacheHit()
+    {
+        KeyCache keyCache = getKeyCache();
+        return keyCache != null ? keyCache.getHits() : 0;
+    }
+
+    /**
+     * @return Number of key cache request
+     */
+    default long getKeyCacheRequest()

Review Comment:
   Nit: Request -> Requests



##########
src/java/org/apache/cassandra/io/sstable/format/SortedTableReader.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.io.sstable.BloomFilterTracker;
+import org.apache.cassandra.utils.IFilter;
+
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+public abstract class SortedTableReader extends SSTableReader

Review Comment:
   Nit: Because the only thing this provides is bloom filter support, doesn't it make better sense to call it `SSTableReaderWithFilter`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1109671033


##########
test/unit/org/apache/cassandra/Util.java:
##########
@@ -1236,4 +1241,18 @@ public static void flush(TableViews view)
     {
         view.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS);
     }
+
+    public static DataInputPlus.DataInputStreamPlus createDataInputStreamPlus(InputStream in)
+    {
+        DataInputStreamPlusImpl impl = new DataInputStreamPlusImpl(in);
+        return Mockito.mock(DataInputPlus.DataInputStreamPlus.class, new ForwardsInvocations(impl));

Review Comment:
   Just very few unit tests uses that in a very inexcessive way



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081430724


##########
src/java/org/apache/cassandra/io/sstable/format/SortedTableWriter.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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.IOException;
+import java.nio.BufferOverflowException;
+import java.util.Set;
+import java.util.function.Consumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionPurger;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.guardrails.Guardrails;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.db.rows.ComplexColumnData;
+import org.apache.cassandra.db.rows.PartitionSerializationException;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundaryMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.compress.CompressedSequentialWriter;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.SchemaConstants;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public abstract class SortedTableWriter<P extends SortedTablePartitionWriter, RIE extends AbstractRowIndexEntry> extends SSTableWriter<RIE>
+{
+    private final static Logger logger = LoggerFactory.getLogger(SortedTableWriter.class);
+
+    protected final SequentialWriter dataWriter;
+    protected final P partitionWriter;
+    private final FileHandle.Builder dataFileBuilder = new FileHandle.Builder(descriptor.fileFor(Component.DATA));
+    private DecoratedKey lastWrittenKey;
+    private DataPosition dataMark;
+    private long lastEarlyOpenLength;
+
+    public SortedTableWriter(SortedTableWriterBuilder<RIE, P, ?, ?> builder, LifecycleNewTracker lifecycleNewTracker)
+    {
+        super(builder, lifecycleNewTracker);
+        checkNotNull(builder.getDataWriter());
+        checkNotNull(builder.getPartitionWriter());
+
+        this.dataWriter = builder.getDataWriter();
+        this.partitionWriter = builder.getPartitionWriter();
+    }
+
+    /**
+     * Appends partition data to this writer.
+     *
+     * @param partition the partition to write
+     * @return the created index entry if something was written, that is if {@code iterator}
+     * wasn't empty, {@code null} otherwise.
+     * @throws FSWriteError if write to the dataFile fails
+     */
+    @Override
+    public final RIE append(UnfilteredRowIterator partition)
+    {
+        if (partition.isEmpty())
+            return null;
+
+        try
+        {
+            if (!verifyPartition(partition.partitionKey()))
+                return null;
+
+            startPartition(partition.partitionKey(), partition.partitionLevelDeletion());
+
+            RIE indexEntry;
+            if (header.hasStatic())
+                addStaticRow(partition.partitionKey(), partition.staticRow());
+
+            while (partition.hasNext())
+                addUnfiltered(partition.partitionKey(), partition.next());
+
+            indexEntry = endPartition(partition.partitionKey(), partition.partitionLevelDeletion());
+
+            return indexEntry;
+        }
+        catch (BufferOverflowException boe)
+        {
+            throw new PartitionSerializationException(partition, boe);
+        }
+        catch (IOException e)
+        {
+            throw new FSWriteError(e, getFilename());
+        }
+    }
+
+    private boolean verifyPartition(DecoratedKey key)
+    {
+        assert key != null : "Keys must not be null"; // empty keys ARE allowed b/c of indexed column values
+
+        if (key.getKey().remaining() > FBUtilities.MAX_UNSIGNED_SHORT)
+        {
+            logger.error("Key size {} exceeds maximum of {}, skipping row", key.getKey().remaining(), FBUtilities.MAX_UNSIGNED_SHORT);
+            return false;
+        }
+
+        if (lastWrittenKey != null && lastWrittenKey.compareTo(key) >= 0)
+            throw new RuntimeException(String.format("Last written key %s >= current key %s, writing into %s", lastWrittenKey, key, getFilename()));
+
+        return true;
+    }
+
+    private void startPartition(DecoratedKey key, DeletionTime partitionLevelDeletion) throws IOException
+    {
+        partitionWriter.start(key, partitionLevelDeletion);
+        metadataCollector.update(partitionLevelDeletion);
+
+        onStartPartition(key);
+    }
+
+    private void addStaticRow(DecoratedKey key, Row row) throws IOException
+    {
+        guardCollectionSize(key, row);
+
+        partitionWriter.addStaticRow(row);
+        if (!row.isEmpty())
+            Rows.collectStats(row, metadataCollector);
+
+        onStaticRow(row);
+    }
+
+    private void addUnfiltered(DecoratedKey key, Unfiltered unfiltered) throws IOException
+    {
+        if (unfiltered.isRow())
+        {
+            addRow(key, (Row) unfiltered);
+        }
+        else
+        {
+            assert unfiltered.isRangeTombstoneMarker();
+            addRangeTomstoneMarker((RangeTombstoneMarker) unfiltered);
+        }
+    }
+
+    private void addRow(DecoratedKey key, Row row) throws IOException
+    {
+        guardCollectionSize(key, row);
+
+        partitionWriter.addUnfiltered(row);
+        metadataCollector.updateClusteringValues(row.clustering());
+        Rows.collectStats(row, metadataCollector);
+
+        onRow(row);
+    }
+
+    private void addRangeTomstoneMarker(RangeTombstoneMarker marker) throws IOException
+    {
+        partitionWriter.addUnfiltered(marker);
+
+        metadataCollector.updateClusteringValues(marker.clustering());
+        if (marker.isBoundary())
+        {
+            RangeTombstoneBoundaryMarker bm = (RangeTombstoneBoundaryMarker) marker;
+            metadataCollector.update(bm.endDeletionTime());
+            metadataCollector.update(bm.startDeletionTime());
+        }
+        else
+        {
+            metadataCollector.update(((RangeTombstoneBoundMarker) marker).deletionTime());
+        }
+
+        onRangeTombstoneMarker(marker);
+    }
+
+    private RIE endPartition(DecoratedKey key, DeletionTime partitionLevelDeletion) throws IOException
+    {
+        long finishResult = partitionWriter.finish();
+
+        long endPosition = dataWriter.position();
+        long rowSize = endPosition - partitionWriter.getInitialPosition();
+        maybeLogLargePartitionWarning(key, rowSize);
+        maybeLogManyTombstonesWarning(key, metadataCollector.totalTombstones);
+        metadataCollector.addPartitionSizeInBytes(rowSize);
+        metadataCollector.addKey(key.getKey());
+        metadataCollector.addCellPerPartitionCount();
+
+        lastWrittenKey = key;
+        last = lastWrittenKey;
+        if (first == null)
+            first = lastWrittenKey;
+
+        if (logger.isTraceEnabled())
+            logger.trace("wrote {} at {}", key, endPosition);
+
+        return createRowIndexEntry(key, partitionLevelDeletion, finishResult);
+    }
+
+    protected void onStartPartition(DecoratedKey key)
+    {
+        notifyObservers(o -> o.startPartition(key, partitionWriter.getInitialPosition(), partitionWriter.getInitialPosition()));
+    }
+
+    protected void onStaticRow(Row row)
+    {
+        notifyObservers(o -> o.staticRow(row));
+    }
+
+    protected void onRow(Row row)
+    {
+        notifyObservers(o -> o.nextUnfilteredCluster(row));
+    }
+
+    protected void onRangeTombstoneMarker(RangeTombstoneMarker marker)
+    {
+        notifyObservers(o -> o.nextUnfilteredCluster(marker));
+    }
+
+    protected abstract RIE createRowIndexEntry(DecoratedKey key, DeletionTime partitionLevelDeletion, long finishResult) throws IOException;
+
+    protected final void notifyObservers(Consumer<SSTableFlushObserver> action)
+    {
+        if (observers != null && !observers.isEmpty())
+            observers.forEach(action);
+    }
+
+    @Override
+    public void mark()
+    {
+        dataMark = dataWriter.mark();
+    }
+
+    @Override
+    public void resetAndTruncate()
+    {
+        dataWriter.resetAndTruncate(dataMark);
+        partitionWriter.reset();
+    }
+
+    @Override
+    public long getFilePointer()
+    {
+        return dataWriter.position();
+    }
+
+    @Override
+    public long getOnDiskFilePointer()
+    {
+        return dataWriter.getOnDiskFilePointer();
+    }
+
+    @Override
+    public long getEstimatedOnDiskBytesWritten()
+    {
+        return dataWriter.getEstimatedOnDiskBytesWritten();
+    }
+
+    protected FileHandle openDataFile(long lengthOverride, StatsMetadata statsMetadata)
+    {
+        int dataBufferSize = ioOptions.diskOptimizationStrategy.bufferSize(statsMetadata.estimatedPartitionSize.percentile(ioOptions.diskOptimizationEstimatePercentile));
+
+        FileHandle dataFile = dataFileBuilder.mmapped(ioOptions.defaultDiskAccessMode == Config.DiskAccessMode.mmap)

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1083869797


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -1983,39 +1380,36 @@ public void addTo(Ref.IdentityCollection identities)
     {
         identities.add(this);
         identities.add(tidy.globalRef);
-        dfile.addTo(identities);
-        ifile.addTo(identities);
-        bf.addTo(identities);
-        indexSummary.addTo(identities);
-
+        tidy.closeables.forEach(c -> {
+            if (c instanceof SharedCloseable)
+                ((SharedCloseable) c).addTo(identities);
+        });
     }
 
     public boolean maybePresent(DecoratedKey key)

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1082594246


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -1973,17 +1970,36 @@ public void run()
                     if (logger.isTraceEnabled())
                         logger.trace("Async instance tidier for {}, after barrier", descriptor);
 
-                    if (bf != null)
-                        bf.close();
-                    if (summary != null)
-                        summary.close();
-                    if (runOnClose != null)
+                    Throwable exceptions = null;
+                    if (runOnClose != null) try
+                    {
                         runOnClose.run();
-                    if (dfile != null)
-                        dfile.close();
-                    if (ifile != null)
-                        ifile.close();
-                    globalRef.release();
+                    }
+                    catch (RuntimeException | Error ex)
+                    {
+                        logger.error("Failed to run on-close listeners for sstable " + descriptor.baseFilename(), ex);
+                        exceptions = ex;
+                    }
+
+                    Throwable closeExceptions = Throwables.close(null, closeables);

Review Comment:
   9ffff01f36d271686bac16b932b57c7d918a355d



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1083786469


##########
src/java/org/apache/cassandra/io/sstable/format/FilterComponent.java:
##########
@@ -90,4 +97,63 @@ public static void saveOrDeleteCorrupted(Descriptor descriptor, IFilter filter)
             throw ex;
         }
     }
+
+    /**
+     * Optionally loads a Bloom filter. If the filter is not needed (FP chance is neglectable), it sets
+     * {@link AlwaysPresentFilter} as a filter in the builder. If the filter is expected to be recreated for various
+     * reasons, it leaves it {@code null} (unchanged). Otherwise, it attempts to load the filter, and if it succeeds,
+     * it is set in the builder. If a filter fails to load, it is left {@code null} (unchanged) meaning that it should
+     * be rebuilt.
+     */
+    public static IFilter maybeLoadBloomFilter(Descriptor descriptor, Set<Component> components, TableMetadata metadata, ValidationMetadata validationMetadata)
+    {
+        double currentFPChance = validationMetadata != null ? validationMetadata.bloomFilterFPChance : Double.NaN;
+        double desiredFPChance = metadata.params.bloomFilterFpChance;
+
+        IFilter filter = null;
+        if (!shouldUseBloomFilter(desiredFPChance))
+        {
+            if (logger.isTraceEnabled())
+                logger.trace("Bloom filter for {} will not be loaded because fpChance={} is neglectable", descriptor, desiredFPChance);
+
+            return FilterFactory.AlwaysPresent;
+        }
+        else if (!components.contains(Component.FILTER) || Double.isNaN(currentFPChance))
+        {
+            if (logger.isTraceEnabled())
+                logger.trace("Bloom filter for {} will not be loaded because filter component is missing or sstable lacks validation metadata", descriptor);
+
+            return null;
+        }
+        else if (!isFPChanceDiffNeglectable(desiredFPChance, currentFPChance) && rebuildFilterOnFPChanceChange)
+        {
+            if (logger.isTraceEnabled())
+                logger.trace("Bloom filter for {} will not be loaded because fpChance has changed from {} to {} and the filter should be recreated", descriptor, currentFPChance, desiredFPChance);
+
+            return null;
+        }
+
+        try
+        {
+            filter = load(descriptor);
+            if (filter == null || filter instanceof AlwaysPresentFilter)
+                logger.info("Bloom filter for {} is missing or invalid", descriptor);
+        }
+        catch (IOException ex)
+        {
+            logger.info("Bloom filter for " + descriptor + " could not be deserialized", ex);
+        }
+
+        return filter;
+    }
+
+    static boolean shouldUseBloomFilter(double fpChance)
+    {
+        return !(Math.abs(1 - fpChance) <= filterFPChanceTolerance);
+    }
+
+    static boolean isFPChanceDiffNeglectable(double fpChance1, double fpChance2)

Review Comment:
   c46630091965b87b2ac4b1f2d4d2e25bfa3f3683



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1083913466


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java:
##########
@@ -395,12 +393,13 @@ void load(boolean recreateBloomFilter,
                   StatsMetadata statsMetadata,
                   Set<Component> components) throws IOException
         {
-            try (FileHandle.Builder ibuilder = new FileHandle.Builder(descriptor.filenameFor(Component.PRIMARY_INDEX))
-                                               .mmapped(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap)
-                                               .withChunkCache(ChunkCache.instance);
-                 FileHandle.Builder dbuilder = new FileHandle.Builder(descriptor.filenameFor(Component.DATA)).mmapped(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap)
-                                                                                                             .withChunkCache(ChunkCache.instance))
+            try
             {
+                FileHandle.Builder ibuilder = new FileHandle.Builder(descriptor.fileFor(Component.PRIMARY_INDEX))

Review Comment:
   The duplication idea sounds good to me.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1094296474


##########
src/java/org/apache/cassandra/io/sstable/KeyCacheSupport.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.IOException;
+
+import org.apache.cassandra.db.ClusteringPrefix;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.filter.ClusteringIndexFilter;
+import org.apache.cassandra.io.sstable.format.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.DataInputPlus;
+
+public interface KeyCacheSupport<T extends SSTableReader & KeyCacheSupport<T>>

Review Comment:
   it is with the recent changes, if we want to keep them



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1071909821


##########
src/java/org/apache/cassandra/service/CacheService.java:
##########
@@ -439,13 +440,14 @@ public void serialize(KeyCacheKey key, DataOutputPlus out, ColumnFamilyStore cfs
                 out.writeInt(Integer.MIN_VALUE); // backwards compatibility for "int based generation only"
                 ByteBufferUtil.writeWithShortLength(key.desc.id.asBytes(), out);
             }
-            out.writeBoolean(true);
-
-            SerializationHeader header = new SerializationHeader(false, cfs.metadata(), cfs.metadata().regularAndStaticColumns(), EncodingStats.NO_STATS);
-            new RowIndexEntry.Serializer(key.desc.version, header).serializeForCache(entry, out);
+            // format type id is stored so that in case there is no sstable for the key we can figure out which
+            // serializer (of which sstable format) was used and thus as can use the right implemnentation to skip
+            // the unmatched entry
+            out.writeByte(key.desc.formatType.ordinal());

Review Comment:
   I know it would be more elegant and I thought about that, but on the other hand, we would have to have `serializedSize` at hand or buffer the whole entry. Serialized size is not available without going through all the columns (for indexed row entry) - I don't know if it is worth the effort. That's why I stored format ordinal



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1072153031


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -1796,9 +1654,33 @@ public static void resetTidying()
         GlobalTidy.lookup.clear();
     }
 
-    public static abstract class Factory
+    public interface Factory<T extends SSTableReader>

Review Comment:
   yes, at this point it does not make sense so I'll remove from this commit



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1069658209


##########
src/java/org/apache/cassandra/io/sstable/IndexSummary.java:
##########
@@ -352,6 +353,37 @@ public void remove()
         };
     }
 
+    public long getScanPositionFromBinarySearch(PartitionPosition key)

Review Comment:
   `FromBinarySearch` does not make much sense here. `getScanPosition(PartitionPosition)` would be better.



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -1033,6 +1012,14 @@ protected void closeInternalComponent(AutoCloseable closeable)
         }
     }
 
+    /**
+     * This method is expected to close the components which occupy memory but are not needed when we just want to
+     * stream the components (for example, when SSTable is opened with SSTableLoader). The method should call
+     * {@link #closeInternalComponent(AutoCloseable)} for each such component. Leaving the implementation empty is
+     * valid, but may impact memory usage.
+     */
+    public abstract void releaseComponents();

Review Comment:
   Nit: This should include some adjective for the type of components released, e.g. `releaseInMemoryComponents`.



##########
src/java/org/apache/cassandra/db/compaction/CompactionManager.java:
##########
@@ -1890,22 +1890,17 @@ public void run()
         return executor.submitIfRunning(runnable, "cache write");
     }
 
-    public List<SSTableReader> runIndexSummaryRedistribution(IndexSummaryRedistribution redistribution) throws IOException
+    public <T, E extends Throwable> T runWithActiveCompactions(Holder holder, ThrowingSupplier<T, E> callable) throws E

Review Comment:
   I don't understand this method's naming -- it is running the callable wrapped as an active compaction with the given info holder?
   
   Maybe `runAsActiveCompaction(CompactionInfo.Holder activeCompactionInfo, ThrowingSupplier callable)`? 
   Or `<T, E extends Throwable, R extends CompactionInfo.Holder & ThrowingSupplier<T, E>> T runAsActiveCompaction(R callable) throws E` and use one argument for both operation info and callable to make it explicit that the info is for the operation that the callable does?



##########
src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java:
##########
@@ -82,46 +82,46 @@ public void build()
                 PerSSTableIndexWriter indexWriter = SASIIndex.newWriter(keyValidator, sstable.descriptor, indexes, OperationType.COMPACTION);
                 targetDirectory = indexWriter.getDescriptor().directory.path();
 
-                long previousKeyPosition = 0;
-                try (KeyIterator keys = new KeyIterator(sstable.descriptor, cfs.metadata()))
+                try (KeyReader keys = sstable.keyReader())
                 {
-                    while (keys.hasNext())
+                    while (!keys.isExhausted())
                     {
                         if (isStopRequested())
                             throw new CompactionInterruptedException(getCompactionInfo());
 
-                        final DecoratedKey key = keys.next();
-                        final long keyPosition = keys.getKeyPosition();
+                        final DecoratedKey key = sstable.decorateKey(keys.key());
+                        final long keyPosition = keys.keyPositionForSecondaryIndex();
 
-                        indexWriter.startPartition(key, keyPosition);
+                        indexWriter.startPartition(key, keys.dataPosition(), keyPosition);
 
-                        try
-                        {
-                            long position = sstable.getPosition(key, SSTableReader.Operator.EQ);
-                            dataFile.seek(position);
-                            ByteBufferUtil.readWithShortLength(dataFile); // key
+                        long position = sstable.getPosition(key, SSTableReader.Operator.EQ);

Review Comment:
   Isn't this `keys.dataPosition()`?



##########
src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java:
##########
@@ -56,22 +55,23 @@ class SASIIndexBuilder extends SecondaryIndexBuilder
     private final SortedMap<SSTableReader, Map<ColumnMetadata, ColumnIndex>> sstables;
 
     private long bytesProcessed = 0;
-    private final long totalSizeInBytes;
+    private final long totalBytesToProcess;
 
     public SASIIndexBuilder(ColumnFamilyStore cfs, SortedMap<SSTableReader, Map<ColumnMetadata, ColumnIndex>> sstables)
     {
-        long totalIndexBytes = 0;
+        long totalBytesToProcess = 0;
         for (SSTableReader sstable : sstables.keySet())
-            totalIndexBytes += getPrimaryIndexLength(sstable);
+            totalBytesToProcess += sstable.uncompressedLength();

Review Comment:
   We probably need a documentation change for this (progress changed from index bytes to data bytes).



##########
src/java/org/apache/cassandra/io/sstable/Descriptor.java:
##########
@@ -171,6 +172,18 @@ public List<File> getTemporaryFiles()
         return ret;
     }
 
+    public Set<Component> getComponents(Set<Component> alwaysAdd, Set<Component> optional)

Review Comment:
   Nit: alwaysAdd -> mandatory?



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -1796,9 +1654,33 @@ public static void resetTidying()
         GlobalTidy.lookup.clear();
     }
 
-    public static abstract class Factory
+    public interface Factory<T extends SSTableReader>

Review Comment:
   Does this type parameter provide any benefit? The user shouldn't know the actual type and will generally keep the generic `SSTableReader` reference.



##########
src/java/org/apache/cassandra/io/sstable/format/FilterComponent.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileInputStreamPlus;
+import org.apache.cassandra.io.util.FileOutputStreamPlus;
+import org.apache.cassandra.utils.BloomFilterSerializer;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+
+public class FilterComponent
+{
+    private static final Logger logger = LoggerFactory.getLogger(FilterComponent.class);
+
+    private FilterComponent()
+    {
+    }
+
+    /**
+     * Load bloom filter from Filter.db file.
+     */
+    public static IFilter load(Descriptor descriptor) throws IOException
+    {
+        File filterFile = descriptor.fileFor(Component.FILTER);
+
+        if (!filterFile.exists())
+            return null;
+
+        if (filterFile.length() == 0)
+            return FilterFactory.AlwaysPresent;
+
+        try (FileInputStreamPlus stream = descriptor.fileFor(Component.FILTER).newInputStream())
+        {
+            return BloomFilterSerializer.forVersion(descriptor.version.hasOldBfFormat()).deserialize(stream);
+        }
+        catch (IOException ex)
+        {
+            throw new IOException("Failed to load Bloom filter for SSTable: " + descriptor.baseFilename(), ex);
+        }
+    }
+
+    public static void save(IFilter filter, Descriptor descriptor) throws IOException
+    {
+        File filterFile = descriptor.fileFor(Component.FILTER);
+        try (FileOutputStreamPlus stream = filterFile.newOutputStream(File.WriteMode.OVERWRITE))
+        {
+            filter.serialize(stream, descriptor.version.hasOldBfFormat());
+            stream.flush();
+            stream.sync(); // is it needed if we close the file right after that?

Review Comment:
   This call maps to `FileChannel.force`, which ensures that the data is flushed out to disk (i.e. persistent) before returning. Closing does not do that (it does do the flushing, but we want that to happen before the sync).



##########
src/java/org/apache/cassandra/io/sstable/format/FilterComponent.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileInputStreamPlus;
+import org.apache.cassandra.io.util.FileOutputStreamPlus;
+import org.apache.cassandra.utils.BloomFilterSerializer;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+
+public class FilterComponent
+{
+    private static final Logger logger = LoggerFactory.getLogger(FilterComponent.class);
+
+    private FilterComponent()
+    {
+    }
+
+    /**
+     * Load bloom filter from Filter.db file.
+     */
+    public static IFilter load(Descriptor descriptor) throws IOException
+    {
+        File filterFile = descriptor.fileFor(Component.FILTER);
+
+        if (!filterFile.exists())
+            return null;
+
+        if (filterFile.length() == 0)
+            return FilterFactory.AlwaysPresent;
+
+        try (FileInputStreamPlus stream = descriptor.fileFor(Component.FILTER).newInputStream())
+        {
+            return BloomFilterSerializer.forVersion(descriptor.version.hasOldBfFormat()).deserialize(stream);
+        }
+        catch (IOException ex)
+        {
+            throw new IOException("Failed to load Bloom filter for SSTable: " + descriptor.baseFilename(), ex);
+        }
+    }
+
+    public static void save(IFilter filter, Descriptor descriptor) throws IOException
+    {
+        File filterFile = descriptor.fileFor(Component.FILTER);
+        try (FileOutputStreamPlus stream = filterFile.newOutputStream(File.WriteMode.OVERWRITE))
+        {
+            filter.serialize(stream, descriptor.version.hasOldBfFormat());
+            stream.flush();
+            stream.sync(); // is it needed if we close the file right after that?
+        }
+        catch (IOException ex)
+        {
+            throw new IOException("Failed to save Bloom filter for SSTable: " + descriptor.baseFilename(), ex);
+        }
+    }
+
+    public static void saveOrDeleteCorrupted(Descriptor descriptor, IFilter filter) throws IOException

Review Comment:
   The name is a little misleading, because it implies one or the other while we do one or both. Maybe `saveWithDeleteOnFailure` or something similar?



##########
src/java/org/apache/cassandra/utils/BloomFilterSerializer.java:
##########
@@ -17,47 +17,66 @@
  */
 package org.apache.cassandra.utils;
 
-import java.io.DataInput;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.OutputStream;
 
 import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IGenericSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.utils.obs.IBitSet;
 import org.apache.cassandra.utils.obs.OffHeapBitSet;
 
-public final class BloomFilterSerializer
+public final class BloomFilterSerializer<I extends InputStream & DataInputPlus, O extends OutputStream & DataOutputPlus> implements IGenericSerializer<BloomFilter, I, O>

Review Comment:
   Is there any benefit from using these types rather than `DataInput/OutputStreamPlus`?



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java:
##########
@@ -439,12 +396,12 @@ void load(boolean recreateBloomFilter,
                   StatsMetadata statsMetadata,
                   Set<Component> components) throws IOException
         {
-            try(FileHandle.Builder ibuilder = new FileHandle.Builder(descriptor.filenameFor(Component.PRIMARY_INDEX))
-                    .mmapped(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap)
-                    .withChunkCache(ChunkCache.instance);
-                    FileHandle.Builder dbuilder = new FileHandle.Builder(descriptor.filenameFor(Component.DATA)).compressed(components.contains(Component.COMPRESSION_INFO))
-                                                                                                                .mmapped(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap)
-                                                                                                                .withChunkCache(ChunkCache.instance))
+            try (FileHandle.Builder ibuilder = new FileHandle.Builder(descriptor.filenameFor(Component.PRIMARY_INDEX))
+                                               .mmapped(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap)
+                                               .withChunkCache(ChunkCache.instance);
+                 FileHandle.Builder dbuilder = new FileHandle.Builder(descriptor.filenameFor(Component.DATA)).compressed(components.contains(Component.COMPRESSION_INFO))

Review Comment:
   Nit: perhaps move to new line and align with new as above?



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java:
##########
@@ -301,11 +257,11 @@ public SSTableReader build()
 
             boolean compression = components.contains(Component.COMPRESSION_INFO);
             try (FileHandle.Builder ibuilder = new FileHandle.Builder(descriptor.filenameFor(Component.PRIMARY_INDEX))
-                    .mmapped(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap)
-                    .withChunkCache(ChunkCache.instance);
-                    FileHandle.Builder dbuilder = new FileHandle.Builder(descriptor.filenameFor(Component.DATA)).compressed(compression)
-                                                                                                                .mmapped(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap)
-                                                                                                                .withChunkCache(ChunkCache.instance))
+                                               .mmapped(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap)
+                                               .withChunkCache(ChunkCache.instance);
+                 FileHandle.Builder dbuilder = new FileHandle.Builder(descriptor.filenameFor(Component.DATA)).compressed(compression)

Review Comment:
   Nit: perhaps move to new line and align with `new` as above?



##########
src/java/org/apache/cassandra/io/sstable/format/big/IndexSummaryComponent.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.big;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+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.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileInputStreamPlus;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class IndexSummaryComponent
+{
+    private static final Logger logger = LoggerFactory.getLogger(IndexSummaryComponent.class);
+
+    public final IndexSummary indexSummary;
+    public final DecoratedKey first;
+    public final DecoratedKey last;
+
+    public IndexSummaryComponent(IndexSummary indexSummary, DecoratedKey first, DecoratedKey last)
+    {
+        this.indexSummary = indexSummary;
+        this.first = first;
+        this.last = last;
+    }
+
+    /**
+     * Load index summary, first key and last key from Summary.db file if it exists.
+     * <p>
+     * if loaded index summary has different index interval from current value stored in schema,
+     * then Summary.db file will be deleted and need to be rebuilt.
+     */
+    public static IndexSummaryComponent load(Descriptor descriptor, TableMetadata metadata) throws IOException
+    {
+        File summaryFile = descriptor.fileFor(Component.SUMMARY);
+        if (!summaryFile.exists())
+        {
+            if (logger.isDebugEnabled())
+                logger.debug("SSTable Summary File {} does not exist", summaryFile.absolutePath());
+            return null;
+        }
+
+        IndexSummary summary = null;
+        try (FileInputStreamPlus iStream = summaryFile.newInputStream())
+        {
+            summary = IndexSummary.serializer.deserialize(iStream,
+                                                          metadata.partitioner,
+                                                          metadata.params.minIndexInterval,
+                                                          metadata.params.maxIndexInterval);
+            DecoratedKey first = metadata.partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
+            DecoratedKey last = metadata.partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
+
+            return new IndexSummaryComponent(summary, first, last);
+        }
+        catch (IOException ex)
+        {
+            if (summary != null)
+                summary.close();
+
+            throw new IOException(String.format("Cannot deserialize SSTable %s component: %s", Component.SUMMARY.name, summaryFile), ex);
+        }
+    }
+
+    public static IndexSummaryComponent loadOrDeleteCorrupted(Descriptor descriptor, TableMetadata metadata) throws IOException
+    {
+        try
+        {
+            return load(descriptor, metadata);
+        }
+        catch (IOException ex)
+        {
+            descriptor.fileFor(Component.SUMMARY).deleteIfExists();
+            throw ex;
+        }
+    }
+
+    /**
+     * Save index summary to Summary.db file.
+     */
+    public void save(Descriptor descriptor) throws IOException
+    {
+        File summaryFile = descriptor.fileFor(Component.SUMMARY);
+        if (summaryFile.exists())
+            summaryFile.delete();
+
+        try (DataOutputStreamPlus oStream = summaryFile.newOutputStream(File.WriteMode.OVERWRITE))
+        {
+            IndexSummary.serializer.serialize(indexSummary, oStream);
+            ByteBufferUtil.writeWithLength(first.getKey(), oStream);
+            ByteBufferUtil.writeWithLength(last.getKey(), oStream);
+        }
+        catch (IOException ex)
+        {
+            throw new IOException("Failed to save index summary for SSTable: " + descriptor.baseFilename(), ex);
+        }
+    }
+
+    public void saveOrDeleteCorrupted(Descriptor descriptor) throws IOException

Review Comment:
   As for bloom filter, change to `WithDeleteIfFailed` or similar.



##########
src/java/org/apache/cassandra/io/sstable/SSTable.java:
##########
@@ -155,9 +155,9 @@ public DecoratedKey decorateKey(ByteBuffer key)
      */
     public static DecoratedKey getMinimalKey(DecoratedKey key)

Review Comment:
   Nit: could this be renamed? E.g. to `onHeapMinimized`? Or something in the vein of `retainable` like the clusterings?



##########
src/java/org/apache/cassandra/io/sstable/Descriptor.java:
##########
@@ -423,6 +425,19 @@ public boolean isCompatible()
         return version.isCompatible();
     }
 
+    public Set<Component> discoverComponents()
+    {
+        Set<Component.Type> knownTypes = Sets.difference(Component.TYPES, Collections.singleton(Component.Type.CUSTOM));
+        Set<Component> components = Sets.newHashSetWithExpectedSize(knownTypes.size());
+        for (Component.Type componentType : knownTypes)
+        {
+            Component component = new Component(componentType);

Review Comment:
   Any reason to not make these singletons?



##########
test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java:
##########
@@ -388,10 +387,10 @@ private void testPosition(IndexSummary original, IndexSummary downsampled, List<
     {
         for (DecoratedKey key : keys)
         {
-            long orig = SSTableReader.getIndexScanPositionFromBinarySearchResult(original.binarySearch(key), original);
+            long orig = original.getScanPositionFromBinarySearchResult(original.binarySearch(key));

Review Comment:
   This can be `original.getScanPosition(key)`.



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java:
##########
@@ -395,12 +393,13 @@ void load(boolean recreateBloomFilter,
                   StatsMetadata statsMetadata,
                   Set<Component> components) throws IOException
         {
-            try (FileHandle.Builder ibuilder = new FileHandle.Builder(descriptor.filenameFor(Component.PRIMARY_INDEX))
-                                               .mmapped(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap)
-                                               .withChunkCache(ChunkCache.instance);
-                 FileHandle.Builder dbuilder = new FileHandle.Builder(descriptor.filenameFor(Component.DATA)).mmapped(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap)
-                                                                                                             .withChunkCache(ChunkCache.instance))
+            try
             {
+                FileHandle.Builder ibuilder = new FileHandle.Builder(descriptor.fileFor(Component.PRIMARY_INDEX))

Review Comment:
   Is everything that can go wrong / throw an exception covered in the `complete` block? Can we leave mmap regions or memory (e.g. compression offsets) unreleased?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1071932857


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java:
##########
@@ -36,6 +39,8 @@
     SSTableWriter.Factory getWriterFactory();
     SSTableReader.Factory getReaderFactory();
 
+    Set<Component> supportedComponents();

Review Comment:
   I don't know, `supported` seemed to most relevant to me as it is intended to return all the components that can be loaded/used/supported by that format. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1072164958


##########
src/java/org/apache/cassandra/io/sstable/format/FilterComponent.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileInputStreamPlus;
+import org.apache.cassandra.io.util.FileOutputStreamPlus;
+import org.apache.cassandra.utils.BloomFilterSerializer;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+
+public class FilterComponent
+{
+    private static final Logger logger = LoggerFactory.getLogger(FilterComponent.class);
+
+    private FilterComponent()
+    {
+    }
+
+    /**
+     * Load bloom filter from Filter.db file.
+     */
+    public static IFilter load(Descriptor descriptor) throws IOException
+    {
+        File filterFile = descriptor.fileFor(Component.FILTER);
+
+        if (!filterFile.exists())
+            return null;
+
+        if (filterFile.length() == 0)
+            return FilterFactory.AlwaysPresent;
+
+        try (FileInputStreamPlus stream = descriptor.fileFor(Component.FILTER).newInputStream())
+        {
+            return BloomFilterSerializer.forVersion(descriptor.version.hasOldBfFormat()).deserialize(stream);
+        }
+        catch (IOException ex)
+        {
+            throw new IOException("Failed to load Bloom filter for SSTable: " + descriptor.baseFilename(), ex);
+        }
+    }
+
+    public static void save(IFilter filter, Descriptor descriptor) throws IOException
+    {
+        File filterFile = descriptor.fileFor(Component.FILTER);
+        try (FileOutputStreamPlus stream = filterFile.newOutputStream(File.WriteMode.OVERWRITE))
+        {
+            filter.serialize(stream, descriptor.version.hasOldBfFormat());
+            stream.flush();
+            stream.sync(); // is it needed if we close the file right after that?
+        }
+        catch (IOException ex)
+        {
+            throw new IOException("Failed to save Bloom filter for SSTable: " + descriptor.baseFilename(), ex);
+        }
+    }
+
+    public static void saveOrDeleteCorrupted(Descriptor descriptor, IFilter filter) throws IOException

Review Comment:
   I'm bad at naming, will change



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1090239891


##########
src/java/org/apache/cassandra/io/sstable/Component.java:
##########
@@ -35,43 +41,49 @@
 {
     public static final char separator = '-';
 
-    final static EnumSet<Type> TYPES = EnumSet.allOf(Type.class);
+    final static ImmutableSet<Type> TYPES = Type.all;
 
     /**
      * WARNING: Be careful while changing the names or string representation of the enum
      * members. Streaming code depends on the names during streaming (Ref: CASSANDRA-14556).
      */
-    public enum Type
+    public static class Type
     {
+        private final static List<Type> singletonsCollector = new ArrayList<>(11);
         // the base data for an sstable: the remaining components can be regenerated
         // based on the data component
-        DATA("Data.db"),
-        // index of the row keys with pointers to their positions in the data file
-        PRIMARY_INDEX("Index.db"),
-        // serialized bloom filter for the row keys in the sstable
-        FILTER("Filter.db"),
+        public static final Type DATA = new Type("DATA", "Data.db", name -> Component.DATA);
         // file to hold information about uncompressed data length, chunk offsets etc.
-        COMPRESSION_INFO("CompressionInfo.db"),
+        public static final Type COMPRESSION_INFO = new Type("COMPRESSION_INFO", "CompressionInfo.db", name -> Component.COMPRESSION_INFO);
         // statistical metadata about the content of the sstable
-        STATS("Statistics.db"),
+        public static final Type STATS = new Type("STATS", "Statistics.db", name -> Component.STATS);
+        // serialized bloom filter for the row keys in the sstable
+        public static final Type FILTER = new Type("FILTER", "Filter.db", name -> Component.FILTER);
         // holds CRC32 checksum of the data file
-        DIGEST("Digest.crc32"),
+        public static final Type DIGEST = new Type("DIGEST","Digest.crc32", name -> Component.DIGEST);
         // holds the CRC32 for chunks in an a uncompressed file.
-        CRC("CRC.db"),
-        // holds SSTable Index Summary (sampling of Index component)
-        SUMMARY("Summary.db"),
+        public static final Type CRC = new Type("CRC","CRC.db", name -> Component.CRC);
         // table of contents, stores the list of all components for the sstable
-        TOC("TOC.txt"),
+        public static final Type TOC = new Type("TOC","TOC.txt", name -> Component.TOC);
         // built-in secondary index (may be multiple per sstable)
-        SECONDARY_INDEX("SI_.*.db"),
+        public static final Type SECONDARY_INDEX = new Type("SECONDARY_INDEX","SI_.*.db", name -> new Component(Type.SECONDARY_INDEX, name));
         // custom component, used by e.g. custom compaction strategy
-        CUSTOM(null);
+        public static final Type CUSTOM = new Type("CUSTOM",null, name -> new Component(Type.CUSTOM, name));
 
-        final String repr;
+        public static final ImmutableSet<Type> all = ImmutableSet.copyOf(singletonsCollector);

Review Comment:
   yes, more work is in some next commit, just wanted you to take an overview of the direction



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1091931571


##########
src/java/org/apache/cassandra/io/sstable/Component.java:
##########
@@ -41,55 +39,76 @@
 {
     public static final char separator = '-';
 
-    final static ImmutableSet<Type> TYPES = Type.all;
+    @SuppressWarnings("rawtypes")
+    public static Set<Component> getSingletonsFor(Class<? extends SSTableFormat> format)
+    {
+        return singletons.stream().filter(c -> c.type.formatClass.isAssignableFrom(format)).collect(Collectors.toSet());
+    }
 
     /**
      * WARNING: Be careful while changing the names or string representation of the enum
      * members. Streaming code depends on the names during streaming (Ref: CASSANDRA-14556).
      */
-    public static class Type
+    public final static class Type
     {
-        private final static List<Type> singletonsCollector = new ArrayList<>(11);
+        private final static CopyOnWriteArrayList<Type> typesCollector = new CopyOnWriteArrayList<>();
         // the base data for an sstable: the remaining components can be regenerated
         // based on the data component
-        public static final Type DATA = new Type("DATA", "Data.db", name -> Component.DATA);
+        public static final Type DATA = new Type("DATA", "Data.db", name -> Component.DATA, true);
         // file to hold information about uncompressed data length, chunk offsets etc.
-        public static final Type COMPRESSION_INFO = new Type("COMPRESSION_INFO", "CompressionInfo.db", name -> Component.COMPRESSION_INFO);
+        public static final Type COMPRESSION_INFO = new Type("COMPRESSION_INFO", "CompressionInfo.db", name -> Component.COMPRESSION_INFO, true);
         // statistical metadata about the content of the sstable
-        public static final Type STATS = new Type("STATS", "Statistics.db", name -> Component.STATS);
+        public static final Type STATS = new Type("STATS", "Statistics.db", name -> Component.STATS, true);
         // serialized bloom filter for the row keys in the sstable
-        public static final Type FILTER = new Type("FILTER", "Filter.db", name -> Component.FILTER);
+        public static final Type FILTER = new Type("FILTER", "Filter.db", name -> Component.FILTER, true);
         // holds CRC32 checksum of the data file
-        public static final Type DIGEST = new Type("DIGEST","Digest.crc32", name -> Component.DIGEST);
-        // holds the CRC32 for chunks in an a uncompressed file.
-        public static final Type CRC = new Type("CRC","CRC.db", name -> Component.CRC);
+        public static final Type DIGEST = new Type("DIGEST","Digest.crc32", name -> Component.DIGEST, true);
+        // holds the CRC32 for chunks in an uncompressed file.
+        public static final Type CRC = new Type("CRC","CRC.db", name -> Component.CRC, true);
         // table of contents, stores the list of all components for the sstable
-        public static final Type TOC = new Type("TOC","TOC.txt", name -> Component.TOC);
+        public static final Type TOC = new Type("TOC","TOC.txt", name -> Component.TOC, true);
         // built-in secondary index (may be multiple per sstable)
-        public static final Type SECONDARY_INDEX = new Type("SECONDARY_INDEX","SI_.*.db", name -> new Component(Type.SECONDARY_INDEX, name));
+        public static final Type SECONDARY_INDEX = new Type("SECONDARY_INDEX","SI_.*.db", name -> new Component(Type.SECONDARY_INDEX, name), false);
         // custom component, used by e.g. custom compaction strategy
-        public static final Type CUSTOM = new Type("CUSTOM",null, name -> new Component(Type.CUSTOM, name));
+        public static final Type CUSTOM = new Type("CUSTOM",null, name -> new Component(Type.CUSTOM, name), false);
 
-        public static final ImmutableSet<Type> all = ImmutableSet.copyOf(singletonsCollector);
+        public static final List<Type> all = Collections.unmodifiableList(typesCollector);
 
         public final int id;
         public final String name;
         public final String repr;
-        private Function<String, Component> componentFactory;
+        private final Function<String, Component> componentFactory;
+        public final boolean isSingleton;
+
+        @SuppressWarnings("rawtypes")
+        public final Class<? extends SSTableFormat> formatClass;
+
+        private Type(String name, String repr, Function<String, Component> componentFactory, boolean isSingleton)
+        {
+            this(name, repr, componentFactory, isSingleton, SSTableFormat.class);
+        }
 
-        protected Type(String name, String repr, Function<String, Component> componentFactory)
+        @SuppressWarnings("rawtypes")
+        public Type(String name, String repr, Function<String, Component> componentFactory, boolean isSingleton, Class<? extends SSTableFormat> formatClass)
         {
             this.name = name;
             this.repr = repr;
-            this.id = singletonsCollector.size();
+            this.id = typesCollector.size();
             this.componentFactory = componentFactory;
-            singletonsCollector.add(this);
+            this.isSingleton = isSingleton;
+            this.formatClass = formatClass;
+            synchronized (typesCollector)
+            {
+                if (typesCollector.stream().anyMatch(t -> Objects.equal(t.name, name) && t.formatClass == formatClass || t.formatClass.isAssignableFrom(formatClass) || formatClass.isAssignableFrom(t.formatClass)))

Review Comment:
   Isn't this expression always true for subtypes, e.g. the big format types?
   
   I.e. is this missing a pair of brackets around the disjunction?



##########
src/java/org/apache/cassandra/io/sstable/SimpleGaugeProvider.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.util.Objects;
+import java.util.function.Function;
+
+import com.google.common.collect.Iterables;
+
+import com.codahale.metrics.Gauge;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.lifecycle.SSTableSet;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+
+public class SimpleGaugeProvider<T extends Number, R extends SSTableReader> extends GaugeProvider<T>
+{
+    private final Function<SSTableReader, R> mapper;
+    private final Function<Iterable<R>, T> combiner;
+
+    public SimpleGaugeProvider(Function<SSTableReader, R> mapper, String name, Function<Iterable<R>, T> combiner)
+    {
+        super(name);
+        this.mapper = mapper;
+        this.combiner = combiner;
+    }
+
+    @Override
+    public Gauge<T> getTableGauge(ColumnFamilyStore cfs)
+    {
+        return () -> combine(cfs.getLiveSSTables());
+    }
+
+    @Override
+    public Gauge<T> getKeyspaceGauge(Keyspace keyspace)
+    {
+        return () -> combine(getAllReaders(keyspace));
+    }
+
+    @Override
+    public Gauge<T> getGlobalGauge()
+    {
+        return () -> combine(Iterables.concat(Iterables.transform(Keyspace.all(), SimpleGaugeProvider::getAllReaders)));
+    }
+
+    private T combine(Iterable<SSTableReader> allReaders)
+    {
+        Iterable<R> readers = Iterables.filter(Iterables.transform(allReaders, mapper::apply), Objects::nonNull);
+        return combiner.apply(readers);
+    }
+
+    private static Iterable<SSTableReader> getAllReaders(Keyspace keyspace)
+    {
+        return Iterables.concat(Iterables.transform(keyspace.getColumnFamilyStores(), cfs -> cfs.getSSTables(SSTableSet.LIVE)));

Review Comment:
   Nit: for consistency, can this also use `cfd.getLiveSSTables()`?



##########
src/java/org/apache/cassandra/io/sstable/BloomFilterMetrics.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.util.Arrays;
+import java.util.List;
+
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+
+public class BloomFilterMetrics<R extends SSTableReader & BloomFilterSupport<R>> extends AbstractMetricsProviders<R>
+{
+    public final static BloomFilterMetrics<?> instance = new BloomFilterMetrics<>();
+
+    @Override
+    protected R map(SSTableReader r)
+    {
+        if (r instanceof BloomFilterSupport<?>)
+            return (R) r;
+        return null;
+    }
+
+    /**
+     * Number of false positives in bloom filter
+     */
+    public final GaugeProvider<Long> bloomFilterFalsePositives = newGaugeProvider("BloomFilterFalsePositives",
+                                                                                  0L,
+                                                                                  r -> r.getBloomFilterTracker().getFalsePositiveCount(),
+                                                                                  Long::sum);
+
+    /**
+     * Number of false positives in bloom filter from last read
+     */
+    public final GaugeProvider<Long> recentBloomFilterFalsePositives = newGaugeProvider("RecentBloomFilterFalsePositives",
+                                                                                        0L,
+                                                                                        r -> r.getBloomFilterTracker().getRecentFalsePositiveCount(),
+                                                                                        Long::sum);
+
+    /**
+     * Disk space used by bloom filter
+     */
+    public final GaugeProvider<Long> bloomFilterDiskSpaceUsed = newGaugeProvider("BloomFilterDiskSpaceUsed",
+                                                                                 0L,
+                                                                                 r -> r.getBloomFilter().serializedSize(((SSTableReader) r).descriptor.version.hasOldBfFormat()),
+                                                                                 Long::sum);
+
+    /**
+     * Off heap memory used by bloom filter
+     */
+    public final GaugeProvider<Long> bloomFilterOffHeapMemoryUsed = newGaugeProvider("BloomFilterOffHeapMemoryUsed",
+                                                                                     0L,
+                                                                                     r -> r.getBloomFilter().offHeapSize(),
+                                                                                     Long::sum);
+
+    /**
+     * False positive ratio of bloom filter
+     */
+    public final GaugeProvider<Double> bloomFilterFalseRatio = newGaugeProvider("BloomFilterFalseRatio", readers -> {
+        long falsePositiveCount = 0L;
+        long truePositiveCount = 0L;
+        long trueNegativeCount = 0L;
+        for (BloomFilterSupport<?> sstable : readers)
+        {
+            falsePositiveCount += sstable.getBloomFilterTracker().getFalsePositiveCount();
+            truePositiveCount += sstable.getBloomFilterTracker().getTruePositiveCount();
+            trueNegativeCount += sstable.getBloomFilterTracker().getTrueNegativeCount();
+        }
+        if (falsePositiveCount == 0L && truePositiveCount == 0L)
+            return 0d;
+        return (double) falsePositiveCount / (truePositiveCount + falsePositiveCount + trueNegativeCount);
+    });
+
+    /**
+     * False positive ratio of bloom filter from last read
+     */
+    public final GaugeProvider<Double> recentBloomFilterFalseRatio = newGaugeProvider("BloomFilterFalseRatio", readers -> {

Review Comment:
   This is the same as above, "Recent" needs to be added to the name as the calls below.



##########
src/java/org/apache/cassandra/io/sstable/KeyCacheSupport.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.IOException;
+
+import org.apache.cassandra.db.ClusteringPrefix;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.filter.ClusteringIndexFilter;
+import org.apache.cassandra.io.sstable.format.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.DataInputPlus;
+
+public interface KeyCacheSupport<T extends SSTableReader & KeyCacheSupport<T>>

Review Comment:
   `T` is never referenced, this should not be generic.



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java:
##########
@@ -115,38 +123,75 @@ default boolean delete(Descriptor desc)
 
     FileTime getLastModifiedTime(Descriptor desc);
 
-    enum Type
+    default void setup(Map<String, String> options)
+    {
+    }
+
+    class Type
     {
-        //The original sstable format
-        BIG("big", BigFormat.instance);
+        private final static ImmutableList<Type> types;
+
+        static
+        {
+            Map<String, Supplier<SSTableFormat<?, ?>>> factories = DatabaseDescriptor.getSSTableFormatFactories();
+            List<Type> typesList = new ArrayList<>(factories.size());
+            factories.forEach((key, factory) -> typesList.add(new Type(typesList.size(), key.toLowerCase(), factory.get())));
+            types = ImmutableList.copyOf(typesList);
+        }
 
+        public final int ordinal;
         public final SSTableFormat<?, ?> info;
         public final String name;
+        private final static Type[] typesArray0 = new Type[0];
+
+        private static Type currentType;
 
         public static Type current()
         {
-            return CassandraRelevantProperties.SSTABLE_FORMAT_DEFAULT.getEnum(true, Type.class);
+            if (currentType != null)
+                return currentType;
+
+            String key = CassandraRelevantProperties.SSTABLE_FORMAT_DEFAULT.getString();
+            try
+            {
+                Type type = validate(key);
+                currentType = type;
+                return type;
+            }
+            catch (RuntimeException ex)
+            {
+                throw new ConfigurationException("SSTable format " + key + " is not registered. Registered formats are: " + types);
+            }
         }
 
-        Type(String name, SSTableFormat<?, ?> info)
+        private Type(int ordinal, String name, SSTableFormat<?, ?> info)
         {
             //Since format comes right after generation
             //we disallow formats with numeric names
             assert !CharMatcher.digit().matchesAllOf(name);
-
+            this.ordinal = ordinal;
             this.name = name;
             this.info = info;
         }
 
         public static Type validate(String name)
         {
-            for (Type valid : Type.values())
-            {
-                if (valid.name.equalsIgnoreCase(name))
-                    return valid;
-            }
+            for (int i = 0; i < types.size(); i++)

Review Comment:
   Nit: let's just lowercase the name instead of using `equalsIgnoreCase` repeatedly.



##########
src/java/org/apache/cassandra/service/CacheService.java:
##########
@@ -467,9 +467,15 @@ public Future<Pair<KeyCacheKey, AbstractRowIndexEntry>> deserialize(DataInputPlu
                                                    ? SSTableIdFactory.instance.fromBytes(ByteBufferUtil.readWithShortLength(input))
                                                    : new SequenceBasedSSTableId(generation); // Backwards compatibility for "int based generation sstables"
             int typeOrdinal = input.readByte();
-            if (typeOrdinal < 0 || typeOrdinal >= SSTableFormat.Type.values().length)
-                throw new IOException("Failed to deserialize key of key cache - invalid type ordinal " + typeOrdinal);
-            SSTableFormat.Type type = SSTableFormat.Type.values()[typeOrdinal];
+            SSTableFormat.Type type;
+            try
+            {
+                type = SSTableFormat.Type.getByOrdinal(typeOrdinal);

Review Comment:
   Some user will change the configuration in a way that is incompatible with this (e.g. reorder); we should put the identifier here.



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -1415,7 +1418,7 @@ public boolean mayContainAssumingKeyIsInRange(DecoratedKey key)
     protected static final class InstanceTidier implements Tidy
     {
         private final Descriptor descriptor;
-        private final TableId tableId;
+        private final Owner owner;

Review Comment:
   I guess I ended up asking the wrong question, I was wondering if it's okay to use the CFS after it has been released.
   
   But the CFS is not reference counted and explicitly released, so this should be fine. And I wouldn't worry about holding on to that reference a little longer than strictly necessary.



##########
src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java:
##########
@@ -687,4 +705,41 @@ private IndexSummary buildSummaryAtLevel(int newSamplingLevel) throws IOExceptio
         }
     }
 
+    @Override
+    protected AbstractRowIndexEntry getCachedPosition(KeyCacheKey unifiedKey, boolean updateStats)
+    {
+        AbstractRowIndexEntry cachedEntry;
+        if (isKeyCacheEnabled())
+        {
+            if (updateStats)
+            {
+                cachedEntry = keyCache.get(unifiedKey);
+                keyCacheRequest.incrementAndGet();
+                if (cachedEntry != null)
+                {
+                    keyCacheHit.incrementAndGet();
+                    bloomFilterTracker.addTruePositive();

Review Comment:
   Preexisting problem, but this violates encapsulation and is much more naturally applied by the caller `getRowIndexEntry`.



##########
src/java/org/apache/cassandra/config/Config.java:
##########
@@ -352,6 +354,8 @@ public MemtableOptions()
 
     public String[] data_file_directories = new String[0];
 
+    public Map<String, SSTableFormatConfig> sstable_formats = ImmutableMap.of("big", new SSTableFormatConfig(BigFormat.class.getName(), Collections.emptyMap()));

Review Comment:
   We usually use `ParameterizedClass` for class-with-options settings such as this one. Is it not suitable here?



##########
src/java/org/apache/cassandra/io/sstable/BloomFilterSupport.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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 org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.utils.IFilter;
+
+public interface BloomFilterSupport<T extends SSTableReader & BloomFilterSupport<T>>
+{
+    IFilter getBloomFilter();
+
+    BloomFilterTracker getBloomFilterTracker();
+
+    T cloneAndReplace(IFilter newBloomFilter);

Review Comment:
   What is the benefit of having this generic over simply making the implementations return the proper type?
   
   Btw. this method appears to be unused.



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java:
##########
@@ -115,38 +123,75 @@ default boolean delete(Descriptor desc)
 
     FileTime getLastModifiedTime(Descriptor desc);
 
-    enum Type
+    default void setup(Map<String, String> options)
+    {
+    }
+
+    class Type
     {
-        //The original sstable format
-        BIG("big", BigFormat.instance);
+        private final static ImmutableList<Type> types;
+
+        static
+        {
+            Map<String, Supplier<SSTableFormat<?, ?>>> factories = DatabaseDescriptor.getSSTableFormatFactories();
+            List<Type> typesList = new ArrayList<>(factories.size());
+            factories.forEach((key, factory) -> typesList.add(new Type(typesList.size(), key.toLowerCase(), factory.get())));
+            types = ImmutableList.copyOf(typesList);
+        }
 
+        public final int ordinal;
         public final SSTableFormat<?, ?> info;
         public final String name;
+        private final static Type[] typesArray0 = new Type[0];
+
+        private static Type currentType;
 
         public static Type current()
         {
-            return CassandraRelevantProperties.SSTABLE_FORMAT_DEFAULT.getEnum(true, Type.class);
+            if (currentType != null)
+                return currentType;
+
+            String key = CassandraRelevantProperties.SSTABLE_FORMAT_DEFAULT.getString();

Review Comment:
   We should use the first defined format as the default if this option is not supplied.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1093257966


##########
src/java/org/apache/cassandra/service/CacheService.java:
##########
@@ -467,9 +467,15 @@ public Future<Pair<KeyCacheKey, AbstractRowIndexEntry>> deserialize(DataInputPlu
                                                    ? SSTableIdFactory.instance.fromBytes(ByteBufferUtil.readWithShortLength(input))
                                                    : new SequenceBasedSSTableId(generation); // Backwards compatibility for "int based generation sstables"
             int typeOrdinal = input.readByte();
-            if (typeOrdinal < 0 || typeOrdinal >= SSTableFormat.Type.values().length)
-                throw new IOException("Failed to deserialize key of key cache - invalid type ordinal " + typeOrdinal);
-            SSTableFormat.Type type = SSTableFormat.Type.values()[typeOrdinal];
+            SSTableFormat.Type type;
+            try
+            {
+                type = SSTableFormat.Type.getByOrdinal(typeOrdinal);

Review Comment:
   I'll add the identifier as a part of configuration in cassandra.yaml



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1090268316


##########
src/java/org/apache/cassandra/io/util/MmappedRegions.java:
##########
@@ -140,6 +147,25 @@ public void extend(long length)
         copy = new State(state);
     }
 
+    public void extend(CompressionMetadata compressionMetadata)
+    {
+        assert !isCopy() : "Copies cannot be extended";
+
+        if (compressionMetadata.dataLength <= state.length)

Review Comment:
   Actually, there is no comment to add. It is simply a bug. Not a harmful one, but a bug. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1090239538


##########
src/java/org/apache/cassandra/io/util/MmappedRegions.java:
##########
@@ -140,6 +147,25 @@ public void extend(long length)
         copy = new State(state);
     }
 
+    public void extend(CompressionMetadata compressionMetadata)
+    {
+        assert !isCopy() : "Copies cannot be extended";
+
+        if (compressionMetadata.dataLength <= state.length)

Review Comment:
   I'll add a comment



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1109546204


##########
test/unit/org/apache/cassandra/Util.java:
##########
@@ -1236,4 +1241,18 @@ public static void flush(TableViews view)
     {
         view.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS);
     }
+
+    public static DataInputPlus.DataInputStreamPlus createDataInputStreamPlus(InputStream in)
+    {
+        DataInputStreamPlusImpl impl = new DataInputStreamPlusImpl(in);
+        return Mockito.mock(DataInputPlus.DataInputStreamPlus.class, new ForwardsInvocations(impl));

Review Comment:
   Isn't it risky to always mock (i.e. track invocations), which can cause a lot of overhead?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1071833912


##########
src/java/org/apache/cassandra/db/compaction/Scrubber.java:
##########
@@ -564,20 +565,27 @@ public ScrubResult(Scrubber scrubber)
     /**
      * During 2.x migration, under some circumstances rows might have gotten duplicated.
      * Merging iterator merges rows with same clustering.
-     *
+     * <p>
      * For more details, refer to CASSANDRA-12144.
      */
-    private static class RowMergingSSTableIterator extends WrappingUnfilteredRowIterator
+    private static class RowMergingSSTableIterator extends UnmodifiableIterator<Unfiltered> implements WrappingUnfilteredRowIterator

Review Comment:
   So `remove` is implemented in Java's `Iterator` in the same way as in `UnmodifiableIterator` - did you mean that?



##########
src/java/org/apache/cassandra/db/compaction/Scrubber.java:
##########
@@ -742,6 +750,12 @@ public FixNegativeLocalDeletionTimeIterator(UnfilteredRowIterator iterator, Outp
             this.negativeLocalExpirationTimeMetrics = negativeLocalDeletionInfoMetrics;
         }
 
+        @Override
+        public UnfilteredRowIterator wrapped()
+        {
+            return iterator;
+        }
+

Review Comment:
   yes, sorry, oversight



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1071845272


##########
src/java/org/apache/cassandra/db/compaction/Scrubber.java:
##########
@@ -742,6 +750,12 @@ public FixNegativeLocalDeletionTimeIterator(UnfilteredRowIterator iterator, Outp
             this.negativeLocalExpirationTimeMetrics = negativeLocalDeletionInfoMetrics;
         }
 
+        @Override
+        public UnfilteredRowIterator wrapped()
+        {
+            return iterator;
+        }
+

Review Comment:
   it was actually applied in some of the next steps, but it should be in this commit



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1072070704


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -1973,17 +1970,36 @@ public void run()
                     if (logger.isTraceEnabled())
                         logger.trace("Async instance tidier for {}, after barrier", descriptor);
 
-                    if (bf != null)
-                        bf.close();
-                    if (summary != null)
-                        summary.close();
-                    if (runOnClose != null)
+                    Throwable exceptions = null;
+                    if (runOnClose != null) try
+                    {
                         runOnClose.run();
-                    if (dfile != null)
-                        dfile.close();
-                    if (ifile != null)
-                        ifile.close();
-                    globalRef.release();
+                    }
+                    catch (RuntimeException | Error ex)
+                    {
+                        logger.error("Failed to run on-close listeners for sstable " + descriptor.baseFilename(), ex);
+                        exceptions = ex;
+                    }
+
+                    Throwable closeExceptions = Throwables.close(null, closeables);

Review Comment:
   I'm not sure TBH, need to get the exceptions reported and log them, add them to the `exceptions`. With `FileUtils.close` I'd have to catch the exception. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1082599640


##########
src/java/org/apache/cassandra/io/sstable/Descriptor.java:
##########
@@ -423,6 +425,19 @@ public boolean isCompatible()
         return version.isCompatible();
     }
 
+    public Set<Component> discoverComponents()
+    {
+        Set<Component.Type> knownTypes = Sets.difference(Component.TYPES, Collections.singleton(Component.Type.CUSTOM));
+        Set<Component> components = Sets.newHashSetWithExpectedSize(knownTypes.size());
+        for (Component.Type componentType : knownTypes)
+        {
+            Component component = new Component(componentType);

Review Comment:
   I don't think I can do that because in that case when `new Component` was called, the `singletons` field would not be initialized yet.
   
   Given your other comments, we will probably go back to `Component` class when making sstable format really pluggable



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1083870360


##########
src/java/org/apache/cassandra/io/sstable/format/FilterComponent.java:
##########
@@ -90,4 +97,63 @@ public static void saveOrDeleteCorrupted(Descriptor descriptor, IFilter filter)
             throw ex;
         }
     }
+
+    /**
+     * Optionally loads a Bloom filter. If the filter is not needed (FP chance is neglectable), it sets
+     * {@link AlwaysPresentFilter} as a filter in the builder. If the filter is expected to be recreated for various
+     * reasons, it leaves it {@code null} (unchanged). Otherwise, it attempts to load the filter, and if it succeeds,
+     * it is set in the builder. If a filter fails to load, it is left {@code null} (unchanged) meaning that it should
+     * be rebuilt.
+     */
+    public static IFilter maybeLoadBloomFilter(Descriptor descriptor, Set<Component> components, TableMetadata metadata, ValidationMetadata validationMetadata)
+    {
+        double currentFPChance = validationMetadata != null ? validationMetadata.bloomFilterFPChance : Double.NaN;
+        double desiredFPChance = metadata.params.bloomFilterFpChance;
+
+        IFilter filter = null;
+        if (!shouldUseBloomFilter(desiredFPChance))
+        {
+            if (logger.isTraceEnabled())
+                logger.trace("Bloom filter for {} will not be loaded because fpChance={} is neglectable", descriptor, desiredFPChance);
+
+            return FilterFactory.AlwaysPresent;
+        }
+        else if (!components.contains(Component.FILTER) || Double.isNaN(currentFPChance))
+        {
+            if (logger.isTraceEnabled())
+                logger.trace("Bloom filter for {} will not be loaded because filter component is missing or sstable lacks validation metadata", descriptor);
+
+            return null;
+        }
+        else if (!isFPChanceDiffNeglectable(desiredFPChance, currentFPChance) && rebuildFilterOnFPChanceChange)
+        {
+            if (logger.isTraceEnabled())
+                logger.trace("Bloom filter for {} will not be loaded because fpChance has changed from {} to {} and the filter should be recreated", descriptor, currentFPChance, desiredFPChance);
+
+            return null;
+        }
+
+        try
+        {
+            filter = load(descriptor);
+            if (filter == null || filter instanceof AlwaysPresentFilter)
+                logger.info("Bloom filter for {} is missing or invalid", descriptor);
+        }
+        catch (IOException ex)
+        {
+            logger.info("Bloom filter for " + descriptor + " could not be deserialized", ex);
+        }
+
+        return filter;
+    }
+
+    static boolean shouldUseBloomFilter(double fpChance)
+    {
+        return !(Math.abs(1 - fpChance) <= filterFPChanceTolerance);
+    }
+
+    static boolean isFPChanceDiffNeglectable(double fpChance1, double fpChance2)

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1083874588


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -1973,17 +1970,36 @@ public void run()
                     if (logger.isTraceEnabled())
                         logger.trace("Async instance tidier for {}, after barrier", descriptor);
 
-                    if (bf != null)
-                        bf.close();
-                    if (summary != null)
-                        summary.close();
-                    if (runOnClose != null)
+                    Throwable exceptions = null;
+                    if (runOnClose != null) try
+                    {
                         runOnClose.run();
-                    if (dfile != null)
-                        dfile.close();
-                    if (ifile != null)
-                        ifile.close();
-                    globalRef.release();
+                    }
+                    catch (RuntimeException | Error ex)
+                    {
+                        logger.error("Failed to run on-close listeners for sstable " + descriptor.baseFilename(), ex);
+                        exceptions = ex;
+                    }
+
+                    Throwable closeExceptions = Throwables.close(null, closeables);

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1083874212


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -808,32 +809,12 @@ 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);
+            tidy.runOnClose = () -> {

Review Comment:
   The new tests look great. Thank you for adding them.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1083888428


##########
src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java:
##########
@@ -56,22 +55,23 @@ class SASIIndexBuilder extends SecondaryIndexBuilder
     private final SortedMap<SSTableReader, Map<ColumnMetadata, ColumnIndex>> sstables;
 
     private long bytesProcessed = 0;
-    private final long totalSizeInBytes;
+    private final long totalBytesToProcess;
 
     public SASIIndexBuilder(ColumnFamilyStore cfs, SortedMap<SSTableReader, Map<ColumnMetadata, ColumnIndex>> sstables)
     {
-        long totalIndexBytes = 0;
+        long totalBytesToProcess = 0;
         for (SSTableReader sstable : sstables.keySet())
-            totalIndexBytes += getPrimaryIndexLength(sstable);
+            totalBytesToProcess += sstable.uncompressedLength();

Review Comment:
   I can't find any doc online that states this, so I don't think we need to do anything here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1072011575


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java:
##########
@@ -395,12 +393,13 @@ void load(boolean recreateBloomFilter,
                   StatsMetadata statsMetadata,
                   Set<Component> components) throws IOException
         {
-            try (FileHandle.Builder ibuilder = new FileHandle.Builder(descriptor.filenameFor(Component.PRIMARY_INDEX))
-                                               .mmapped(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap)
-                                               .withChunkCache(ChunkCache.instance);
-                 FileHandle.Builder dbuilder = new FileHandle.Builder(descriptor.filenameFor(Component.DATA)).mmapped(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap)
-                                                                                                             .withChunkCache(ChunkCache.instance))
+            try
             {
+                FileHandle.Builder ibuilder = new FileHandle.Builder(descriptor.fileFor(Component.PRIMARY_INDEX))

Review Comment:
   `FileHandleBuilder` being closeable always annoyed me; I do prefer to make it a trivial builder that cannot fail or that doesn't manage any state. However, we will have to manage that state explicitly in its users now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081243042


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java:
##########
@@ -1983,39 +1380,36 @@ public void addTo(Ref.IdentityCollection identities)
     {
         identities.add(this);
         identities.add(tidy.globalRef);
-        dfile.addTo(identities);
-        ifile.addTo(identities);
-        bf.addTo(identities);
-        indexSummary.addTo(identities);
-
+        tidy.closeables.forEach(c -> {
+            if (c instanceof SharedCloseable)
+                ((SharedCloseable) c).addTo(identities);
+        });
     }
 
     public boolean maybePresent(DecoratedKey key)

Review Comment:
   Sure, but as I'm looking at this method, I'm not sure what's the contract of it? depending on the filter we return whether it is present or may be present. Though, the usages suggest that we rather expect it to return a definite answer. Is this method really needed in that form?
   
   I'd rather implement it this way:
   
   ```java
   return key.compareTo(first) >= 0 && key.compareTo(last) <= 0 && bf.isPresent(key) && getPosition(key, EQ) >= 0
   ```
   
   wdyt?
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081296715


##########
src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java:
##########
@@ -374,4 +380,20 @@ public ClusteringPrefix<?> getLowerBoundPrefixFromCache(DecoratedKey partitionKe
         }
 
     }
-}
+
+    @Override
+    public IScrubber getScrubber(LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(metadata());

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081306889


##########
src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java:
##########
@@ -125,6 +130,24 @@ public AbstractRowIndexEntry.KeyCacheValueSerializer<BigTableReader, RowIndexEnt
         return KeyCacheValueSerializer.instance;
     }
 
+    @Override
+    public BigTableReader cast(SSTableReader sstr)
+    {
+        return sstr == null ? null : (BigTableReader) sstr;

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081326401


##########
src/java/org/apache/cassandra/io/sstable/Descriptor.java:
##########
@@ -171,6 +172,18 @@ public List<File> getTemporaryFiles()
         return ret;
     }
 
+    public Set<Component> getComponents(Set<Component> alwaysAdd, Set<Component> optional)

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081329270


##########
src/java/org/apache/cassandra/io/sstable/format/FilterComponent.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileInputStreamPlus;
+import org.apache.cassandra.io.util.FileOutputStreamPlus;
+import org.apache.cassandra.utils.BloomFilterSerializer;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+
+public class FilterComponent
+{
+    private static final Logger logger = LoggerFactory.getLogger(FilterComponent.class);
+
+    private FilterComponent()
+    {
+    }
+
+    /**
+     * Load bloom filter from Filter.db file.
+     */
+    public static IFilter load(Descriptor descriptor) throws IOException
+    {
+        File filterFile = descriptor.fileFor(Component.FILTER);
+
+        if (!filterFile.exists())
+            return null;
+
+        if (filterFile.length() == 0)
+            return FilterFactory.AlwaysPresent;
+
+        try (FileInputStreamPlus stream = descriptor.fileFor(Component.FILTER).newInputStream())
+        {
+            return BloomFilterSerializer.forVersion(descriptor.version.hasOldBfFormat()).deserialize(stream);
+        }
+        catch (IOException ex)
+        {
+            throw new IOException("Failed to load Bloom filter for SSTable: " + descriptor.baseFilename(), ex);
+        }
+    }
+
+    public static void save(IFilter filter, Descriptor descriptor) throws IOException
+    {
+        File filterFile = descriptor.fileFor(Component.FILTER);
+        try (FileOutputStreamPlus stream = filterFile.newOutputStream(File.WriteMode.OVERWRITE))
+        {
+            filter.serialize(stream, descriptor.version.hasOldBfFormat());
+            stream.flush();
+            stream.sync(); // is it needed if we close the file right after that?
+        }
+        catch (IOException ex)
+        {
+            throw new IOException("Failed to save Bloom filter for SSTable: " + descriptor.baseFilename(), ex);
+        }
+    }
+
+    public static void saveOrDeleteCorrupted(Descriptor descriptor, IFilter filter) throws IOException

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2064: CASSANDRA-17056: CEP-17

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1081428139


##########
src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java:
##########
@@ -43,11 +43,11 @@
  * for on-close (i.e. when all references expire) that drops the page cache prior to that key position
  *
  * hard-links are created for each partially written sstable so that readers opened against them continue to work past
- * the rename of the temporary file, which is deleted once all readers against the hard-link have been closed.
+ * renaming of the temporary file, which is deleted once all readers against the hard-link have been closed.
  * If for any reason the writer is rolled over, we immediately rename and fully expose the completed file in the Tracker.
  *
- * On abort we restore the original lower bounds to the existing readers and delete any temporary files we had in progress,
- * but leave any hard-links in place for the readers we opened to cleanup when they're finished as we would had we finished
+ * On abort, we restore the original lower bounds to the existing readers and delete any temporary files we had in progress,
+ * but leave any hard-links in place for the readers we opened to clean-up when they're finished as we would have finished

Review Comment:
   looks good



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org