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

[GitHub] [cassandra] dcapwell commented on a diff in pull request #2295: Cassandra 18441

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