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


##########
cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/CassandraDataLayer.java:
##########
@@ -341,8 +347,15 @@ private CompletionStage<Map<String, AvailabilityHint>> 
createSnapshot(RingRespon
                     LOGGER.info("Creating snapshot on instance snapshotName={} 
keyspace={} table={} datacenter={} fqdn={}",
                                 snapshotName, maybeQuotedKeyspace, 
maybeQuotedTable, datacenter, ringEntry.fqdn());
                     SidecarInstance sidecarInstance = new 
SidecarInstanceImpl(ringEntry.fqdn(), sidecarClientConfig.effectivePort());
-                    createSnapshotFuture = sidecar
-                                           .createSnapshot(sidecarInstance, 
maybeQuotedKeyspace, maybeQuotedTable, snapshotName)
+                    String resolvedSnapshotTtl = clearSnapshot ? 
effectiveSnapshotTtl : null;
+                    if (!clearSnapshot && userProvidedSnapshotTtl != null)
+                    {
+                        LOGGER.error("Snapshot TTL option was provided along 
with Clear Snapshot option, bulk reader" +

Review Comment:
   warn instead?



##########
cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/ClientConfig.java:
##########
@@ -91,6 +101,8 @@ private ClientConfig(Map<String, String> options)
         this.datacenter = options.get(MapUtils.lowerCaseKey(DC_KEY));
         this.createSnapshot = MapUtils.getBoolean(options, 
CREATE_SNAPSHOT_KEY, true);
         this.clearSnapshot = MapUtils.getBoolean(options, CLEAR_SNAPSHOT_KEY, 
createSnapshot);
+        this.userProvidedSnapshotTtl = MapUtils.getOrDefault(options, 
USER_PROVIDED_SNAPSHOT_TTL, null);
+        this.effectiveSnapshotTtl = this.userProvidedSnapshotTtl == null ? 
DEFAULT_SNAPSHOT_TTL : USER_PROVIDED_SNAPSHOT_TTL;

Review Comment:
   this looks incorrect, did you mean?
   ```suggestion
           this.effectiveSnapshotTtl = this.userProvidedSnapshotTtl == null ? 
DEFAULT_SNAPSHOT_TTL : this.userProvidedSnapshotTtl;
   ```



##########
cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/CassandraDataLayerTests.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * 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 java.util.HashMap;
+import java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class CassandraDataLayerTests

Review Comment:
   nit
   ```suggestion
   class CassandraDataLayerTests
   ```



##########
cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/ClientConfig.java:
##########
@@ -43,6 +43,14 @@ public final class ClientConfig
     public static final String DC_KEY = "dc";
     public static final String CREATE_SNAPSHOT_KEY = "createSnapshot";
     public static final String CLEAR_SNAPSHOT_KEY = "clearSnapshot";
+    /**
+     * snapshotTTL a 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 USER_PROVIDED_SNAPSHOT_TTL = "snapshot_ttl";
+    public static final String DEFAULT_SNAPSHOT_TTL = "2d";

Review Comment:
   this seems like a reasonable default. I would expect most jobs would 
complete in well under 1 day



##########
cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/CassandraDataLayerTests.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * 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 java.util.HashMap;
+import java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class CassandraDataLayerTests
+{
+    public static final Map<String, String> REQUIRED_CLIENT_CONFIG_OPTIONS = 
ImmutableMap.of(
+    "keyspace", "big-data",
+    "table", "customers",
+    "sidecar_instances", "localhost");
+
+    @Test
+    public void testDefaultSnapshotTTL()

Review Comment:
   NIT:
   ```suggestion
      void testDefaultSnapshotTTL()
   ```



##########
cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/CassandraDataLayer.java:
##########
@@ -341,8 +347,15 @@ private CompletionStage<Map<String, AvailabilityHint>> 
createSnapshot(RingRespon
                     LOGGER.info("Creating snapshot on instance snapshotName={} 
keyspace={} table={} datacenter={} fqdn={}",
                                 snapshotName, maybeQuotedKeyspace, 
maybeQuotedTable, datacenter, ringEntry.fqdn());
                     SidecarInstance sidecarInstance = new 
SidecarInstanceImpl(ringEntry.fqdn(), sidecarClientConfig.effectivePort());
-                    createSnapshotFuture = sidecar
-                                           .createSnapshot(sidecarInstance, 
maybeQuotedKeyspace, maybeQuotedTable, snapshotName)
+                    String resolvedSnapshotTtl = clearSnapshot ? 
effectiveSnapshotTtl : null;
+                    if (!clearSnapshot && userProvidedSnapshotTtl != null)
+                    {
+                        LOGGER.error("Snapshot TTL option was provided along 
with Clear Snapshot option, bulk reader" +
+                                     "can honor only one of the 2. Clear 
Snapshot takes precedence over Snapshot TTL, " +
+                                     "hence snapshot {} will not be cleared 
with TTL", snapshotName);

Review Comment:
   nit
   ```suggestion
                                        "hence snapshot {} will not be removed 
after the specified snapshot TTL option", snapshotName);
   ```



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