frankgh commented on code in PR #31:
URL: 
https://github.com/apache/cassandra-analytics/pull/31#discussion_r1462559574


##########
cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/ClientConfig.java:
##########
@@ -109,6 +134,50 @@ private ClientConfig(Map<String, String> options)
         this.quoteIdentifiers = MapUtils.getBoolean(options, 
QUOTE_IDENTIFIERS, false);
     }
 
+    private ClearSnapshotStrategy parseClearSnapshotStrategy(boolean 
hasDeprecatedOption,

Review Comment:
   maybe make it protected for extensibility?



##########
cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/ClientConfig.java:
##########
@@ -109,6 +134,50 @@ private ClientConfig(Map<String, String> options)
         this.quoteIdentifiers = MapUtils.getBoolean(options, 
QUOTE_IDENTIFIERS, false);
     }
 
+    private ClearSnapshotStrategy parseClearSnapshotStrategy(boolean 
hasDeprecatedOption,
+                                                             boolean 
clearSnapshot,
+                                                             String 
clearSnapshotStrategyOption)
+    {
+        if (hasDeprecatedOption)
+        {
+            LOGGER.warn("The deprecated option 'clearSnapshot' is set. Please 
set 'clearSnapshotStrategy' instead.");
+            if (clearSnapshotStrategyOption == null)
+            {
+                return clearSnapshot ? ClearSnapshotStrategy.defaultStrategy() 
: new ClearSnapshotStrategy.NoOp();
+            }
+        }
+        if (clearSnapshotStrategyOption == null)
+        {
+            LOGGER.debug("No clearSnapshotStrategy is set. Using the default 
strategy");
+            return ClearSnapshotStrategy.defaultStrategy();
+        }
+        String[] strategyParts = clearSnapshotStrategyOption.split(" ");
+        String strategyName;
+        String snapshotTTL = null;
+        if (strategyParts.length == 1)
+        {
+            strategyName = strategyParts[0].trim();
+        }
+        else if (strategyParts.length == 2)
+        {
+            strategyName = strategyParts[0].trim();
+            snapshotTTL = strategyParts[1].trim();
+            if (!Pattern.matches(SNAPSHOT_TTL_PATTERN, snapshotTTL))
+            {
+                String msg = "Incorrect value set for clearSnapshotStrategy, 
expected format is " +
+                             "{strategy [snapshotTTLvalue]}. TTL value 
specified must contain unit along. " +
+                             "For e.g. 2d represents a TTL for 2 days";
+                throw new IllegalArgumentException(msg);
+            }
+        }
+        else
+        {
+            LOGGER.error("Invalid value for ClearSnapshotStrategy: '{}'", 
clearSnapshotStrategyOption);
+            throw new IllegalArgumentException("Invalid value: " + 
clearSnapshotStrategyOption);

Review Comment:
   In ClearSnapshotStrategy.create the fallback behavior is to return the 
default strategy. Should we consider that fallback strategy here as well? 



##########
cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/ClientConfig.java:
##########
@@ -24,25 +24,45 @@
 import java.util.Map;
 import java.util.Optional;
 import java.util.UUID;
+import java.util.regex.Pattern;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.bridge.BigNumberConfigImpl;
 import org.apache.cassandra.spark.config.SchemaFeature;
 import org.apache.cassandra.spark.config.SchemaFeatureSet;
 import org.apache.cassandra.spark.data.partitioner.ConsistencyLevel;
 import org.apache.cassandra.spark.utils.MapUtils;
+import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
 import static 
org.apache.cassandra.spark.data.CassandraDataLayer.aliasLastModifiedTimestamp;
 
 public final class ClientConfig
 {
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(ClientConfig.class);
+
     public static final String SIDECAR_INSTANCES = "sidecar_instances";
     public static final String KEYSPACE_KEY = "keyspace";
     public static final String TABLE_KEY = "table";
     public static final String SNAPSHOT_NAME_KEY = "snapshotName";
     public static final String DC_KEY = "dc";
     public static final String CREATE_SNAPSHOT_KEY = "createSnapshot";
     public static final String CLEAR_SNAPSHOT_KEY = "clearSnapshot";
+    /**
+     * Format of clearSnapshotStrategy is {strategy [snapshotTTLvalue]}, 
clearSnapshotStrategy holds both the strategy
+     * and in case of TTL based strategy, TTL value. For e.g. 
onCompletionOrTTL 2d, TTL 2d, noOp, onCompletion. For
+     * clear snapshot strategies allowed check {@link ClearSnapshotStrategy}
+     */
+    public static final String CLEAR_SNAPSHOT_STRATEGY_KEY = 
"clearSnapshotStrategy";
+    /**
+     * TTL value is time to live option for the snapshot (available since 
Cassandra 4.1+). TTL value specified must
+     * contain unit along. For e.g. 2d represents a TTL for 2 days; 1h 
represents a TTL of 1 hour, etc.
+     * Valid units are {@code d}, {@code h}, {@code s}, {@code ms}, {@code 
us}, {@code µs}, {@code ns}, and {@code m}.
+     */
+    public static final String DEFAULT_SNAPSHOT_TTL_VALUE = "2d";
+    public static final String SNAPSHOT_TTL_PATTERN = 
"\\d+(d|h|m|s|ms)|(\\d+|\\d+\\.\\d+|\\.\\d+)[eE][+-](\\d+|\\d+\\.\\d+|\\.\\d+)(us|µs|ns)";

Review Comment:
   TTL support is limited to days , hours, minutes and seconds, and it doesn't 
support e notation. This is the pattern used in Cassandra
   
   ```
   private static final Pattern UNITS_PATTERN = 
Pattern.compile(("^(\\d+)(d|h|s|ms|us|µs|ns|m)$"));
   ```
   
   we'll need to limit it to `d|h|m|s`



##########
cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/ClientConfig.java:
##########
@@ -237,4 +311,125 @@ private List<SchemaFeature> 
initRequestedFeatures(Map<String, String> options)
         }
         return requestedFeatures;
     }
+
+    abstract static class ClearSnapshotStrategy
+    {
+        private final String snapshotTTL;
+
+        static ClearSnapshotStrategy create(String name, String snapshotTTL)
+        {
+            String stripped = name.trim();
+            if (stripped.equalsIgnoreCase(OnCompletion.class.getSimpleName()))
+            {
+                return new OnCompletion();
+            }
+            else if (stripped.equalsIgnoreCase(TTL.class.getSimpleName()))
+            {
+                return new TTL(snapshotTTL);
+            }
+            else if 
(stripped.equalsIgnoreCase(OnCompletionOrTTL.class.getSimpleName()))
+            {
+                return new OnCompletionOrTTL(snapshotTTL);
+            }
+            else if (stripped.equalsIgnoreCase(NoOp.class.getSimpleName()))
+            {
+                return new NoOp();
+            }
+            else
+            {
+                ClearSnapshotStrategy defaultStrategy = defaultStrategy();
+                LOGGER.warn("Unknown ClearSnapshotStrategy {} is passed. Fall 
back to default strategy {}.",
+                            name, defaultStrategy);
+                return defaultStrategy;
+            }
+        }
+
+        static ClearSnapshotStrategy defaultStrategy()

Review Comment:
   can we make it public?



##########
cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/ClientConfigTests.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.spark.data;
+
+import org.junit.jupiter.api.Test;
+
+import static 
org.apache.cassandra.spark.data.ClientConfig.SNAPSHOT_TTL_PATTERN;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+class ClientConfigTests
+{
+    @Test
+    void testPositiveSnapshotTTLPatterns()
+    {
+        assertTrue("2h".matches(SNAPSHOT_TTL_PATTERN));
+        assertTrue("200s".matches(SNAPSHOT_TTL_PATTERN));
+        assertTrue("20000ms".matches(SNAPSHOT_TTL_PATTERN));

Review Comment:
   ms is not supported by the backend



-- 
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: commits-unsubscr...@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to