smiklosovic commented on code in PR #4399:
URL: https://github.com/apache/cassandra/pull/4399#discussion_r2410269478


##########
src/java/org/apache/cassandra/db/compression/CompressionDictionaryScheduler.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.db.compression;
+
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.concurrent.ScheduledExecutors;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import 
org.apache.cassandra.db.compression.ICompressionDictionaryTrainer.TrainingStatus;
+import org.apache.cassandra.schema.SystemDistributedKeyspace;
+
+/**
+ * Manages scheduled tasks for compression dictionary operations.
+ * <p>
+ * This class handles:
+ * - Periodic refresh of dictionaries from system tables
+ * - Manual training task scheduling and monitoring
+ * - Cleanup of scheduled tasks
+ */
+public class CompressionDictionaryScheduler implements 
ICompressionDictionaryScheduler
+{
+    private static final Logger logger = 
LoggerFactory.getLogger(CompressionDictionaryScheduler.class);
+
+    private final String keyspaceName;
+    private final String tableName;
+    private final ICompressionDictionaryCache cache;
+
+    private volatile ScheduledFuture<?> scheduledRefreshTask;
+    private volatile ScheduledFuture<?> scheduledManualTrainingTask;
+    private volatile boolean isEnabled;
+
+    public CompressionDictionaryScheduler(String keyspaceName,
+                                          String tableName,
+                                          ICompressionDictionaryCache cache,
+                                          boolean isEnabled)
+    {
+        this.keyspaceName = keyspaceName;
+        this.tableName = tableName;
+        this.cache = cache;
+        this.isEnabled = isEnabled;
+    }
+
+    /**
+     * Schedules the periodic dictionary refresh task if not already scheduled.
+     */
+    public void scheduleRefreshTask()
+    {
+        if (scheduledRefreshTask != null)
+            return;
+
+        this.scheduledRefreshTask = 
ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(
+        this::refreshDictionaryFromSystemTable,
+        
DatabaseDescriptor.getCompressionDictionaryRefreshInitialDelaySeconds(),
+        DatabaseDescriptor.getCompressionDictionaryRefreshIntervalSeconds(),
+        TimeUnit.SECONDS
+        );
+    }
+
+    @Override
+    public void scheduleManualTraining(ManualTrainingOptions options, 
ICompressionDictionaryTrainer trainer)
+    {
+        if (scheduledManualTrainingTask != null)
+        {
+            throw new IllegalStateException("Training already in progress for 
table " + keyspaceName + '.' + tableName);
+        }
+
+        int maxSamplingDurationSeconds = 
options.getMaxSamplingDurationSeconds();
+
+        logger.info("Starting manual dictionary training for {}.{} with max 
sampling duration: {} seconds",
+                    keyspaceName, tableName, maxSamplingDurationSeconds);
+
+        long deadlineMillis = System.currentTimeMillis() + 
TimeUnit.SECONDS.toMillis(maxSamplingDurationSeconds);

Review Comment:
   Avoid System for time, should use `org.apache.cassandra.utils.Clock.Global` 
or `org.apache.cassandra.utils.Clock` interface [blockSystemClock]



##########
src/java/org/apache/cassandra/io/compress/CompressionMetadata.java:
##########
@@ -139,11 +165,54 @@ private CompressionMetadata(CompressionMetadata copy)
         this.compressedFileLength = copy.compressedFileLength;
         this.chunkOffsets = copy.chunkOffsets;
         this.chunkOffsetsSize = copy.chunkOffsetsSize;
+        this.compressionDictionary = copy.compressionDictionary;
     }
 
     public ICompressor compressor()
     {
-        return parameters.getSstableCompressor();
+        // classic double-checked locking to call resolveCompressor method 
just once per CompressionMetadata object
+        ICompressor result = resolvedCompressor;
+        if (result != null)
+            return result;
+
+        synchronized (this)
+        {
+            result = resolvedCompressor;
+            if (result == null)
+            {
+                result = resolveCompressor(parameters.getSstableCompressor(), 
compressionDictionary);
+                resolvedCompressor = result;
+            }
+            return result;
+        }
+    }
+
+    static ICompressor resolveCompressor(ICompressor compressor, 
CompressionDictionary dictionary)
+    {
+        if (dictionary == null)
+            return compressor;
+
+        // When the attached dictionary can be consumed by the current 
dictionary compressor
+        if (compressor instanceof IDictionaryCompressor)
+        {
+            IDictionaryCompressor dictionaryCompressor = 
(IDictionaryCompressor) compressor;
+            if (dictionaryCompressor.canConsumeDictionary(dictionary))
+                return 
dictionaryCompressor.getOrCopyWithDictionary(dictionary);
+        }
+
+        // When the current compressor is not compatible with the dictionary. 
It could happen in the read path when:
+        // 1. The current compressor is not a dictionary compressor, but there 
is dictionary attached
+        // 2. The current dictionary compressor is a different type, e.g. 
table schema is changed
+        // In those cases, we should get the compatible dictionary compressor 
based on the dictionary
+        if (dictionary.kind() == ZSTD)

Review Comment:
   We are again too "ZSTD-centric" here. I would expect this does not contain 
anything ZSTD specific so we do not need to touch this anymore. The fact that 
dictionary is not empty again pretty much guarantees that we can create a 
compressor from it, no? Why would we have a dictionary, of 
`CompressionDictionary` type, if we were not able to create a compressor from 
it? That would be a pretty useless dictionary to me. 
   
   So maybe doing something like 
   
   `return dictionary.kind().getCompressor(dictionary);`
   
   and calling `ZstdDictionaryCompressor.create(zstdDict)` in there would be 
way more simpler and at one place. 
   
   You would need to return `ICompressor` from `getCompressor` as 
`resolveCompressor` returns it too. If `IDictionaryCompressor` extended 
`ICompressor` as already suggested elsewhere (as you need to instantiate 
`ZstdDictionaryCompressor` there) it would be a piece of cake. 



##########
src/java/org/apache/cassandra/db/compression/CompressionDictionaryEventHandler.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.db.compression;
+
+import org.apache.cassandra.concurrent.ScheduledExecutors;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.net.Message;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.Verb;
+import org.apache.cassandra.schema.SystemDistributedKeyspace;
+import org.apache.cassandra.tcm.ClusterMetadata;
+import org.apache.cassandra.utils.FBUtilities;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;

Review Comment:
   checkstyle (`ant realclean && ant artifacts`) evaluated this as illegal 
import, see checkstyle.xml and grep `CompletableFuture` there to see. I am not 
completely sure yet but I think it should be replaced by something in 
`org.apache.cassandra.utils.concurrent` package. 



##########
src/java/org/apache/cassandra/io/compress/CompressionMetadata.java:
##########
@@ -139,11 +165,54 @@ private CompressionMetadata(CompressionMetadata copy)
         this.compressedFileLength = copy.compressedFileLength;
         this.chunkOffsets = copy.chunkOffsets;
         this.chunkOffsetsSize = copy.chunkOffsetsSize;
+        this.compressionDictionary = copy.compressionDictionary;
     }
 
     public ICompressor compressor()
     {
-        return parameters.getSstableCompressor();
+        // classic double-checked locking to call resolveCompressor method 
just once per CompressionMetadata object
+        ICompressor result = resolvedCompressor;
+        if (result != null)
+            return result;
+
+        synchronized (this)
+        {
+            result = resolvedCompressor;
+            if (result == null)
+            {
+                result = resolveCompressor(parameters.getSstableCompressor(), 
compressionDictionary);
+                resolvedCompressor = result;
+            }
+            return result;
+        }
+    }
+
+    static ICompressor resolveCompressor(ICompressor compressor, 
CompressionDictionary dictionary)
+    {
+        if (dictionary == null)
+            return compressor;
+
+        // When the attached dictionary can be consumed by the current 
dictionary compressor
+        if (compressor instanceof IDictionaryCompressor)
+        {
+            IDictionaryCompressor dictionaryCompressor = 
(IDictionaryCompressor) compressor;
+            if (dictionaryCompressor.canConsumeDictionary(dictionary))
+                return 
dictionaryCompressor.getOrCopyWithDictionary(dictionary);
+        }
+
+        // When the current compressor is not compatible with the dictionary. 
It could happen in the read path when:
+        // 1. The current compressor is not a dictionary compressor, but there 
is dictionary attached
+        // 2. The current dictionary compressor is a different type, e.g. 
table schema is changed
+        // In those cases, we should get the compatible dictionary compressor 
based on the dictionary
+        if (dictionary.kind() == ZSTD)

Review Comment:
   or even better
   
        return dictionary.getCompressor();
   
   which would be like
   
       default IDictionaryCompressor getCompressor()
       {
           return kind().getCompressor(this);
       }
   
   



##########
conf/cassandra.yaml:
##########
@@ -617,6 +617,54 @@ counter_cache_save_period: 7200s
 # Disabled by default, meaning all keys are going to be saved
 # counter_cache_keys_to_save: 100
 
+# Dictionary compression settings for ZSTD dictionary-based compression

Review Comment:
   We most probably need to copy all of this to cassandra_latest.yaml as well. 



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

To unsubscribe, e-mail: [email protected]

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


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

Reply via email to