You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by "jacek-lewandowski (via GitHub)" <gi...@apache.org> on 2023/04/25 15:07:57 UTC

[GitHub] [cassandra] jacek-lewandowski opened a new pull request, #2295: Cassandra 18441

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

   Thanks for sending a pull request! Here are some tips if you're new here:
    
    * Ensure you have added or run the [appropriate tests](https://cassandra.apache.org/_/development/testing.html) for your PR.
    * Be sure to keep the PR description updated to reflect all changes.
    * Write your PR title to summarize what this PR proposes.
    * If possible, provide a concise example to reproduce the issue for a faster review.
    * Read our [contributor guidelines](https://cassandra.apache.org/_/development/index.html)
    * If you're making a documentation change, see our [guide to documentation contribution](https://cassandra.apache.org/_/development/documentation.html)
    
   Commit messages should follow the following format:
   
   ```
   <One sentence description, usually Jira title or CHANGES.txt summary>
   
   <Optional lengthier description (context on patch)>
   
   patch by <Authors>; reviewed by <Reviewers> for CASSANDRA-#####
   
   Co-authored-by: Name1 <email1>
   Co-authored-by: Name2 <email2>
   
   ```
   
   The [Cassandra Jira](https://issues.apache.org/jira/projects/CASSANDRA/issues/)
   
   


-- 
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 #2295: Cassandra 18441

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


##########
src/java/org/apache/cassandra/db/lifecycle/LogFile.java:
##########
@@ -504,7 +507,7 @@ List<String> getFilePaths()
 
     private String getFileName()
     {
-        return StringUtils.join(SSTableFormat.Type.current().info.getLatestVersion(), LogFile.SEP, // remove version and separator when downgrading to 4.x is becomes unsupported
+        return StringUtils.join(version, LogFile.SEP, // remove version and separator when downgrading to 4.x is becomes unsupported TODO should this include format type as well?

Review Comment:
   Yes, I think that I oversaw that in CEP-17



-- 
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 #2295: Cassandra 18441

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


##########
src/java/org/apache/cassandra/config/Config.java:
##########
@@ -359,9 +352,14 @@ public MemtableOptions()
 
     public String[] data_file_directories = new String[0];
 
-    public List<ParameterizedClass> sstable_formats = ImmutableList.of(new ParameterizedClass(BigFormat.class.getName(),// "org.apache.cassandra.io.sstable.format.big.BigFormat",
-                                                                                              ImmutableMap.of(SSTABLE_FORMAT_ID, "0",
-                                                                                                              SSTABLE_FORMAT_NAME, "big")));
+    public static class SSTableFormats
+    {
+        public String selected_format;
+        public String selected_version;
+        public Map<String, Map<String, String>> options = new HashMap<>();
+    }
+
+    public SSTableFormats sstable_formats = new SSTableFormats();

Review Comment:
   fixed



-- 
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] dcapwell commented on a diff in pull request #2295: Cassandra 18441

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


##########
src/java/org/apache/cassandra/config/Config.java:
##########
@@ -359,9 +352,14 @@ public MemtableOptions()
 
     public String[] data_file_directories = new String[0];
 
-    public List<ParameterizedClass> sstable_formats = ImmutableList.of(new ParameterizedClass(BigFormat.class.getName(),// "org.apache.cassandra.io.sstable.format.big.BigFormat",
-                                                                                              ImmutableMap.of(SSTABLE_FORMAT_ID, "0",
-                                                                                                              SSTABLE_FORMAT_NAME, "big")));
+    public static class SSTableFormats
+    {
+        public String selected_format;
+        public String selected_version;
+        public Map<String, Map<String, String>> options = new HashMap<>();
+    }
+
+    public SSTableFormats sstable_formats = new SSTableFormats();

Review Comment:
   nit: can use `final` here



##########
src/java/org/apache/cassandra/config/Config.java:
##########
@@ -359,9 +352,14 @@ public MemtableOptions()
 
     public String[] data_file_directories = new String[0];
 
-    public List<ParameterizedClass> sstable_formats = ImmutableList.of(new ParameterizedClass(BigFormat.class.getName(),// "org.apache.cassandra.io.sstable.format.big.BigFormat",
-                                                                                              ImmutableMap.of(SSTABLE_FORMAT_ID, "0",
-                                                                                                              SSTABLE_FORMAT_NAME, "big")));
+    public static class SSTableFormats
+    {
+        public String selected_format;
+        public String selected_version;
+        public Map<String, Map<String, String>> options = new HashMap<>();
+    }
+
+    public SSTableFormats sstable_formats = new SSTableFormats();

Review Comment:
   This doesn't seem to match the yaml format talked about in JIRA?
   
   ```
     sstable:
       format:
         default: bti
         bti:
           row_index_granularity: 4KiB
   ```



##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -1349,68 +1359,97 @@ public static void applyPartitioner(Config conf)
         paritionerName = partitioner.getClass().getCanonicalName();
     }
 
-    @VisibleForTesting
-    public static Map<String, Supplier<SSTableFormat<?, ?>>> loadSSTableFormatFactories(List<ParameterizedClass> configuredFormats)
+    private static void validateSSTableFormatFactories(Iterable<SSTableFormat.Factory> factories)
     {
-        ImmutableMap.Builder<String, Supplier<SSTableFormat<?, ?>>> sstableFormatFactories = ImmutableMap.builderWithExpectedSize(configuredFormats.size());
-        Set<String> names = new HashSet<>(configuredFormats.size());
-        Set<Integer> ids = new HashSet<>(configuredFormats.size());
+        Map<String, SSTableFormat.Factory> factoryByName = new HashMap<>();
+        for (SSTableFormat.Factory factory : factories)
+        {
+            if (factory.name() == null)
+                throw new ConfigurationException(String.format("SSTable format name in %s cannot be null", factory.getClass().getName()));
+
+            if (!factory.name().matches("^[a-z]+$"))
+                throw new ConfigurationException(String.format("SSTable format name for %s must be non-empty, lower-case letters only string", factory.getClass().getName()));
 
-        for (ParameterizedClass formatConfig : configuredFormats)
+            SSTableFormat.Factory prev = factoryByName.put(factory.name(), factory);
+            if (prev != null)
+                throw new ConfigurationException(String.format("Multiple sstable format implementations with the same name %s: %s and %s", factory.name(), factory.getClass().getName(), prev.getClass().getName()));
+        }
+    }
+
+    private static ImmutableMap<String, Supplier<SSTableFormat<?, ?>>> validateAndMatchSSTableFormatOptions(Iterable<SSTableFormat.Factory> factories, Map<String, Map<String, String>> options)
+    {
+        ImmutableMap.Builder<String, Supplier<SSTableFormat<?, ?>>> providersBuilder = ImmutableMap.builder();
+        for (SSTableFormat.Factory factory : factories)
         {
-            assert formatConfig.parameters != null;
-            Map<String, String> params = new HashMap<>(formatConfig.parameters);
-
-            String name = params.get(Config.SSTABLE_FORMAT_NAME);
-            if (name == null)
-                throw new ConfigurationException("Missing 'name' parameter in sstable format configuration for " + formatConfig.class_name);
-            if (!name.matches("^[a-z]+$"))
-                throw new ConfigurationException("'name' parameter in sstable format configuration for " + formatConfig.class_name + " must be non-empty, lower-case letters only string");
-            if (names.contains(name))
-                throw new ConfigurationException("Name '" + name + "' of sstable format " + formatConfig.class_name + " is already defined for another sstable format");
-            params.remove(Config.SSTABLE_FORMAT_NAME);
-
-            String idString = params.get(Config.SSTABLE_FORMAT_ID);
-            if (idString == null)
-                throw new ConfigurationException("Missing 'id' parameter in sstable format configuration for " + formatConfig.class_name);
-            int id;
-            try
-            {
-                id = Integer.parseInt(idString);
-            }
-            catch (RuntimeException ex)
-            {
-                throw new ConfigurationException("'id' parameter in sstable format configuration for " + formatConfig.class_name + " must be an integer");
-            }
-            if (id < 0 || id > 127)
-                throw new ConfigurationException("'id' parameter in sstable format configuration for " + formatConfig.class_name + " must be within bounds [0..127] range");
-            if (ids.contains(id))
-                throw new ConfigurationException("ID '" + id + "' of sstable format " + formatConfig.class_name + " is already defined for another sstable format");
-            params.remove(Config.SSTABLE_FORMAT_ID);
-
-            Supplier<SSTableFormat<?, ?>> factory = () -> {
-                Class<SSTableFormat<?, ?>> cls = FBUtilities.classForName(formatConfig.class_name, "sstable format");
-                if (!SSTableFormat.class.isAssignableFrom(cls))
-                    throw new ConfigurationException(String.format("Class %s for sstable format %s does not implement %s", formatConfig.class_name, name, SSTableFormat.class.getName()));
-
-                SSTableFormat<?, ?> sstableFormat = FBUtilities.instanceOrConstruct(cls.getName(), "sstable format");
-                sstableFormat.setup(id, name, params);
-                return sstableFormat;
-            };
-            sstableFormatFactories.put(name, factory);
-            names.add(name);
-            ids.add(id);
+            Map<String, String> formatOptions = options != null ? ImmutableMap.copyOf(options.getOrDefault(factory.name(), ImmutableMap.of())) : ImmutableMap.of();
+            providersBuilder.put(factory.name(), () -> factory.getInstance(formatOptions));
         }
+        ImmutableMap<String, Supplier<SSTableFormat<?, ?>>> providers = providersBuilder.build();
+        if (options != null)
+        {
+            Sets.SetView<String> unknownFormatNames = Sets.difference(options.keySet(), providers.keySet());
+            if (!unknownFormatNames.isEmpty())
+                throw new ConfigurationException(String.format("Configuration contains options of unknown sstable formats: %s", unknownFormatNames));
+        }
+        return providers;
+    }
 
-        return sstableFormatFactories.build();
+    private static Pair<SSTableFormat<?, ?>, Version> getAndValidateFormatAndVersion(Map<String, SSTableFormat<?, ?>> sstableFormats, String selectedFormatName, String selectedVersionStr)
+    {
+        SSTableFormat<?, ?> selectedFormat;
+        if (StringUtils.isBlank(selectedFormatName))
+            selectedFormatName = BigFormat.NAME;
+        selectedFormat = sstableFormats.get(selectedFormatName);
+        if (selectedFormat == null)
+            throw new ConfigurationException(String.format("Selected sstable format '%s' is not available.", selectedFormatName));
+
+        Version selectedVersion;
+        if (StringUtils.isBlank(selectedVersionStr))
+            selectedVersion = selectedFormat.getLatestVersion();
+        else
+            selectedVersion = selectedFormat.getVersion(selectedVersionStr);
+        if (selectedVersion == null)
+            throw new ConfigurationException(String.format("Selected sstable version '%s' is not available for the selected sstable format '%s'", selectedVersionStr, selectedFormat.name()));
+        if (!selectedVersion.isCompatible())
+            throw new ConfigurationException(String.format("Unsupported version '%s' for the selected sstable format '%s'", selectedVersion, selectedFormat.name()));
+
+        return Pair.create(selectedFormat, selectedVersion);
     }
 
     private static void applySSTableFormats()
     {
-        if (sstableFormatFactories != null)
-            logger.warn("Reinitializing SSTableFactories - this should happen only in tests");
+        ServiceLoader<SSTableFormat.Factory> loader = ServiceLoader.load(SSTableFormat.Factory.class, DatabaseDescriptor.class.getClassLoader());
+        List<SSTableFormat.Factory> factories = Iterables.toList(loader);
+        if (factories.isEmpty())
+            applySSTableFormats(ImmutableList.of(new BigFormat.BigFormatFactory()), conf.sstable_formats);
+        else
+            applySSTableFormats(factories, conf.sstable_formats);

Review Comment:
   ```suggestion
           if (factories.isEmpty())
               factories = ImmutableList.of(new BigFormat.BigFormatFactory());
           applySSTableFormats(factories, conf.sstable_formats);
   ```



##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -1349,68 +1359,97 @@ public static void applyPartitioner(Config conf)
         paritionerName = partitioner.getClass().getCanonicalName();
     }
 
-    @VisibleForTesting
-    public static Map<String, Supplier<SSTableFormat<?, ?>>> loadSSTableFormatFactories(List<ParameterizedClass> configuredFormats)
+    private static void validateSSTableFormatFactories(Iterable<SSTableFormat.Factory> factories)
     {
-        ImmutableMap.Builder<String, Supplier<SSTableFormat<?, ?>>> sstableFormatFactories = ImmutableMap.builderWithExpectedSize(configuredFormats.size());
-        Set<String> names = new HashSet<>(configuredFormats.size());
-        Set<Integer> ids = new HashSet<>(configuredFormats.size());
+        Map<String, SSTableFormat.Factory> factoryByName = new HashMap<>();
+        for (SSTableFormat.Factory factory : factories)
+        {
+            if (factory.name() == null)
+                throw new ConfigurationException(String.format("SSTable format name in %s cannot be null", factory.getClass().getName()));
+
+            if (!factory.name().matches("^[a-z]+$"))
+                throw new ConfigurationException(String.format("SSTable format name for %s must be non-empty, lower-case letters only string", factory.getClass().getName()));

Review Comment:
   ```suggestion
                   throw new ConfigurationException(String.format("SSTable format name for %s must be non-empty, lower-case letters only string", factory.getClass().getCanonicalName()));
   ```



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java:
##########
@@ -54,9 +50,14 @@
  */
 public interface SSTableFormat<R extends SSTableReader, W extends SSTableWriter>
 {
-    int ordinal();
     String name();
 
+    @Deprecated
+    default Type getType()

Review Comment:
   only used for tests, and is redundant as `name` and `this.getClass()` is known by holding reference to this



##########
test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java:
##########
@@ -198,10 +200,11 @@
     "org.apache.cassandra.io.sstable.MetricsProviders",
     "org.apache.cassandra.io.sstable.SSTable",
     "org.apache.cassandra.io.sstable.SSTable$Builder",
-    "org.apache.cassandra.io.sstable.format.AbstractSSTableFormat",
     "org.apache.cassandra.io.sstable.format.SSTableFormat",
     "org.apache.cassandra.io.sstable.format.SSTableFormat$Components",
     "org.apache.cassandra.io.sstable.format.SSTableFormat$Components$Types",
+    "org.apache.cassandra.io.sstable.format.SSTableFormat$Factory",
+    "org.apache.cassandra.io.sstable.format.SSTableFormat$Factory",

Review Comment:
   duplicate



##########
test/unit/org/apache/cassandra/schema/MockSchema.java:
##########
@@ -170,7 +170,7 @@ public static SSTableReader sstable(int generation, int size, boolean keepRef, l
                                                sstableId(generation),
                                                format.getType());
 
-        if (format == BigFormat.getInstance())
+        if (format instanceof BigFormat)

Review Comment:
   I am ok with this, but the rest of the patch uses `org.apache.cassandra.io.sstable.format.big.BigFormat#is`, which does a `name` check



##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -1349,68 +1359,97 @@ public static void applyPartitioner(Config conf)
         paritionerName = partitioner.getClass().getCanonicalName();
     }
 
-    @VisibleForTesting
-    public static Map<String, Supplier<SSTableFormat<?, ?>>> loadSSTableFormatFactories(List<ParameterizedClass> configuredFormats)
+    private static void validateSSTableFormatFactories(Iterable<SSTableFormat.Factory> factories)
     {
-        ImmutableMap.Builder<String, Supplier<SSTableFormat<?, ?>>> sstableFormatFactories = ImmutableMap.builderWithExpectedSize(configuredFormats.size());
-        Set<String> names = new HashSet<>(configuredFormats.size());
-        Set<Integer> ids = new HashSet<>(configuredFormats.size());
+        Map<String, SSTableFormat.Factory> factoryByName = new HashMap<>();
+        for (SSTableFormat.Factory factory : factories)
+        {
+            if (factory.name() == null)
+                throw new ConfigurationException(String.format("SSTable format name in %s cannot be null", factory.getClass().getName()));

Review Comment:
   ```suggestion
                   throw new ConfigurationException(String.format("SSTable format name in %s cannot be null", factory.getClass().getCanonicalName()));
   ```



##########
src/java/org/apache/cassandra/config/YamlConfigurationLoader.java:
##########
@@ -203,6 +202,8 @@ private static void verifyReplacements(Map<Class<?>, Map<String, Replacement>> r
         Yaml rawYaml = new Yaml(loaderOptions);
 
         Map<String, Object> rawConfig = rawYaml.load(new ByteArrayInputStream(configBytes));
+        if (rawConfig == null)

Review Comment:
   What case caused this?



##########
src/java/org/apache/cassandra/config/Config.java:
##########
@@ -359,9 +352,14 @@ public MemtableOptions()
 
     public String[] data_file_directories = new String[0];
 
-    public List<ParameterizedClass> sstable_formats = ImmutableList.of(new ParameterizedClass(BigFormat.class.getName(),// "org.apache.cassandra.io.sstable.format.big.BigFormat",
-                                                                                              ImmutableMap.of(SSTABLE_FORMAT_ID, "0",
-                                                                                                              SSTABLE_FORMAT_NAME, "big")));
+    public static class SSTableFormats
+    {
+        public String selected_format;
+        public String selected_version;

Review Comment:
   I honestly keep debating this personally... should this be in `options`?



##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -1349,68 +1359,97 @@ public static void applyPartitioner(Config conf)
         paritionerName = partitioner.getClass().getCanonicalName();
     }
 
-    @VisibleForTesting
-    public static Map<String, Supplier<SSTableFormat<?, ?>>> loadSSTableFormatFactories(List<ParameterizedClass> configuredFormats)
+    private static void validateSSTableFormatFactories(Iterable<SSTableFormat.Factory> factories)
     {
-        ImmutableMap.Builder<String, Supplier<SSTableFormat<?, ?>>> sstableFormatFactories = ImmutableMap.builderWithExpectedSize(configuredFormats.size());
-        Set<String> names = new HashSet<>(configuredFormats.size());
-        Set<Integer> ids = new HashSet<>(configuredFormats.size());
+        Map<String, SSTableFormat.Factory> factoryByName = new HashMap<>();
+        for (SSTableFormat.Factory factory : factories)
+        {
+            if (factory.name() == null)
+                throw new ConfigurationException(String.format("SSTable format name in %s cannot be null", factory.getClass().getName()));
+
+            if (!factory.name().matches("^[a-z]+$"))
+                throw new ConfigurationException(String.format("SSTable format name for %s must be non-empty, lower-case letters only string", factory.getClass().getName()));
 
-        for (ParameterizedClass formatConfig : configuredFormats)
+            SSTableFormat.Factory prev = factoryByName.put(factory.name(), factory);
+            if (prev != null)
+                throw new ConfigurationException(String.format("Multiple sstable format implementations with the same name %s: %s and %s", factory.name(), factory.getClass().getName(), prev.getClass().getName()));

Review Comment:
   ```suggestion
                   throw new ConfigurationException(String.format("Multiple sstable format implementations with the same name %s: %s and %s", factory.name(), factory.getClass().getCanonicalName(), prev.getClass().getCanonicalName()));
   ```



##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -1349,68 +1359,97 @@ public static void applyPartitioner(Config conf)
         paritionerName = partitioner.getClass().getCanonicalName();
     }
 
-    @VisibleForTesting
-    public static Map<String, Supplier<SSTableFormat<?, ?>>> loadSSTableFormatFactories(List<ParameterizedClass> configuredFormats)
+    private static void validateSSTableFormatFactories(Iterable<SSTableFormat.Factory> factories)
     {
-        ImmutableMap.Builder<String, Supplier<SSTableFormat<?, ?>>> sstableFormatFactories = ImmutableMap.builderWithExpectedSize(configuredFormats.size());
-        Set<String> names = new HashSet<>(configuredFormats.size());
-        Set<Integer> ids = new HashSet<>(configuredFormats.size());
+        Map<String, SSTableFormat.Factory> factoryByName = new HashMap<>();
+        for (SSTableFormat.Factory factory : factories)
+        {
+            if (factory.name() == null)
+                throw new ConfigurationException(String.format("SSTable format name in %s cannot be null", factory.getClass().getName()));
+
+            if (!factory.name().matches("^[a-z]+$"))
+                throw new ConfigurationException(String.format("SSTable format name for %s must be non-empty, lower-case letters only string", factory.getClass().getName()));
 
-        for (ParameterizedClass formatConfig : configuredFormats)
+            SSTableFormat.Factory prev = factoryByName.put(factory.name(), factory);
+            if (prev != null)
+                throw new ConfigurationException(String.format("Multiple sstable format implementations with the same name %s: %s and %s", factory.name(), factory.getClass().getName(), prev.getClass().getName()));
+        }
+    }
+
+    private static ImmutableMap<String, Supplier<SSTableFormat<?, ?>>> validateAndMatchSSTableFormatOptions(Iterable<SSTableFormat.Factory> factories, Map<String, Map<String, String>> options)
+    {
+        ImmutableMap.Builder<String, Supplier<SSTableFormat<?, ?>>> providersBuilder = ImmutableMap.builder();
+        for (SSTableFormat.Factory factory : factories)
         {
-            assert formatConfig.parameters != null;
-            Map<String, String> params = new HashMap<>(formatConfig.parameters);
-
-            String name = params.get(Config.SSTABLE_FORMAT_NAME);
-            if (name == null)
-                throw new ConfigurationException("Missing 'name' parameter in sstable format configuration for " + formatConfig.class_name);
-            if (!name.matches("^[a-z]+$"))
-                throw new ConfigurationException("'name' parameter in sstable format configuration for " + formatConfig.class_name + " must be non-empty, lower-case letters only string");
-            if (names.contains(name))
-                throw new ConfigurationException("Name '" + name + "' of sstable format " + formatConfig.class_name + " is already defined for another sstable format");
-            params.remove(Config.SSTABLE_FORMAT_NAME);
-
-            String idString = params.get(Config.SSTABLE_FORMAT_ID);
-            if (idString == null)
-                throw new ConfigurationException("Missing 'id' parameter in sstable format configuration for " + formatConfig.class_name);
-            int id;
-            try
-            {
-                id = Integer.parseInt(idString);
-            }
-            catch (RuntimeException ex)
-            {
-                throw new ConfigurationException("'id' parameter in sstable format configuration for " + formatConfig.class_name + " must be an integer");
-            }
-            if (id < 0 || id > 127)
-                throw new ConfigurationException("'id' parameter in sstable format configuration for " + formatConfig.class_name + " must be within bounds [0..127] range");
-            if (ids.contains(id))
-                throw new ConfigurationException("ID '" + id + "' of sstable format " + formatConfig.class_name + " is already defined for another sstable format");
-            params.remove(Config.SSTABLE_FORMAT_ID);
-
-            Supplier<SSTableFormat<?, ?>> factory = () -> {
-                Class<SSTableFormat<?, ?>> cls = FBUtilities.classForName(formatConfig.class_name, "sstable format");
-                if (!SSTableFormat.class.isAssignableFrom(cls))
-                    throw new ConfigurationException(String.format("Class %s for sstable format %s does not implement %s", formatConfig.class_name, name, SSTableFormat.class.getName()));
-
-                SSTableFormat<?, ?> sstableFormat = FBUtilities.instanceOrConstruct(cls.getName(), "sstable format");
-                sstableFormat.setup(id, name, params);
-                return sstableFormat;
-            };
-            sstableFormatFactories.put(name, factory);
-            names.add(name);
-            ids.add(id);
+            Map<String, String> formatOptions = options != null ? ImmutableMap.copyOf(options.getOrDefault(factory.name(), ImmutableMap.of())) : ImmutableMap.of();

Review Comment:
   rather than checking `options` in the loop you can do the following outside the loop
   
   ```
   if (options == null)
     options = ImmutableMap.of();
   ```



##########
test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java:
##########
@@ -217,6 +220,8 @@
     "org.apache.cassandra.io.sstable.format.SortedTableWriter$Builder",
     "org.apache.cassandra.io.sstable.format.Version",
     "org.apache.cassandra.io.sstable.format.big.BigFormat",
+    "org.apache.cassandra.io.sstable.format.big.BigFormat$BigFormatFactory",
+    "org.apache.cassandra.io.sstable.format.big.BigFormat$BigFormatFactory",

Review Comment:
   duplicate



##########
src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java:
##########
@@ -113,99 +114,85 @@ IScrubber getScrubber(ColumnFamilyStore cfs,
 
     void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components);
 
-    void setup(int id, String name, Map<String, String> options);
-
-    Type getType();
-
     /**
      * Deletes the existing components of the sstables represented by the provided descriptor.
      * The method is also responsible for cleaning up the in-memory resources occupied by the stuff related to that
      * sstables, such as row key cache entries.
      */
     void delete(Descriptor descriptor);
 
+    /**
+     * This class is not completely redundant

Review Comment:
   I do feel that this is redundant, its just another reference to the values in `org.apache.cassandra.config.DatabaseDescriptor#sstableFormats` and 100% depends on that...
   
   `current` -> `DD.getSelectedSSTableFormat`
   `getByName` -> `DD.sstableFormats.get(name)`
   `getByClass` -> 
   
   ```
   DatabaseDescriptor.getSSTableFormats().values().stream()
                                 .filter(f -> f.getClass().equals(formatClass))
                                 .findFirst()
                                 .orElseGet(() -> {
                                     throw new NoSuchElementException("Unknown sstable format class: " + formatClass);
                                 });
   ```



##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -1349,68 +1359,97 @@ public static void applyPartitioner(Config conf)
         paritionerName = partitioner.getClass().getCanonicalName();
     }
 
-    @VisibleForTesting
-    public static Map<String, Supplier<SSTableFormat<?, ?>>> loadSSTableFormatFactories(List<ParameterizedClass> configuredFormats)
+    private static void validateSSTableFormatFactories(Iterable<SSTableFormat.Factory> factories)
     {
-        ImmutableMap.Builder<String, Supplier<SSTableFormat<?, ?>>> sstableFormatFactories = ImmutableMap.builderWithExpectedSize(configuredFormats.size());
-        Set<String> names = new HashSet<>(configuredFormats.size());
-        Set<Integer> ids = new HashSet<>(configuredFormats.size());
+        Map<String, SSTableFormat.Factory> factoryByName = new HashMap<>();
+        for (SSTableFormat.Factory factory : factories)
+        {
+            if (factory.name() == null)
+                throw new ConfigurationException(String.format("SSTable format name in %s cannot be null", factory.getClass().getName()));
+
+            if (!factory.name().matches("^[a-z]+$"))
+                throw new ConfigurationException(String.format("SSTable format name for %s must be non-empty, lower-case letters only string", factory.getClass().getName()));
 
-        for (ParameterizedClass formatConfig : configuredFormats)
+            SSTableFormat.Factory prev = factoryByName.put(factory.name(), factory);
+            if (prev != null)
+                throw new ConfigurationException(String.format("Multiple sstable format implementations with the same name %s: %s and %s", factory.name(), factory.getClass().getName(), prev.getClass().getName()));
+        }
+    }
+
+    private static ImmutableMap<String, Supplier<SSTableFormat<?, ?>>> validateAndMatchSSTableFormatOptions(Iterable<SSTableFormat.Factory> factories, Map<String, Map<String, String>> options)
+    {
+        ImmutableMap.Builder<String, Supplier<SSTableFormat<?, ?>>> providersBuilder = ImmutableMap.builder();
+        for (SSTableFormat.Factory factory : factories)
         {
-            assert formatConfig.parameters != null;
-            Map<String, String> params = new HashMap<>(formatConfig.parameters);
-
-            String name = params.get(Config.SSTABLE_FORMAT_NAME);
-            if (name == null)
-                throw new ConfigurationException("Missing 'name' parameter in sstable format configuration for " + formatConfig.class_name);
-            if (!name.matches("^[a-z]+$"))
-                throw new ConfigurationException("'name' parameter in sstable format configuration for " + formatConfig.class_name + " must be non-empty, lower-case letters only string");
-            if (names.contains(name))
-                throw new ConfigurationException("Name '" + name + "' of sstable format " + formatConfig.class_name + " is already defined for another sstable format");
-            params.remove(Config.SSTABLE_FORMAT_NAME);
-
-            String idString = params.get(Config.SSTABLE_FORMAT_ID);
-            if (idString == null)
-                throw new ConfigurationException("Missing 'id' parameter in sstable format configuration for " + formatConfig.class_name);
-            int id;
-            try
-            {
-                id = Integer.parseInt(idString);
-            }
-            catch (RuntimeException ex)
-            {
-                throw new ConfigurationException("'id' parameter in sstable format configuration for " + formatConfig.class_name + " must be an integer");
-            }
-            if (id < 0 || id > 127)
-                throw new ConfigurationException("'id' parameter in sstable format configuration for " + formatConfig.class_name + " must be within bounds [0..127] range");
-            if (ids.contains(id))
-                throw new ConfigurationException("ID '" + id + "' of sstable format " + formatConfig.class_name + " is already defined for another sstable format");
-            params.remove(Config.SSTABLE_FORMAT_ID);
-
-            Supplier<SSTableFormat<?, ?>> factory = () -> {
-                Class<SSTableFormat<?, ?>> cls = FBUtilities.classForName(formatConfig.class_name, "sstable format");
-                if (!SSTableFormat.class.isAssignableFrom(cls))
-                    throw new ConfigurationException(String.format("Class %s for sstable format %s does not implement %s", formatConfig.class_name, name, SSTableFormat.class.getName()));
-
-                SSTableFormat<?, ?> sstableFormat = FBUtilities.instanceOrConstruct(cls.getName(), "sstable format");
-                sstableFormat.setup(id, name, params);
-                return sstableFormat;
-            };
-            sstableFormatFactories.put(name, factory);
-            names.add(name);
-            ids.add(id);
+            Map<String, String> formatOptions = options != null ? ImmutableMap.copyOf(options.getOrDefault(factory.name(), ImmutableMap.of())) : ImmutableMap.of();
+            providersBuilder.put(factory.name(), () -> factory.getInstance(formatOptions));
         }
+        ImmutableMap<String, Supplier<SSTableFormat<?, ?>>> providers = providersBuilder.build();
+        if (options != null)
+        {
+            Sets.SetView<String> unknownFormatNames = Sets.difference(options.keySet(), providers.keySet());
+            if (!unknownFormatNames.isEmpty())
+                throw new ConfigurationException(String.format("Configuration contains options of unknown sstable formats: %s", unknownFormatNames));
+        }
+        return providers;
+    }
 
-        return sstableFormatFactories.build();
+    private static Pair<SSTableFormat<?, ?>, Version> getAndValidateFormatAndVersion(Map<String, SSTableFormat<?, ?>> sstableFormats, String selectedFormatName, String selectedVersionStr)
+    {
+        SSTableFormat<?, ?> selectedFormat;
+        if (StringUtils.isBlank(selectedFormatName))
+            selectedFormatName = BigFormat.NAME;

Review Comment:
   I prefer this to live in `Config` so defaults are more clear and not scattered all over the place



-- 
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] dcapwell commented on a diff in pull request #2295: Cassandra 18441

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


##########
src/java/org/apache/cassandra/cache/AutoSavingCache.java:
##########
@@ -196,12 +205,18 @@ public int loadSaved()
         // modern format, allows both key and value (so key cache load can be purely sequential)
         File dataPath = getCacheDataPath(CURRENT_VERSION);
         File crcPath = getCacheCrcPath(CURRENT_VERSION);
-        if (dataPath.exists() && crcPath.exists())
+        File metadataPath = getCacheMetadataPath(CURRENT_VERSION);
+        if (dataPath.exists() && crcPath.exists() && metadataPath.exists())

Review Comment:
   unrelated to this patch, sad that upgrading from 4.1 -> 5.0 ignores the cache as we don't support reading older versions.



-- 
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] dcapwell commented on a diff in pull request #2295: Cassandra 18441

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


##########
src/java/org/apache/cassandra/db/lifecycle/LogFile.java:
##########
@@ -504,7 +507,7 @@ List<String> getFilePaths()
 
     private String getFileName()
     {
-        return StringUtils.join(SSTableFormat.Type.current().info.getLatestVersion(), LogFile.SEP, // remove version and separator when downgrading to 4.x is becomes unsupported
+        return StringUtils.join(version, LogFile.SEP, // remove version and separator when downgrading to 4.x is becomes unsupported TODO should this include format type as well?

Review Comment:
   > TODO should this include format type as well?
   
   if the version does not include the format, I think it would be good... `nc` isn't useful as I don't know this is `big` or `bti`
   
   Another option is to use `version.toFormatAndVersionString`?



##########
conf/cassandra.yaml:
##########
@@ -1928,14 +1928,5 @@ drop_compact_storage_enabled: false
 #    excluded_keyspaces: # comma separated list of keyspaces to exclude from the check
 #    excluded_tables: # comma separated list of keyspace.table pairs to exclude from the check
 
-# Supported sstable formats
-# This is a list of elements consisting of class_name and parameters, where class_name should point to the class
-# implementing org.apache.cassandra.io.sstable.format.SSTableFormat. Parameters must include unique 'id' integer
-# which is used in some serialization to denote the format type in a compact way (such as local key cache); and 'name'
-# which will be used to recognize the format type - in particular that name will be used in sstable file names and in
-# stream headers so the name has to be the same for the same format across all the nodes in the cluster.
-sstable_formats:
-  - class_name: org.apache.cassandra.io.sstable.format.big.BigFormat
-    parameters:
-      id: 0
-      name: big
+#sstable_formats:

Review Comment:
   can you update the yaml to match the `Config` 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] dcapwell commented on a diff in pull request #2295: Cassandra 18441

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


##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -1349,68 +1359,98 @@ public static void applyPartitioner(Config conf)
         paritionerName = partitioner.getClass().getCanonicalName();
     }
 
-    @VisibleForTesting
-    public static Map<String, Supplier<SSTableFormat<?, ?>>> loadSSTableFormatFactories(List<ParameterizedClass> configuredFormats)
+    private static void validateSSTableFormatFactories(Iterable<SSTableFormat.Factory> factories)
     {
-        ImmutableMap.Builder<String, Supplier<SSTableFormat<?, ?>>> sstableFormatFactories = ImmutableMap.builderWithExpectedSize(configuredFormats.size());
-        Set<String> names = new HashSet<>(configuredFormats.size());
-        Set<Integer> ids = new HashSet<>(configuredFormats.size());
+        Map<String, SSTableFormat.Factory> factoryByName = new HashMap<>();
+        for (SSTableFormat.Factory factory : factories)
+        {
+            if (factory.name() == null)
+                throw new ConfigurationException(String.format("SSTable format name in %s cannot be null", factory.getClass().getCanonicalName()));
+
+            if (!factory.name().matches("^[a-z]+$"))
+                throw new ConfigurationException(String.format("SSTable format name for %s must be non-empty, lower-case letters only string", factory.getClass().getCanonicalName()));
 
-        for (ParameterizedClass formatConfig : configuredFormats)
+            SSTableFormat.Factory prev = factoryByName.put(factory.name(), factory);
+            if (prev != null)
+                throw new ConfigurationException(String.format("Multiple sstable format implementations with the same name %s: %s and %s", factory.name(), factory.getClass().getCanonicalName(), prev.getClass().getCanonicalName()));
+        }
+    }
+
+    private static ImmutableMap<String, Supplier<SSTableFormat<?, ?>>> validateAndMatchSSTableFormatOptions(Iterable<SSTableFormat.Factory> factories, Map<String, Map<String, String>> options)
+    {
+        ImmutableMap.Builder<String, Supplier<SSTableFormat<?, ?>>> providersBuilder = ImmutableMap.builder();
+        if (options == null)
+            options = ImmutableMap.of();
+        for (SSTableFormat.Factory factory : factories)
         {
-            assert formatConfig.parameters != null;
-            Map<String, String> params = new HashMap<>(formatConfig.parameters);
-
-            String name = params.get(Config.SSTABLE_FORMAT_NAME);
-            if (name == null)
-                throw new ConfigurationException("Missing 'name' parameter in sstable format configuration for " + formatConfig.class_name);
-            if (!name.matches("^[a-z]+$"))
-                throw new ConfigurationException("'name' parameter in sstable format configuration for " + formatConfig.class_name + " must be non-empty, lower-case letters only string");
-            if (names.contains(name))
-                throw new ConfigurationException("Name '" + name + "' of sstable format " + formatConfig.class_name + " is already defined for another sstable format");
-            params.remove(Config.SSTABLE_FORMAT_NAME);
-
-            String idString = params.get(Config.SSTABLE_FORMAT_ID);
-            if (idString == null)
-                throw new ConfigurationException("Missing 'id' parameter in sstable format configuration for " + formatConfig.class_name);
-            int id;
-            try
-            {
-                id = Integer.parseInt(idString);
-            }
-            catch (RuntimeException ex)
-            {
-                throw new ConfigurationException("'id' parameter in sstable format configuration for " + formatConfig.class_name + " must be an integer");
-            }
-            if (id < 0 || id > 127)
-                throw new ConfigurationException("'id' parameter in sstable format configuration for " + formatConfig.class_name + " must be within bounds [0..127] range");
-            if (ids.contains(id))
-                throw new ConfigurationException("ID '" + id + "' of sstable format " + formatConfig.class_name + " is already defined for another sstable format");
-            params.remove(Config.SSTABLE_FORMAT_ID);
-
-            Supplier<SSTableFormat<?, ?>> factory = () -> {
-                Class<SSTableFormat<?, ?>> cls = FBUtilities.classForName(formatConfig.class_name, "sstable format");
-                if (!SSTableFormat.class.isAssignableFrom(cls))
-                    throw new ConfigurationException(String.format("Class %s for sstable format %s does not implement %s", formatConfig.class_name, name, SSTableFormat.class.getName()));
-
-                SSTableFormat<?, ?> sstableFormat = FBUtilities.instanceOrConstruct(cls.getName(), "sstable format");
-                sstableFormat.setup(id, name, params);
-                return sstableFormat;
-            };
-            sstableFormatFactories.put(name, factory);
-            names.add(name);
-            ids.add(id);
+            Map<String, String> formatOptions = ImmutableMap.copyOf(options.getOrDefault(factory.name(), ImmutableMap.of()));
+            providersBuilder.put(factory.name(), () -> factory.getInstance(formatOptions));
+        }
+        ImmutableMap<String, Supplier<SSTableFormat<?, ?>>> providers = providersBuilder.build();
+        if (options != null)

Review Comment:
   not possible after latest 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] dcapwell commented on a diff in pull request #2295: Cassandra 18441

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


##########
src/java/org/apache/cassandra/config/Config.java:
##########
@@ -359,9 +352,14 @@ public MemtableOptions()
 
     public String[] data_file_directories = new String[0];
 
-    public List<ParameterizedClass> sstable_formats = ImmutableList.of(new ParameterizedClass(BigFormat.class.getName(),// "org.apache.cassandra.io.sstable.format.big.BigFormat",
-                                                                                              ImmutableMap.of(SSTABLE_FORMAT_ID, "0",
-                                                                                                              SSTABLE_FORMAT_NAME, "big")));
+    public static class SSTableFormats
+    {
+        public String selected_format;
+        public String selected_version;

Review Comment:
   > this should always be changed together
   
   agree, but we would need locking or copy-on-write configs to support that, which we don't have in this patch (and not arguing we add).  This means a config change happens each time a field is modified, which could put us in a inconsistence state waiting for the operator to update the other fields.
   
   So, by having the version in `options` this at least scopes things to where they make sense... if you are using `big` and want to test out `bti v42` you can update the configs for `bti.version=42`, this would have no runtime impact as its untouched.  But let's say you are using `bti v41` and want to switch to `v42`?  you then do `bit.version=42` and the next sstable uses `v42`.  By having the version in the options it makes the concurrency easier to reason 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 pull request #2295: Cassandra 18441

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

   Thank you @dcapwell for quick feedback, really appreciate 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 #2295: Cassandra 18441

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


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java:
##########
@@ -113,99 +114,85 @@ IScrubber getScrubber(ColumnFamilyStore cfs,
 
     void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components);
 
-    void setup(int id, String name, Map<String, String> options);
-
-    Type getType();
-
     /**
      * Deletes the existing components of the sstables represented by the provided descriptor.
      * The method is also responsible for cleaning up the in-memory resources occupied by the stuff related to that
      * sstables, such as row key cache entries.
      */
     void delete(Descriptor descriptor);
 
+    /**
+     * This class is not completely redundant

Review Comment:
   Did I really write "not"??? 🤦🏻‍♂️ I wanted to write it is redundant and that's why I marked it deprecated. That "not" is a typing mistake.
   
   The only reason I haven't removed it is that the additional changeset would be quite large. So if we agree on that, we can do 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 #2295: Cassandra 18441

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


##########
src/java/org/apache/cassandra/config/YamlConfigurationLoader.java:
##########
@@ -203,6 +202,8 @@ private static void verifyReplacements(Map<Class<?>, Map<String, Replacement>> r
         Yaml rawYaml = new Yaml(loaderOptions);
 
         Map<String, Object> rawConfig = rawYaml.load(new ByteArrayInputStream(configBytes));
+        if (rawConfig == null)

Review Comment:
   Just for testing - wanted to make it possible to pass an empty yaml to assert the defaults. I think this is a valid situation - whole yaml should be optional.



-- 
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 #2295: Cassandra 18441

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


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java:
##########
@@ -54,9 +50,14 @@
  */
 public interface SSTableFormat<R extends SSTableReader, W extends SSTableWriter>
 {
-    int ordinal();
     String name();
 
+    @Deprecated
+    default Type getType()

Review Comment:
   yes, that's why it is marked as deprecated and left only to not pollute the pull request.



-- 
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 #2295: Cassandra 18441

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


##########
src/java/org/apache/cassandra/config/Config.java:
##########
@@ -359,9 +352,14 @@ public MemtableOptions()
 
     public String[] data_file_directories = new String[0];
 
-    public List<ParameterizedClass> sstable_formats = ImmutableList.of(new ParameterizedClass(BigFormat.class.getName(),// "org.apache.cassandra.io.sstable.format.big.BigFormat",
-                                                                                              ImmutableMap.of(SSTABLE_FORMAT_ID, "0",
-                                                                                                              SSTABLE_FORMAT_NAME, "big")));
+    public static class SSTableFormats
+    {
+        public String selected_format;
+        public String selected_version;

Review Comment:
   agreed; though I was also thinking about specifying the write format like:
   
   ```
   write_format: big
   ```
   or
   ```
   write_format: big-nc
   ```
   
   Though, as discussed in Jira, we will define a version per 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] dcapwell commented on a diff in pull request #2295: Cassandra 18441

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


##########
src/java/org/apache/cassandra/config/Config.java:
##########
@@ -359,9 +352,14 @@ public MemtableOptions()
 
     public String[] data_file_directories = new String[0];
 
-    public List<ParameterizedClass> sstable_formats = ImmutableList.of(new ParameterizedClass(BigFormat.class.getName(),// "org.apache.cassandra.io.sstable.format.big.BigFormat",
-                                                                                              ImmutableMap.of(SSTABLE_FORMAT_ID, "0",
-                                                                                                              SSTABLE_FORMAT_NAME, "big")));
+    public static class SSTableFormats
+    {
+        public String selected_format;
+        public String selected_version;

Review Comment:
   my thinking is, if we do want to later (not this patch) allow overriding via JMX or Settings table, how can you do thing safely?
   
   Lets say we have the following cases
   
   1) keep same format, just change version (use case; major/minor upgrade wants to delay using latest until "stable")
   
   ```
   sstable_formats.selected_version = next // this is "safe"
   ```
   
   2) switch from big to bti or some 3rd party format
   
   ```
   sstable_formats.selected_version = bti_version // unsafe, breaks BIG
   sstable_formats.selected_format = bti
   ```
   
   or
   
   ```
   sstable_formats.selected_format = bti // unsafe as "selected_version" still points to BIG version
   sstable_formats.selected_version = bti_version
   ```



-- 
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 #2295: Cassandra 18441

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


##########
src/java/org/apache/cassandra/config/Config.java:
##########
@@ -359,9 +352,14 @@ public MemtableOptions()
 
     public String[] data_file_directories = new String[0];
 
-    public List<ParameterizedClass> sstable_formats = ImmutableList.of(new ParameterizedClass(BigFormat.class.getName(),// "org.apache.cassandra.io.sstable.format.big.BigFormat",
-                                                                                              ImmutableMap.of(SSTABLE_FORMAT_ID, "0",
-                                                                                                              SSTABLE_FORMAT_NAME, "big")));
+    public static class SSTableFormats
+    {
+        public String selected_format;
+        public String selected_version;

Review Comment:
   Where would we express the expected target format and version then? If we ever go for changing that at runtime, this should always be changed together. 
   



-- 
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] dcapwell commented on a diff in pull request #2295: Cassandra 18441

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


##########
src/java/org/apache/cassandra/cache/AutoSavingCache.java:
##########
@@ -196,12 +205,18 @@ public int loadSaved()
         // modern format, allows both key and value (so key cache load can be purely sequential)
         File dataPath = getCacheDataPath(CURRENT_VERSION);
         File crcPath = getCacheCrcPath(CURRENT_VERSION);
-        if (dataPath.exists() && crcPath.exists())
+        File metadataPath = getCacheMetadataPath(CURRENT_VERSION);
+        if (dataPath.exists() && crcPath.exists() && metadataPath.exists())

Review Comment:
   Given that this change already landed, I don't mind this being punted from this PR, but would be good to ask in dev@?  This would be a performance issue during major upgrades but once warm you stop seeing it...



-- 
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 #2295: Cassandra 18441

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


##########
src/java/org/apache/cassandra/cache/AutoSavingCache.java:
##########
@@ -196,12 +205,18 @@ public int loadSaved()
         // modern format, allows both key and value (so key cache load can be purely sequential)
         File dataPath = getCacheDataPath(CURRENT_VERSION);
         File crcPath = getCacheCrcPath(CURRENT_VERSION);
-        if (dataPath.exists() && crcPath.exists())
+        File metadataPath = getCacheMetadataPath(CURRENT_VERSION);
+        if (dataPath.exists() && crcPath.exists() && metadataPath.exists())

Review Comment:
   This is doable, but I don't know if complicating this code is worth restoring the cache from previous version 🤷🏻‍♂️ 



-- 
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 #2295: Cassandra 18441

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


##########
test/unit/org/apache/cassandra/schema/MockSchema.java:
##########
@@ -170,7 +170,7 @@ public static SSTableReader sstable(int generation, int size, boolean keepRef, l
                                                sstableId(generation),
                                                format.getType());
 
-        if (format == BigFormat.getInstance())
+        if (format instanceof BigFormat)

Review Comment:
   fixed



-- 
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] dcapwell commented on a diff in pull request #2295: Cassandra 18441

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


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java:
##########
@@ -113,99 +114,85 @@ IScrubber getScrubber(ColumnFamilyStore cfs,
 
     void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components);
 
-    void setup(int id, String name, Map<String, String> options);
-
-    Type getType();
-
     /**
      * Deletes the existing components of the sstables represented by the provided descriptor.
      * The method is also responsible for cleaning up the in-memory resources occupied by the stuff related to that
      * sstables, such as row key cache entries.
      */
     void delete(Descriptor descriptor);
 
+    /**
+     * This class is not completely redundant

Review Comment:
   I am cool with you leaving this until the review is complete, then fixing at the end.  



-- 
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 #2295: Cassandra 18441

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


##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -1349,68 +1359,97 @@ public static void applyPartitioner(Config conf)
         paritionerName = partitioner.getClass().getCanonicalName();
     }
 
-    @VisibleForTesting
-    public static Map<String, Supplier<SSTableFormat<?, ?>>> loadSSTableFormatFactories(List<ParameterizedClass> configuredFormats)
+    private static void validateSSTableFormatFactories(Iterable<SSTableFormat.Factory> factories)
     {
-        ImmutableMap.Builder<String, Supplier<SSTableFormat<?, ?>>> sstableFormatFactories = ImmutableMap.builderWithExpectedSize(configuredFormats.size());
-        Set<String> names = new HashSet<>(configuredFormats.size());
-        Set<Integer> ids = new HashSet<>(configuredFormats.size());
+        Map<String, SSTableFormat.Factory> factoryByName = new HashMap<>();
+        for (SSTableFormat.Factory factory : factories)
+        {
+            if (factory.name() == null)
+                throw new ConfigurationException(String.format("SSTable format name in %s cannot be null", factory.getClass().getName()));
+
+            if (!factory.name().matches("^[a-z]+$"))
+                throw new ConfigurationException(String.format("SSTable format name for %s must be non-empty, lower-case letters only string", factory.getClass().getName()));
 
-        for (ParameterizedClass formatConfig : configuredFormats)
+            SSTableFormat.Factory prev = factoryByName.put(factory.name(), factory);
+            if (prev != null)
+                throw new ConfigurationException(String.format("Multiple sstable format implementations with the same name %s: %s and %s", factory.name(), factory.getClass().getName(), prev.getClass().getName()));
+        }
+    }
+
+    private static ImmutableMap<String, Supplier<SSTableFormat<?, ?>>> validateAndMatchSSTableFormatOptions(Iterable<SSTableFormat.Factory> factories, Map<String, Map<String, String>> options)
+    {
+        ImmutableMap.Builder<String, Supplier<SSTableFormat<?, ?>>> providersBuilder = ImmutableMap.builder();
+        for (SSTableFormat.Factory factory : factories)
         {
-            assert formatConfig.parameters != null;
-            Map<String, String> params = new HashMap<>(formatConfig.parameters);
-
-            String name = params.get(Config.SSTABLE_FORMAT_NAME);
-            if (name == null)
-                throw new ConfigurationException("Missing 'name' parameter in sstable format configuration for " + formatConfig.class_name);
-            if (!name.matches("^[a-z]+$"))
-                throw new ConfigurationException("'name' parameter in sstable format configuration for " + formatConfig.class_name + " must be non-empty, lower-case letters only string");
-            if (names.contains(name))
-                throw new ConfigurationException("Name '" + name + "' of sstable format " + formatConfig.class_name + " is already defined for another sstable format");
-            params.remove(Config.SSTABLE_FORMAT_NAME);
-
-            String idString = params.get(Config.SSTABLE_FORMAT_ID);
-            if (idString == null)
-                throw new ConfigurationException("Missing 'id' parameter in sstable format configuration for " + formatConfig.class_name);
-            int id;
-            try
-            {
-                id = Integer.parseInt(idString);
-            }
-            catch (RuntimeException ex)
-            {
-                throw new ConfigurationException("'id' parameter in sstable format configuration for " + formatConfig.class_name + " must be an integer");
-            }
-            if (id < 0 || id > 127)
-                throw new ConfigurationException("'id' parameter in sstable format configuration for " + formatConfig.class_name + " must be within bounds [0..127] range");
-            if (ids.contains(id))
-                throw new ConfigurationException("ID '" + id + "' of sstable format " + formatConfig.class_name + " is already defined for another sstable format");
-            params.remove(Config.SSTABLE_FORMAT_ID);
-
-            Supplier<SSTableFormat<?, ?>> factory = () -> {
-                Class<SSTableFormat<?, ?>> cls = FBUtilities.classForName(formatConfig.class_name, "sstable format");
-                if (!SSTableFormat.class.isAssignableFrom(cls))
-                    throw new ConfigurationException(String.format("Class %s for sstable format %s does not implement %s", formatConfig.class_name, name, SSTableFormat.class.getName()));
-
-                SSTableFormat<?, ?> sstableFormat = FBUtilities.instanceOrConstruct(cls.getName(), "sstable format");
-                sstableFormat.setup(id, name, params);
-                return sstableFormat;
-            };
-            sstableFormatFactories.put(name, factory);
-            names.add(name);
-            ids.add(id);
+            Map<String, String> formatOptions = options != null ? ImmutableMap.copyOf(options.getOrDefault(factory.name(), ImmutableMap.of())) : ImmutableMap.of();

Review Comment:
   fixed



-- 
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] dcapwell commented on a diff in pull request #2295: Cassandra 18441

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


##########
src/java/org/apache/cassandra/config/YamlConfigurationLoader.java:
##########
@@ -203,6 +202,8 @@ private static void verifyReplacements(Map<Class<?>, Map<String, Replacement>> r
         Yaml rawYaml = new Yaml(loaderOptions);
 
         Map<String, Object> rawConfig = rawYaml.load(new ByteArrayInputStream(configBytes));
+        if (rawConfig == null)

Review Comment:
   I am ok with it, just making sure there wasn't an existing bug lurking



-- 
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 #2295: Cassandra 18441

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


##########
test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java:
##########
@@ -198,10 +200,11 @@
     "org.apache.cassandra.io.sstable.MetricsProviders",
     "org.apache.cassandra.io.sstable.SSTable",
     "org.apache.cassandra.io.sstable.SSTable$Builder",
-    "org.apache.cassandra.io.sstable.format.AbstractSSTableFormat",
     "org.apache.cassandra.io.sstable.format.SSTableFormat",
     "org.apache.cassandra.io.sstable.format.SSTableFormat$Components",
     "org.apache.cassandra.io.sstable.format.SSTableFormat$Components$Types",
+    "org.apache.cassandra.io.sstable.format.SSTableFormat$Factory",
+    "org.apache.cassandra.io.sstable.format.SSTableFormat$Factory",

Review 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 #2295: Cassandra 18441

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


##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -1349,68 +1359,97 @@ public static void applyPartitioner(Config conf)
         paritionerName = partitioner.getClass().getCanonicalName();
     }
 
-    @VisibleForTesting
-    public static Map<String, Supplier<SSTableFormat<?, ?>>> loadSSTableFormatFactories(List<ParameterizedClass> configuredFormats)
+    private static void validateSSTableFormatFactories(Iterable<SSTableFormat.Factory> factories)
     {
-        ImmutableMap.Builder<String, Supplier<SSTableFormat<?, ?>>> sstableFormatFactories = ImmutableMap.builderWithExpectedSize(configuredFormats.size());
-        Set<String> names = new HashSet<>(configuredFormats.size());
-        Set<Integer> ids = new HashSet<>(configuredFormats.size());
+        Map<String, SSTableFormat.Factory> factoryByName = new HashMap<>();
+        for (SSTableFormat.Factory factory : factories)
+        {
+            if (factory.name() == null)
+                throw new ConfigurationException(String.format("SSTable format name in %s cannot be null", factory.getClass().getName()));
+
+            if (!factory.name().matches("^[a-z]+$"))
+                throw new ConfigurationException(String.format("SSTable format name for %s must be non-empty, lower-case letters only string", factory.getClass().getName()));
 
-        for (ParameterizedClass formatConfig : configuredFormats)
+            SSTableFormat.Factory prev = factoryByName.put(factory.name(), factory);
+            if (prev != null)
+                throw new ConfigurationException(String.format("Multiple sstable format implementations with the same name %s: %s and %s", factory.name(), factory.getClass().getName(), prev.getClass().getName()));
+        }
+    }
+
+    private static ImmutableMap<String, Supplier<SSTableFormat<?, ?>>> validateAndMatchSSTableFormatOptions(Iterable<SSTableFormat.Factory> factories, Map<String, Map<String, String>> options)
+    {
+        ImmutableMap.Builder<String, Supplier<SSTableFormat<?, ?>>> providersBuilder = ImmutableMap.builder();
+        for (SSTableFormat.Factory factory : factories)
         {
-            assert formatConfig.parameters != null;
-            Map<String, String> params = new HashMap<>(formatConfig.parameters);
-
-            String name = params.get(Config.SSTABLE_FORMAT_NAME);
-            if (name == null)
-                throw new ConfigurationException("Missing 'name' parameter in sstable format configuration for " + formatConfig.class_name);
-            if (!name.matches("^[a-z]+$"))
-                throw new ConfigurationException("'name' parameter in sstable format configuration for " + formatConfig.class_name + " must be non-empty, lower-case letters only string");
-            if (names.contains(name))
-                throw new ConfigurationException("Name '" + name + "' of sstable format " + formatConfig.class_name + " is already defined for another sstable format");
-            params.remove(Config.SSTABLE_FORMAT_NAME);
-
-            String idString = params.get(Config.SSTABLE_FORMAT_ID);
-            if (idString == null)
-                throw new ConfigurationException("Missing 'id' parameter in sstable format configuration for " + formatConfig.class_name);
-            int id;
-            try
-            {
-                id = Integer.parseInt(idString);
-            }
-            catch (RuntimeException ex)
-            {
-                throw new ConfigurationException("'id' parameter in sstable format configuration for " + formatConfig.class_name + " must be an integer");
-            }
-            if (id < 0 || id > 127)
-                throw new ConfigurationException("'id' parameter in sstable format configuration for " + formatConfig.class_name + " must be within bounds [0..127] range");
-            if (ids.contains(id))
-                throw new ConfigurationException("ID '" + id + "' of sstable format " + formatConfig.class_name + " is already defined for another sstable format");
-            params.remove(Config.SSTABLE_FORMAT_ID);
-
-            Supplier<SSTableFormat<?, ?>> factory = () -> {
-                Class<SSTableFormat<?, ?>> cls = FBUtilities.classForName(formatConfig.class_name, "sstable format");
-                if (!SSTableFormat.class.isAssignableFrom(cls))
-                    throw new ConfigurationException(String.format("Class %s for sstable format %s does not implement %s", formatConfig.class_name, name, SSTableFormat.class.getName()));
-
-                SSTableFormat<?, ?> sstableFormat = FBUtilities.instanceOrConstruct(cls.getName(), "sstable format");
-                sstableFormat.setup(id, name, params);
-                return sstableFormat;
-            };
-            sstableFormatFactories.put(name, factory);
-            names.add(name);
-            ids.add(id);
+            Map<String, String> formatOptions = options != null ? ImmutableMap.copyOf(options.getOrDefault(factory.name(), ImmutableMap.of())) : ImmutableMap.of();
+            providersBuilder.put(factory.name(), () -> factory.getInstance(formatOptions));
         }
+        ImmutableMap<String, Supplier<SSTableFormat<?, ?>>> providers = providersBuilder.build();
+        if (options != null)
+        {
+            Sets.SetView<String> unknownFormatNames = Sets.difference(options.keySet(), providers.keySet());
+            if (!unknownFormatNames.isEmpty())
+                throw new ConfigurationException(String.format("Configuration contains options of unknown sstable formats: %s", unknownFormatNames));
+        }
+        return providers;
+    }
 
-        return sstableFormatFactories.build();
+    private static Pair<SSTableFormat<?, ?>, Version> getAndValidateFormatAndVersion(Map<String, SSTableFormat<?, ?>> sstableFormats, String selectedFormatName, String selectedVersionStr)
+    {
+        SSTableFormat<?, ?> selectedFormat;
+        if (StringUtils.isBlank(selectedFormatName))
+            selectedFormatName = BigFormat.NAME;

Review Comment:
   fixed



-- 
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 #2295: Cassandra 18441

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


##########
test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java:
##########
@@ -198,10 +200,11 @@
     "org.apache.cassandra.io.sstable.MetricsProviders",
     "org.apache.cassandra.io.sstable.SSTable",
     "org.apache.cassandra.io.sstable.SSTable$Builder",
-    "org.apache.cassandra.io.sstable.format.AbstractSSTableFormat",
     "org.apache.cassandra.io.sstable.format.SSTableFormat",
     "org.apache.cassandra.io.sstable.format.SSTableFormat$Components",
     "org.apache.cassandra.io.sstable.format.SSTableFormat$Components$Types",
+    "org.apache.cassandra.io.sstable.format.SSTableFormat$Factory",
+    "org.apache.cassandra.io.sstable.format.SSTableFormat$Factory",

Review Comment:
   fixed



##########
test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java:
##########
@@ -217,6 +220,8 @@
     "org.apache.cassandra.io.sstable.format.SortedTableWriter$Builder",
     "org.apache.cassandra.io.sstable.format.Version",
     "org.apache.cassandra.io.sstable.format.big.BigFormat",
+    "org.apache.cassandra.io.sstable.format.big.BigFormat$BigFormatFactory",
+    "org.apache.cassandra.io.sstable.format.big.BigFormat$BigFormatFactory",

Review Comment:
   fixed



-- 
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 pull request #2295: CASSANDRA-18441/trunk: Fix sstable formats configuration

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

   Committed https://github.com/apache/cassandra/commit/fe0e04c2319afab958b3da83e7b54c84bced9dc2
   


-- 
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 closed pull request #2295: CASSANDRA-18441/trunk: Fix sstable formats configuration

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski closed pull request #2295: CASSANDRA-18441/trunk: Fix sstable formats configuration
URL: https://github.com/apache/cassandra/pull/2295


-- 
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