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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to