You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2021/12/07 13:51:43 UTC

[GitHub] [cassandra] adelapena commented on a change in pull request #1276: CASSANDRA-17048: UUID based sstable generation numbers

adelapena commented on a change in pull request #1276:
URL: https://github.com/apache/cassandra/pull/1276#discussion_r763987532



##########
File path: src/java/org/apache/cassandra/db/ColumnFamilyStore.java
##########
@@ -194,7 +194,7 @@
     public final OpOrder readOrdering = new OpOrder();
 
     /* This is used to generate the next index for a SSTable */
-    private final AtomicInteger fileIndexGenerator = new AtomicInteger(0);
+    private final java.util.function.Supplier<? extends SSTableUniqueIdentifier> fileIndexGenerator;

Review comment:
       Nit: maybe we can extend the import of `com.google.common.base.*` to:
   ```
   import com.google.common.base.Function;
   import com.google.common.base.Joiner;
   import com.google.common.base.Predicate;
   import com.google.common.base.Predicates;
   ```
   And import `java.util.function.Supplier` to use just `Supplier` without the qualified class name.

##########
File path: src/java/org/apache/cassandra/db/ColumnFamilyStore.java
##########
@@ -373,7 +373,7 @@ public void setCompressionParametersJson(String options)
     @VisibleForTesting
     public ColumnFamilyStore(Keyspace keyspace,
                              String columnFamilyName,
-                             int generation,
+                             java.util.function.Supplier<? extends SSTableUniqueIdentifier> fileIndexGenerator,

Review comment:
       This is not related with the changes, but maybe we can remove the unused `offline` argument here.

##########
File path: src/java/org/apache/cassandra/db/ColumnFamilyStore.java
##########
@@ -758,19 +746,13 @@ public void loadNewSSTables()
 
     Descriptor getUniqueDescriptorFor(Descriptor descriptor, File targetDirectory)
     {
-        Descriptor newDescriptor;
-        do
-        {
-            newDescriptor = new Descriptor(descriptor.version,
+            Descriptor newDescriptor = new Descriptor(descriptor.version,

Review comment:
       Nit: aligning
   ```
           Descriptor newDescriptor = new Descriptor(descriptor.version,
                                                     targetDirectory,
                                                     descriptor.ksname,
                                                     descriptor.cfname,
                                                     fileIndexGenerator.get(),
                                                     descriptor.formatType);
   ```

##########
File path: src/java/org/apache/cassandra/io/sstable/SSTableUniqueIdentifier.java
##########
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable;
+
+import java.nio.ByteBuffer;
+import java.util.function.Supplier;
+import java.util.stream.Stream;
+
+/**
+ * Represents a unique identifier in the sstable descriptor filename.
+ * This ensures each sstable file uniqueness in the entire Cassandra cluster.
+ * <p>
+ * A new implementation must adhere to the following invariants:
+ * - Must be locally sortable - that is, the comparison must reflect the comparison of generation times of identifiers
+ * generated on the same node
+ * - String representation must *not* include the '-' character see {@link Descriptor#filenameSplitter}

Review comment:
       Nit: `Descriptor#filenameSplitter` is not visible from here. Maybe we can put  the `'-'` separator into a public constant in descriptor, and make `Descriptor#filenameSplitter` and these comments use it?

##########
File path: src/java/org/apache/cassandra/io/sstable/SSTable.java
##########
@@ -101,6 +102,11 @@ protected SSTable(Descriptor descriptor, Set<Component> components, TableMetadat
         this.optimizationStrategy = Objects.requireNonNull(optimizationStrategy);
     }
 
+    public Set<Component> getComponents()

Review comment:
       Nit: this is only used by tests, we should probably mark it with `@VisibleForTesting`.

##########
File path: src/java/org/apache/cassandra/io/sstable/Descriptor.java
##########
@@ -260,14 +263,14 @@ public static Descriptor fromFilename(File file)
         if (!Version.validate(versionString))
             throw invalidSSTable(name, "invalid version %s", versionString);
 
-        int generation;
+        SSTableUniqueIdentifier generation;
         try
         {
-            generation = Integer.parseInt(tokens.get(1));
+            generation = SSTableUniqueIdentifierFactory.instance.fromString(tokens.get(1));
         }
-        catch (NumberFormatException e)
+        catch (RuntimeException e)

Review comment:
       Wouldn't it be `IllegalArgumentException`?

##########
File path: src/java/org/apache/cassandra/index/sasi/SASIIndex.java
##########
@@ -72,9 +73,7 @@ public SecondaryIndexBuilder getIndexBuildTask(ColumnFamilyStore cfs,
                                                        Set<Index> indexes,
                                                        Collection<SSTableReader> sstablesToRebuild)
         {
-            NavigableMap<SSTableReader, Map<ColumnMetadata, ColumnIndex>> sstables = new TreeMap<>((a, b) -> {
-                return Integer.compare(a.descriptor.generation, b.descriptor.generation);
-            });
+            NavigableMap<SSTableReader, Map<ColumnMetadata, ColumnIndex>> sstables = new TreeMap<>(Comparator.comparing(t -> t.descriptor.generation, SSTableUniqueIdentifierFactory.COMPARATOR));

Review comment:
       We can use `SSTableReader.generationComparator`:
   ```suggestion
               NavigableMap<SSTableReader, Map<ColumnMetadata, ColumnIndex>> sstables = new TreeMap<>(SSTableReader.generationComparator);
   ```

##########
File path: test/unit/org/apache/cassandra/cql3/ViewComplexDeletionsTest.java
##########
@@ -354,7 +356,7 @@ private void testCommutativeRowDeletion(boolean flush) throws Throwable
             ColumnFamilyStore cfs = ks.getColumnFamilyStore("mv");
             List<String> sstables = cfs.getLiveSSTables()
                                        .stream()
-                                       .sorted((s1, s2) -> s1.descriptor.generation - s2.descriptor.generation)
+                                       .sorted(Comparator.comparing(s -> s.descriptor.generation, SSTableUniqueIdentifierFactory.COMPARATOR))

Review comment:
       We can use `SSTableReader.generationComparator`:
   ```suggestion
                                          .sorted(SSTableReader.generationComparator)
   ```

##########
File path: src/java/org/apache/cassandra/index/sasi/SASIIndex.java
##########
@@ -113,8 +112,7 @@ public SASIIndex(ColumnFamilyStore baseCfs, IndexMetadata config)
         Tracker tracker = baseCfs.getTracker();
         tracker.subscribe(this);
 
-        SortedMap<SSTableReader, Map<ColumnMetadata, ColumnIndex>> toRebuild = new TreeMap<>((a, b)
-                                                -> Integer.compare(a.descriptor.generation, b.descriptor.generation));
+        SortedMap<SSTableReader, Map<ColumnMetadata, ColumnIndex>> toRebuild = new TreeMap<>(Comparator.comparing(t -> t.descriptor.generation, SSTableUniqueIdentifierFactory.COMPARATOR));

Review comment:
       We can use `SSTableReader.generationComparator`:
   ```suggestion
           SortedMap<SSTableReader, Map<ColumnMetadata, ColumnIndex>> toRebuild = new TreeMap<>(SSTableReader.generationComparator);
   ```




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