diff --git a/build.gradle b/build.gradle
index 065e903f8..7b3c01701 100644
--- a/build.gradle
+++ b/build.gradle
@@ -461,3 +461,14 @@ sourcesDistTar.finalizedBy generateDistributionChecksums
sourcesDistZip.finalizedBy generateDistributionChecksums
assembleDist.finalizedBy generateDistributionChecksums
assembleSourcesDist.finalizedBy generateDistributionChecksums
+
+// Wire shadowJar to run as part of the core subproject's assemble (and therefore the
+// root aggregate assemble), without creating a root-level finalizedBy on shadowJar.
+// The latter chains through distTar/distZip (which `dependsOn` every subproject Jar via
+// the `distributions { from subprojects.collect { it.tasks.withType(Jar) } }` block) and
+// loops back to :assemble, causing a Gradle 8 circular-dependency failure.
+project(':cassandra-analytics-core') {
+ afterEvaluate {
+ tasks.named('assemble').configure { dependsOn tasks.named('shadowJar') }
+ }
+}
diff --git a/cassandra-analytics-common/src/main/java/org/apache/cassandra/analytics/stats/Stats.java b/cassandra-analytics-common/src/main/java/org/apache/cassandra/analytics/stats/Stats.java
index b9fa17f59..b3df2a6f5 100644
--- a/cassandra-analytics-common/src/main/java/org/apache/cassandra/analytics/stats/Stats.java
+++ b/cassandra-analytics-common/src/main/java/org/apache/cassandra/analytics/stats/Stats.java
@@ -25,6 +25,7 @@
import java.util.Set;
import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.FileType;
import org.apache.cassandra.spark.data.SSTable;
import org.apache.cassandra.spark.data.SSTablesSupplier;
import org.apache.cassandra.spark.reader.IndexEntry;
@@ -498,4 +499,50 @@ public void indexFileSkipped()
{
}
+
+ /**
+ * S3 headObject operation performed for existence check.
+ *
+ * @param timeNanos time taken in nanoseconds for the S3 headObject operation
+ */
+ public void s3HeadObjectOperation(long timeNanos)
+ {
+
+ }
+
+ /**
+ * S3 getObject operation performed for data retrieval.
+ *
+ * @param timeNanos time taken in nanoseconds for the S3 getObject operation
+ */
+ public void s3GetObjectOperation(long timeNanos)
+ {
+
+ }
+
+ /**
+ * Mutable SSTable metadata objects (Summary.db, Filter.db, Statistics.db) can be rewritten in place
+ * on Cassandra live data directories. Backup manifests can therefore carry a stale size for the
+ * same S3 key. This event records when the actual object size differs from the manifest-provided
+ * size.
+ *
+ * @param fileType mutable SSTable component type
+ * @param manifestSize size recorded in the autosnap manifest
+ * @param actualSize current S3 object size
+ */
+ public void s3MutableMetadataDriftDetected(FileType fileType, long manifestSize, long actualSize)
+ {
+
+ }
+
+ /**
+ * Mutable metadata object was large enough that the reader used HEAD + exact ranged GET instead
+ * of a single open-ended GET.
+ *
+ * @param fileType mutable SSTable component type
+ */
+ public void s3MutableMetadataHeadFallback(FileType fileType)
+ {
+
+ }
}
diff --git a/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/data/FileType.java b/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/data/FileType.java
index 3d4523270..48234ea5c 100644
--- a/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/data/FileType.java
+++ b/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/data/FileType.java
@@ -83,4 +83,15 @@ public String getFileSuffix()
{
return fileSuffix;
}
+
+ /**
+ * Whether the on-disk size of this component can drift from the value recorded in a backup
+ * manifest. Cassandra rewrites Summary/Filter/Statistics in place during compaction, so a
+ * stale manifest size for these components can produce truncated ranged-GETs against the
+ * backing store. The data layer treats these components specially when issuing reads.
+ */
+ public boolean isMutableMetadata()
+ {
+ return this == SUMMARY || this == FILTER || this == STATISTICS;
+ }
}
diff --git a/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/data/partitioner/CassandraRing.java b/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/data/partitioner/CassandraRing.java
index 3d142f8df..84fdb6889 100644
--- a/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/data/partitioner/CassandraRing.java
+++ b/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/data/partitioner/CassandraRing.java
@@ -29,8 +29,10 @@
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
+import java.util.Set;
import java.util.stream.Collectors;
import java.util.Objects;
@@ -59,9 +61,15 @@
* It is made to be immutable for the sake of simplicity.
*
* Token ranges are calculated assuming Cassandra racks are not being used, but controlled by assigning tokens properly.
+ * Callers that need rack-aware placement (e.g. sidecar {@code tokenRangeReplicas}, or any
+ * external source-of-truth topology metadata) should use the 5-arg constructor and supply
+ * an authoritative {@link RangeMap} keyed by {@code (start, end]} sub-ranges; see
+ * CASSANALYTICS-79.
*
- * {@link #equals(Object)} and {@link #hashCode()} don't take {@link #replicas} and {@link #tokenRangeMap}
- * into consideration as they are just derived fields.
+ * {@link #equals(Object)} and {@link #hashCode()} include {@link #replicas} and
+ * {@link #tokenRangeMap}, so rings with the same {@code instances} but different
+ * authoritative placements compare unequal. The fields remain {@code transient}:
+ * {@link #init} rebuilds them from the serialized inputs before any comparison.
*/
@SuppressWarnings({"UnstableApiUsage", "unused", "WeakerAccess"})
public class CassandraRing implements Serializable
@@ -74,6 +82,12 @@ public class CassandraRing implements Serializable
private ReplicationFactor replicationFactor;
private List instances;
+ // Authoritative per-range replicas supplied by the 5-arg constructor, in a flat
+ // Serializable shape. Null when the 4-arg (naive) constructor was used. Carries the
+ // input through Java/Kryo serde so the receiving end can rebuild the transient
+ // {@code replicas} / {@code tokenRangeMap} fields identically.
+ private List authoritativeReplicas;
+
private transient RangeMap> replicas;
private transient Multimap> tokenRangeMap;
@@ -117,10 +131,147 @@ public CassandraRing(Partitioner partitioner,
this.instances = instances.stream()
.sorted(Comparator.comparing(instance -> new BigInteger(instance.token())))
.collect(Collectors.toCollection(ArrayList::new));
+ this.authoritativeReplicas = null;
+ this.init();
+ }
+
+ /**
+ * Authoritative-replica constructor. {@code authoritativeReplicas} is adopted verbatim
+ * as the source of truth for per-range placement, bypassing the naive
+ * {@link RangeUtils#calculateTokenRanges} derivation used by the 4-arg constructor.
+ * Intended for callers that can supply a rack-aware mapping (e.g. sidecar
+ * {@code tokenRangeReplicas} or any external source-of-truth topology metadata).
+ *
+ * Validation (fail loud, {@link IllegalArgumentException}):
+ *
+ * - each replica list is non-null and non-empty;
+ * - each range has {@code lower < upper} (Guava's
+ * {@link Range#openClosed(Comparable, Comparable)} enforces this for individual
+ * entries; cross-entry inversion is caught here);
+ * - every replica {@link CassandraInstance} is element-equal to one in
+ * {@code instances}, so the ring and the replica map agree on node identity;
+ * - the union of supplied ranges equals {@code (minToken, maxToken]} (no gaps; an
+ * uncovered sub-range would be silently skipped by
+ * {@code TokenPartitioner.subRanges});
+ * - no two supplied ranges overlap (Guava's {@link TreeRangeMap#put} would otherwise
+ * silently overwrite).
+ *
+ */
+ public CassandraRing(Partitioner partitioner,
+ String keyspace,
+ ReplicationFactor replicationFactor,
+ Collection instances,
+ RangeMap> authoritativeReplicas)
+ {
+ Preconditions.checkArgument(authoritativeReplicas != null,
+ "authoritativeReplicas must not be null; use the 4-arg constructor for naive derivation");
+ this.partitioner = partitioner;
+ this.keyspace = keyspace;
+ this.replicationFactor = replicationFactor;
+ this.instances = instances.stream()
+ .sorted(Comparator.comparing(instance -> new BigInteger(instance.token())))
+ .collect(Collectors.toCollection(ArrayList::new));
+ this.authoritativeReplicas = validateAndFlatten(authoritativeReplicas, this.instances,
+ partitioner);
this.init();
}
+ /**
+ * Validates the supplied authoritative mapping and flattens it into a serializable list.
+ * See the 5-arg constructor javadoc for the invariants enforced here.
+ */
+ private static List validateAndFlatten(RangeMap> authoritative,
+ List sortedInstances,
+ Partitioner partitioner)
+ {
+ Set known = new HashSet<>(sortedInstances);
+ Map, List> asMap = authoritative.asMapOfRanges();
+ Preconditions.checkArgument(!asMap.isEmpty(),
+ "authoritativeReplicas must contain at least one range");
+
+ List flat = new ArrayList<>(asMap.size());
+ for (Map.Entry, List> entry : asMap.entrySet())
+ {
+ Range range = entry.getKey();
+ List replicaList = entry.getValue();
+ Preconditions.checkArgument(replicaList != null && !replicaList.isEmpty(),
+ "replica list must be non-null and non-empty for range %s",
+ range);
+ // Defense in depth: Guava's openClosed enforces lower < upper at construction,
+ // but this guard catches any non-canonical RangeMap input.
+ Preconditions.checkArgument(range.lowerEndpoint().compareTo(range.upperEndpoint()) < 0,
+ "range lower must be strictly less than upper: %s", range);
+ for (CassandraInstance replica : replicaList)
+ {
+ Preconditions.checkArgument(known.contains(replica),
+ "replica %s for range %s is not in the supplied instances collection",
+ replica, range);
+ }
+ flat.add(new RangeReplicas(range.lowerEndpoint(), range.upperEndpoint(),
+ new ArrayList<>(replicaList)));
+ }
+
+ // Sort by lower, then walk for contiguous full-ring tiling.
+ flat.sort(Comparator.comparing(rr -> rr.lower));
+ BigInteger expectedLower = partitioner.minToken();
+ BigInteger expectedUpper = partitioner.maxToken();
+ for (int i = 0; i < flat.size(); i++)
+ {
+ RangeReplicas rr = flat.get(i);
+ if (i == 0)
+ {
+ Preconditions.checkArgument(rr.lower.equals(expectedLower),
+ "authoritative replicas must start at partitioner minToken %s, got %s",
+ expectedLower, rr.lower);
+ }
+ else
+ {
+ BigInteger prevUpper = flat.get(i - 1).upper;
+ Preconditions.checkArgument(rr.lower.compareTo(prevUpper) >= 0,
+ "authoritative replicas have overlapping ranges near lower=%s prevUpper=%s",
+ rr.lower, prevUpper);
+ Preconditions.checkArgument(rr.lower.equals(prevUpper),
+ "authoritative replicas have a gap between prevUpper=%s and lower=%s",
+ prevUpper, rr.lower);
+ }
+ }
+ BigInteger lastUpper = flat.get(flat.size() - 1).upper;
+ Preconditions.checkArgument(lastUpper.equals(expectedUpper),
+ "authoritative replicas must end at partitioner maxToken %s, got %s",
+ expectedUpper, lastUpper);
+ return flat;
+ }
+
private void init()
+ {
+ if (authoritativeReplicas == null)
+ {
+ initFromInstancesNaive();
+ return;
+ }
+ replicas = TreeRangeMap.create();
+ tokenRangeMap = ArrayListMultimap.create();
+ for (RangeReplicas rr : authoritativeReplicas)
+ {
+ Range r = Range.openClosed(rr.lower, rr.upper);
+ // Authoritative input is already aggregated and disjoint per validateAndFlatten,
+ // so SET (put) rather than ADD-with-split-on-overlap (addReplica).
+ replicas.put(r, rr.replicas);
+ for (CassandraInstance i : rr.replicas)
+ {
+ tokenRangeMap.put(i, r);
+ }
+ }
+ }
+
+ /**
+ * Rack-unaware naive derivation invoked when no authoritative per-range replica map is
+ * supplied. Sorts tokens and assigns the next RF nodes as replicas, ignoring rack
+ * placement; correct only on clusters that are not using NTS racks. Retained as a
+ * fallback for the CDC and sidecar bulk-read paths; their migration to an authoritative
+ * map is deferred to CASSANALYTICS-79.
+ */
+ private void initFromInstancesNaive()
{
// Setup token range map
replicas = TreeRangeMap.create();
@@ -278,6 +429,29 @@ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundE
{
this.instances.add(new CassandraInstance(in.readUTF(), in.readUTF(), in.readUTF()));
}
+
+ // Authoritative replicas: boolean presence flag matches the readNullable/writeNullable
+ // pattern used elsewhere in this package. Replica encoding mirrors the instance block
+ // above (triple-UTF token/nodeName/dataCenter) so CassandraInstance itself does not need
+ // to participate in serialization.
+ if (in.readBoolean())
+ {
+ int rangeCount = in.readInt();
+ List flat = new ArrayList<>(rangeCount);
+ for (int r = 0; r < rangeCount; r++)
+ {
+ BigInteger lower = new BigInteger(in.readUTF());
+ BigInteger upper = new BigInteger(in.readUTF());
+ int replicaCount = in.readShort();
+ ArrayList replicas = new ArrayList<>(replicaCount);
+ for (int j = 0; j < replicaCount; j++)
+ {
+ replicas.add(new CassandraInstance(in.readUTF(), in.readUTF(), in.readUTF()));
+ }
+ flat.add(new RangeReplicas(lower, upper, replicas));
+ }
+ this.authoritativeReplicas = flat;
+ }
this.init();
}
@@ -303,6 +477,24 @@ private void writeObject(ObjectOutputStream out) throws IOException, ClassNotFou
out.writeUTF(instance.nodeName());
out.writeUTF(instance.dataCenter());
}
+
+ out.writeBoolean(this.authoritativeReplicas != null);
+ if (this.authoritativeReplicas != null)
+ {
+ out.writeInt(this.authoritativeReplicas.size());
+ for (RangeReplicas rr : this.authoritativeReplicas)
+ {
+ out.writeUTF(rr.lower.toString());
+ out.writeUTF(rr.upper.toString());
+ out.writeShort(rr.replicas.size());
+ for (CassandraInstance i : rr.replicas)
+ {
+ out.writeUTF(i.token());
+ out.writeUTF(i.nodeName());
+ out.writeUTF(i.dataCenter());
+ }
+ }
+ }
}
public static class Serializer extends com.esotericsoftware.kryo.Serializer
@@ -314,17 +506,77 @@ public void write(Kryo kryo, Output out, CassandraRing ring)
out.writeString(ring.keyspace);
kryo.writeObject(out, ring.replicationFactor);
kryo.writeObject(out, ring.instances);
+
+ out.writeBoolean(ring.authoritativeReplicas != null);
+ if (ring.authoritativeReplicas != null)
+ {
+ out.writeInt(ring.authoritativeReplicas.size());
+ for (RangeReplicas rr : ring.authoritativeReplicas)
+ {
+ out.writeString(rr.lower.toString());
+ out.writeString(rr.upper.toString());
+ out.writeShort(rr.replicas.size());
+ for (CassandraInstance i : rr.replicas)
+ {
+ out.writeString(i.token());
+ out.writeString(i.nodeName());
+ out.writeString(i.dataCenter());
+ }
+ }
+ }
}
@Override
@SuppressWarnings("unchecked")
public CassandraRing read(Kryo kryo, Input in, Class type)
{
- return new CassandraRing(in.readByte() == 1 ? Partitioner.RandomPartitioner
- : Partitioner.Murmur3Partitioner,
- in.readString(),
- kryo.readObject(in, ReplicationFactor.class),
- kryo.readObject(in, ArrayList.class));
+ Partitioner partitioner = in.readByte() == 1 ? Partitioner.RandomPartitioner
+ : Partitioner.Murmur3Partitioner;
+ String keyspace = in.readString();
+ ReplicationFactor rf = kryo.readObject(in, ReplicationFactor.class);
+ ArrayList instances = kryo.readObject(in, ArrayList.class);
+
+ if (!in.readBoolean())
+ {
+ return new CassandraRing(partitioner, keyspace, rf, instances);
+ }
+ int rangeCount = in.readInt();
+ RangeMap> auth = TreeRangeMap.create();
+ for (int r = 0; r < rangeCount; r++)
+ {
+ BigInteger lower = new BigInteger(in.readString());
+ BigInteger upper = new BigInteger(in.readString());
+ int replicaCount = in.readShort();
+ List replicas = new ArrayList<>(replicaCount);
+ for (int j = 0; j < replicaCount; j++)
+ {
+ replicas.add(new CassandraInstance(in.readString(), in.readString(), in.readString()));
+ }
+ auth.put(Range.openClosed(lower, upper), replicas);
+ }
+ return new CassandraRing(partitioner, keyspace, rf, instances, auth);
+ }
+ }
+
+ /**
+ * Flat {@link Serializable} carrier for a single {@code (start, end] -> replicas} entry.
+ * Lives on {@link CassandraRing} as a non-transient list so the authoritative mapping
+ * survives JDK and Kryo serialization without depending on Guava's {@link TreeRangeMap}
+ * (which is not {@link Serializable}). On deserialize, {@link CassandraRing#init()}
+ * rebuilds the transient {@code replicas} / {@code tokenRangeMap} fields from this list.
+ */
+ static final class RangeReplicas implements Serializable
+ {
+ private static final long serialVersionUID = 1L;
+ final BigInteger lower;
+ final BigInteger upper;
+ final ArrayList replicas;
+
+ RangeReplicas(BigInteger lower, BigInteger upper, ArrayList replicas)
+ {
+ this.lower = lower;
+ this.upper = upper;
+ this.replicas = replicas;
}
}
}
diff --git a/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/data/partitioner/TokenPartitioner.java b/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/data/partitioner/TokenPartitioner.java
index 6f63c357a..979fd5f78 100644
--- a/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/data/partitioner/TokenPartitioner.java
+++ b/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/data/partitioner/TokenPartitioner.java
@@ -81,12 +81,43 @@ public TokenPartitioner(CassandraRing ring, int defaultParallelism, int numCores
public TokenPartitioner(CassandraRing ring, int defaultParallelism, int numCores, boolean shuffle)
{
- LOGGER.info("Creating TokenPartitioner defaultParallelism={} numCores={}", defaultParallelism, numCores);
+ this(ring, null, defaultParallelism, numCores, shuffle);
+ }
+
+ /**
+ * Variant that accepts an optional user-specified override for the number of splits per token
+ * range. When {@code userSpecifiedNumberSplits} is non-null and {@code > 0}, it bypasses the
+ * default {@code max(cores, defaultParallelism) / ranges} formula. Values {@code <= 0} (and
+ * {@code null}) fall back to the default formula, matching the existing 2-/3-arg overloads.
+ *
+ * Mirrors the analogous knob on
+ * {@code org.apache.cassandra.spark.bulkwriter.TokenPartitioner} in cassandra-analytics-core
+ * (referenced as text to avoid a reverse module dependency).
+ * Motivation: the S3 batch-read path reconstructs the ring from an autosnap manifest rather
+ * than a live cluster, so the auto formula can pick parallelism that's a poor fit for the
+ * SSTable count / per-node size. Plumbed through {@code S3DataSourceClientConfig.number_splits}.
+ */
+ public TokenPartitioner(CassandraRing ring,
+ Integer userSpecifiedNumberSplits,
+ int defaultParallelism,
+ int numCores)
+ {
+ this(ring, userSpecifiedNumberSplits, defaultParallelism, numCores, false);
+ }
+
+ public TokenPartitioner(CassandraRing ring,
+ Integer userSpecifiedNumberSplits,
+ int defaultParallelism,
+ int numCores,
+ boolean shuffle)
+ {
+ LOGGER.info("Creating TokenPartitioner userSpecifiedNumberSplits={} defaultParallelism={} numCores={}",
+ userSpecifiedNumberSplits, defaultParallelism, numCores);
this.partitionMap = TreeRangeMap.create();
this.reversePartitionMap = new HashMap<>();
this.ring = ring;
- int numSplits = TokenPartitioner.calculateSplits(ring, defaultParallelism, numCores);
+ int numSplits = TokenPartitioner.calculateSplits(ring, userSpecifiedNumberSplits, defaultParallelism, numCores);
this.subRanges = ring.rangeMap().asMapOfRanges().keySet().stream()
.flatMap(tr -> RangeUtils.split(tr, numSplits).stream()).collect(Collectors.toList());
@@ -134,6 +165,26 @@ private static int calculateSplits(CassandraRing ring, int defaultParallelism, I
return calculatedSplits;
}
+ /**
+ * Override-aware variant. Uses {@code userSpecifiedNumberSplits} when {@code > 0}, otherwise
+ * falls through to the default formula. {@code 0} and negative values mean "auto" — matching
+ * the {@code S3DataSourceClientConfig#DEFAULT_NUM_SPLITS} sentinel (defined in
+ * cassandra-analytics-core to avoid a reverse module dependency) and avoiding the degenerate
+ * "zero splits per range" case that an unguarded {@code >= 0} check would allow.
+ */
+ private static int calculateSplits(CassandraRing ring,
+ Integer userSpecifiedNumberSplits,
+ int defaultParallelism,
+ Integer cores)
+ {
+ if (userSpecifiedNumberSplits != null && userSpecifiedNumberSplits > 0)
+ {
+ LOGGER.info("Using user-specified number of splits: {}", userSpecifiedNumberSplits);
+ return userSpecifiedNumberSplits;
+ }
+ return calculateSplits(ring, defaultParallelism, cores);
+ }
+
public CassandraRing ring()
{
return ring;
diff --git a/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/sparksql/CellIterator.java b/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/sparksql/CellIterator.java
index 59707a6a1..8d353017d 100644
--- a/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/sparksql/CellIterator.java
+++ b/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/sparksql/CellIterator.java
@@ -134,6 +134,17 @@ public CqlTable cqlTable()
return cqlTable;
}
+ /**
+ * Exposes the {@link Stats} instance threaded through scanner construction. Used by
+ * Spark-side row iterators (e.g. {@code SparkRowIterator#currentMetricsValues}) to surface
+ * per-task custom metrics — those metrics are accumulated on the {@code Stats} implementation
+ * (specifically {@code SparkCustomMetricsStats}) during scan execution.
+ */
+ public Stats stats()
+ {
+ return stats;
+ }
+
public boolean hasProjectedValueColumns()
{
return hasProjectedValueColumns;
diff --git a/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/utils/Properties.java b/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/utils/Properties.java
index 346d76c67..9fe738582 100644
--- a/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/utils/Properties.java
+++ b/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/utils/Properties.java
@@ -42,6 +42,10 @@ public final class Properties
public static final int DEFAULT_MAX_POOL_SIZE = 64;
public static final long DEFAULT_MAX_BUFFER_SIZE = 6 * MEBI_BYTES;
public static final long DEFAULT_CHUNK_BUFFER_SIZE = 4 * MEBI_BYTES;
+ // Defaults for Data.db on S3-backed SSTables. Sized to balance per-GET (and SSE-KMS decrypt) overhead against
+ // executor heap pressure when many overlapping SSTables stream concurrently per Spark task.
+ public static final long DEFAULT_S3_DATA_CHUNK_BUFFER_SIZE = 8 * MEBI_BYTES;
+ public static final long DEFAULT_S3_DATA_MAX_BUFFER_SIZE = 32 * MEBI_BYTES;
public static final Map DEFAULT_MAX_BUFFER_OVERRIDE = Map.ofEntries(
entry(FileType.INDEX, 128 * KIBI_BYTES),
entry(FileType.SUMMARY, 256 * KIBI_BYTES),
diff --git a/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/utils/S3SnapshotTimeProvider.java b/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/utils/S3SnapshotTimeProvider.java
new file mode 100644
index 000000000..9e0cb2656
--- /dev/null
+++ b/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/utils/S3SnapshotTimeProvider.java
@@ -0,0 +1,48 @@
+/*
+ * 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.utils;
+
+/**
+ * A {@link TimeProvider} for S3-based reads that sets the reference epoch from autosnap timestamps.
+ *
+ * The reference epoch drives TTL and tombstone evaluation in {@code CompactionIterator}'s
+ * {@code cell.isLive(nowInSec)} checks. To avoid silently dropping cells that were still alive
+ * at the time of an earlier node's snapshot, the reference epoch should be the earliest
+ * (minimum) autosnap epoch across all nodes. This ensures conservative TTL behavior: a cell
+ * is only expired if it was already expired at every node's snapshot time.
+ */
+public class S3SnapshotTimeProvider implements TimeProvider
+{
+ private final long referenceEpochInSeconds;
+
+ /**
+ * @param referenceEpochInSeconds the earliest autosnap epoch in seconds across all nodes
+ */
+ public S3SnapshotTimeProvider(long referenceEpochInSeconds)
+ {
+ this.referenceEpochInSeconds = referenceEpochInSeconds;
+ }
+
+ @Override
+ public long referenceEpochInSeconds()
+ {
+ return referenceEpochInSeconds;
+ }
+}
diff --git a/cassandra-analytics-common/src/test/java/org/apache/cassandra/spark/data/partitioner/CassandraRingTests.java b/cassandra-analytics-common/src/test/java/org/apache/cassandra/spark/data/partitioner/CassandraRingTests.java
index c66290f0f..68adce8fd 100644
--- a/cassandra-analytics-common/src/test/java/org/apache/cassandra/spark/data/partitioner/CassandraRingTests.java
+++ b/cassandra-analytics-common/src/test/java/org/apache/cassandra/spark/data/partitioner/CassandraRingTests.java
@@ -20,20 +20,25 @@
package org.apache.cassandra.spark.data.partitioner;
import java.math.BigInteger;
+import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
+import java.util.Collections;
import java.util.List;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Multimap;
import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
import com.google.common.collect.RangeSet;
+import com.google.common.collect.TreeRangeMap;
import com.google.common.collect.TreeRangeSet;
import org.junit.jupiter.api.Test;
import org.apache.cassandra.spark.data.ReplicationFactor;
import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
@SuppressWarnings("UnstableApiUsage")
public class CassandraRingTests
@@ -347,6 +352,195 @@ public void testNetworkStrategyRF11()
Partitioner.Murmur3Partitioner.maxToken()));
}
+ // ---------- 5-arg authoritative-replica constructor ----------
+
+ private static ReplicationFactor ntsRf33()
+ {
+ return new ReplicationFactor(ImmutableMap.of(
+ "class", "org.apache.cassandra.locator.NetworkTopologyStrategy",
+ "DC1", "3",
+ "DC2", "3"));
+ }
+
+ private static List twoDcSixInstances()
+ {
+ return new ArrayList<>(Arrays.asList(
+ new CassandraInstance("0", "dc1-a", "DC1"),
+ new CassandraInstance("100", "dc1-b", "DC1"),
+ new CassandraInstance("200", "dc1-c", "DC1"),
+ new CassandraInstance("1", "dc2-a", "DC2"),
+ new CassandraInstance("101", "dc2-b", "DC2"),
+ new CassandraInstance("201", "dc2-c", "DC2")));
+ }
+
+ /**
+ * Build a single-range full-ring authoritative mapping where every supplied instance is a
+ * replica. Useful for tests where the per-range placement is uninteresting but full-ring
+ * coverage must be satisfied.
+ */
+ private static RangeMap> fullRingAllReplicas(List instances)
+ {
+ RangeMap> rm = TreeRangeMap.create();
+ rm.put(Range.openClosed(Partitioner.Murmur3Partitioner.minToken(),
+ Partitioner.Murmur3Partitioner.maxToken()),
+ new ArrayList<>(instances));
+ return rm;
+ }
+
+ @Test
+ public void testAuthoritativeCtorReplicasAdoptedVerbatim()
+ {
+ List instances = twoDcSixInstances();
+ BigInteger minToken = Partitioner.Murmur3Partitioner.minToken();
+ BigInteger maxToken = Partitioner.Murmur3Partitioner.maxToken();
+ RangeMap> auth = TreeRangeMap.create();
+ Range lowHalf = Range.openClosed(minToken, BigInteger.ZERO);
+ Range highHalf = Range.openClosed(BigInteger.ZERO, maxToken);
+ List lowReplicas = Arrays.asList(instances.get(0), instances.get(3));
+ List highReplicas = Arrays.asList(instances.get(1), instances.get(2),
+ instances.get(4), instances.get(5));
+ auth.put(lowHalf, lowReplicas);
+ auth.put(highHalf, highReplicas);
+
+ CassandraRing ring = new CassandraRing(Partitioner.Murmur3Partitioner, "ks", ntsRf33(), instances, auth);
+
+ // Exact map equality - the supplied mapping IS what rangeMap() returns.
+ assertThat(ring.rangeMap().asMapOfRanges()).isEqualTo(auth.asMapOfRanges());
+
+ // Instances sorted by token.
+ assertThat(new ArrayList<>(ring.instances()).stream().map(CassandraInstance::token))
+ .containsExactly("0", "1", "100", "101", "200", "201");
+
+ // tokens() and tokens(dc) still derive from instances correctly.
+ assertThat(ring.tokens(Partitioner.Murmur3Partitioner.name())).isNotNull();
+ assertThat(ring.tokens("DC1")).containsExactlyInAnyOrder(
+ BigInteger.valueOf(0L), BigInteger.valueOf(100L), BigInteger.valueOf(200L));
+
+ // tokenRangeMap (multimap inverse) must list every instance against the range it was assigned to.
+ Multimap> inverse = ring.tokenRanges();
+ for (CassandraInstance i : lowReplicas)
+ {
+ assertThat(inverse.get(i)).contains(lowHalf);
+ }
+ for (CassandraInstance i : highReplicas)
+ {
+ assertThat(inverse.get(i)).contains(highHalf);
+ }
+ }
+
+ @Test
+ public void testAuthoritativeCtorRejectsNullMap()
+ {
+ List instances = twoDcSixInstances();
+ assertThatThrownBy(() -> new CassandraRing(Partitioner.Murmur3Partitioner, "ks", ntsRf33(),
+ instances, null))
+ .isInstanceOf(IllegalArgumentException.class);
+ }
+
+ @Test
+ public void testAuthoritativeCtorRejectsEmptyReplicaList()
+ {
+ List instances = twoDcSixInstances();
+ RangeMap> auth = TreeRangeMap.create();
+ auth.put(Range.openClosed(Partitioner.Murmur3Partitioner.minToken(),
+ Partitioner.Murmur3Partitioner.maxToken()),
+ Collections.emptyList());
+ assertThatThrownBy(() -> new CassandraRing(Partitioner.Murmur3Partitioner, "ks", ntsRf33(), instances, auth))
+ .isInstanceOf(IllegalArgumentException.class)
+ .hasMessageContaining("non-null and non-empty");
+ }
+
+ @Test
+ public void testAuthoritativeCtorRejectsUnknownReplica()
+ {
+ List instances = twoDcSixInstances();
+ CassandraInstance stranger = new CassandraInstance("9999", "ghost", "DC1");
+ RangeMap> auth = TreeRangeMap.create();
+ auth.put(Range.openClosed(Partitioner.Murmur3Partitioner.minToken(),
+ Partitioner.Murmur3Partitioner.maxToken()),
+ Arrays.asList(instances.get(0), stranger));
+ assertThatThrownBy(() -> new CassandraRing(Partitioner.Murmur3Partitioner, "ks", ntsRf33(), instances, auth))
+ .isInstanceOf(IllegalArgumentException.class)
+ .hasMessageContaining("not in the supplied instances collection");
+ }
+
+ @Test
+ public void testAuthoritativeCtorRejectsRangeGap()
+ {
+ List instances = twoDcSixInstances();
+ BigInteger minToken = Partitioner.Murmur3Partitioner.minToken();
+ BigInteger maxToken = Partitioner.Murmur3Partitioner.maxToken();
+ RangeMap> auth = TreeRangeMap.create();
+ // Leave a gap between -10 and +10
+ auth.put(Range.openClosed(minToken, BigInteger.valueOf(-10L)),
+ new ArrayList<>(instances));
+ auth.put(Range.openClosed(BigInteger.valueOf(10L), maxToken),
+ new ArrayList<>(instances));
+ assertThatThrownBy(() -> new CassandraRing(Partitioner.Murmur3Partitioner, "ks", ntsRf33(), instances, auth))
+ .isInstanceOf(IllegalArgumentException.class)
+ .hasMessageContaining("gap");
+ }
+
+ @Test
+ public void testAuthoritativeCtorRejectsPartialCoverageMissingTail()
+ {
+ // Guava's TreeRangeMap.put auto-overlap-resolves; non-overlapping partial coverage
+ // (missing the head or tail) is the relevant failure mode here.
+ List instances = twoDcSixInstances();
+ BigInteger minToken = Partitioner.Murmur3Partitioner.minToken();
+ RangeMap> auth = TreeRangeMap.create();
+ auth.put(Range.openClosed(minToken, BigInteger.valueOf(0L)),
+ new ArrayList<>(instances));
+ assertThatThrownBy(() -> new CassandraRing(Partitioner.Murmur3Partitioner, "ks", ntsRf33(), instances, auth))
+ .isInstanceOf(IllegalArgumentException.class)
+ .hasMessageContaining("maxToken");
+ }
+
+ @Test
+ public void testAuthoritativeCtorRejectsMissingHead()
+ {
+ List instances = twoDcSixInstances();
+ BigInteger maxToken = Partitioner.Murmur3Partitioner.maxToken();
+ RangeMap> auth = TreeRangeMap.create();
+ auth.put(Range.openClosed(BigInteger.valueOf(0L), maxToken),
+ new ArrayList<>(instances));
+ assertThatThrownBy(() -> new CassandraRing(Partitioner.Murmur3Partitioner, "ks", ntsRf33(), instances, auth))
+ .isInstanceOf(IllegalArgumentException.class)
+ .hasMessageContaining("minToken");
+ }
+
+ @Test
+ public void testAuthoritativeCtorRejectsEmptyMap()
+ {
+ List instances = twoDcSixInstances();
+ RangeMap> empty = TreeRangeMap.create();
+ assertThatThrownBy(() -> new CassandraRing(Partitioner.Murmur3Partitioner, "ks", ntsRf33(), instances, empty))
+ .isInstanceOf(IllegalArgumentException.class);
+ }
+
+ @Test
+ public void testAuthoritativeAndNaiveRingsAreNotEqualForDifferentPlacement()
+ {
+ List instances = twoDcSixInstances();
+ CassandraRing naive = new CassandraRing(Partitioner.Murmur3Partitioner, "ks", ntsRf33(), instances);
+ // Authoritative: all 6 instances replicate the full ring (very different from RF=3-per-DC naive placement)
+ CassandraRing authoritative = new CassandraRing(Partitioner.Murmur3Partitioner, "ks", ntsRf33(),
+ instances, fullRingAllReplicas(instances));
+ assertThat(authoritative).isNotEqualTo(naive);
+ assertThat(authoritative.hashCode()).isNotEqualTo(naive.hashCode());
+ }
+
+ @Test
+ public void testAuthoritativeCtorRoundTripsEqual()
+ {
+ List instances = twoDcSixInstances();
+ RangeMap> auth = fullRingAllReplicas(instances);
+ CassandraRing a = new CassandraRing(Partitioner.Murmur3Partitioner, "ks", ntsRf33(), instances, auth);
+ CassandraRing b = new CassandraRing(Partitioner.Murmur3Partitioner, "ks", ntsRf33(), instances, auth);
+ assertThat(a).isEqualTo(b);
+ assertThat(a.hashCode()).isEqualTo(b.hashCode());
+ }
+
@Test
public void testNetworkStrategyRF22()
{
diff --git a/cassandra-analytics-core/build.gradle b/cassandra-analytics-core/build.gradle
index 8df6b1399..5ea9b750f 100644
--- a/cassandra-analytics-core/build.gradle
+++ b/cassandra-analytics-core/build.gradle
@@ -23,6 +23,10 @@ plugins {
id('java-library')
id('jacoco')
id('maven-publish')
+ id('com.github.johnrengelman.shadow')
+ // Publishes src/testFixtures so other modules can reuse cross-module test utilities such as
+ // org.apache.cassandra.bridge.VersionRunner via testImplementation(testFixtures(project(...))).
+ id('java-test-fixtures')
}
if (propertyWithDefault("artifactType", null) == "spark")
@@ -40,6 +44,39 @@ java {
}
}
+// Add shadow publication alongside the default maven publication inherited from common/publishing.gradle.
+// Only register when the spark publishing is active; otherwise rootProject.group is unresolved for non-spark
+// artifactType invocations of this module.
+if (propertyWithDefault("artifactType", null) == "spark")
+{
+ publishing {
+ publications {
+ shadow(MavenPublication) {
+ // Use the shadowJar task artifact directly instead of components.shadow
+ artifact shadowJar
+ groupId rootProject.group
+ artifactId "${archivesBaseName}-shadow"
+ version System.getenv("CODE_VERSION") ?: "${project.version}"
+
+ if (mavenRepositoryUrl != "") {
+ pom {
+ name = 'Apache Cassandra Analytics Core (Shadow)'
+ description = 'Apache Cassandra Analytics Core module with all dependencies (Shadow JAR)'
+ url = "https://cassandra.apache.org/"
+ licenses {
+ license {
+ name = "The Apache License, Version 2.0"
+ url = "https://www.apache.org/licenses/LICENSE-2.0.txt"
+ distribution = "repo"
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+}
+
configurations {
fourzero {
description = 'Cassandra 4.0 dependency'
@@ -95,14 +132,25 @@ dependencies {
implementation(group: 'org.scala-lang.modules', name: "scala-java8-compat_2.11", version: '1.0.1', transitive: false)
}
+ // Spark 3.5 no longer brings both commons-lang variants transitively. Core classes use both
+ // package names, so keep the dependencies explicit.
+ implementation "org.apache.commons:commons-lang3:${project.commonsLang3Version}"
+ implementation "commons-lang:commons-lang:2.6"
+
// aws sdk BOM + s3
implementation 'software.amazon.awssdk:s3'
+ implementation 'software.amazon.awssdk:s3-transfer-manager'
implementation 'software.amazon.awssdk:netty-nio-client'
implementation platform(group: 'software.amazon.awssdk', name:'bom', version:"${project.aswSdkVersion}")
compileOnly(group: "${sparkGroupId}", name: "spark-core_${scalaMajorVersion}", version: "${project.rootProject.sparkVersion}")
compileOnly(group: "${sparkGroupId}", name: "spark-sql_${scalaMajorVersion}", version: "${project.rootProject.sparkVersion}")
+ // testFixtures: cross-module test utilities (e.g. VersionRunner). Kept minimal so consumers
+ // pull in only what the published fixtures require.
+ testFixturesApi(project(':cassandra-bridge'))
+ testFixturesApi("org.junit.jupiter:junit-jupiter-api:${project.junitVersion}")
+
testImplementation(group: 'com.google.guava', name: 'guava', version: '19.0')
testImplementation(testFixtures(project(':cassandra-bridge')))
testImplementation(project(':cassandra-analytics-spark-converter'))
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/KryoRegister.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/KryoRegister.java
index f53112169..d9490a11e 100644
--- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/KryoRegister.java
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/KryoRegister.java
@@ -39,6 +39,8 @@
import org.apache.cassandra.spark.data.CassandraDataLayer;
import org.apache.cassandra.spark.data.LocalDataLayer;
import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.apache.cassandra.spark.data.S3CassandraDataLayer;
+import org.apache.cassandra.spark.data.S3ClientConfig;
import org.apache.cassandra.spark.data.partitioner.CassandraInstance;
import org.apache.cassandra.spark.data.partitioner.CassandraRing;
import org.apache.cassandra.spark.data.partitioner.TokenPartitioner;
@@ -72,6 +74,8 @@ public class KryoRegister implements KryoRegistrator
KRYO_SERIALIZERS.put(CassandraRing.class, new CassandraRing.Serializer());
KRYO_SERIALIZERS.put(TokenPartitioner.class, new TokenPartitioner.Serializer());
KRYO_SERIALIZERS.put(CassandraDataLayer.class, new CassandraDataLayer.Serializer());
+ KRYO_SERIALIZERS.put(S3CassandraDataLayer.class, new S3CassandraDataLayer.Serializer());
+ KRYO_SERIALIZERS.put(S3ClientConfig.class, new S3ClientConfig.Serializer());
KRYO_SERIALIZERS.put(BigNumberConfigImpl.class, new BigNumberConfigImpl.Serializer());
KRYO_SERIALIZERS.put(SslConfig.class, new SslConfig.Serializer());
KRYO_SERIALIZERS.put(SSTableTimeRangeFilter.class, new SSTableTimeRangeFilter.Serializer());
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/S3SizingFactory.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/S3SizingFactory.java
new file mode 100644
index 000000000..31690b2ab
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/S3SizingFactory.java
@@ -0,0 +1,86 @@
+/*
+ * 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.common;
+
+import org.apache.cassandra.spark.data.DefaultSizing;
+import org.apache.cassandra.spark.data.DynamicSizing;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.apache.cassandra.spark.data.backup.BackupReader;
+import org.apache.cassandra.spark.data.S3DataSourceClientConfig;
+import org.apache.cassandra.spark.data.S3TableSizeProvider;
+import org.apache.cassandra.spark.data.Sizing;
+import org.apache.cassandra.spark.data.TableSizeProvider;
+import org.apache.cassandra.spark.data.partitioner.ConsistencyLevel;
+
+import static org.apache.cassandra.spark.data.S3DataSourceClientConfig.SIZING_DEFAULT;
+import static org.apache.cassandra.spark.data.S3DataSourceClientConfig.SIZING_DYNAMIC;
+
+/**
+ * A factory class that creates {@link Sizing} for S3-based data layers without sidecar dependency
+ */
+public class S3SizingFactory
+{
+ /**
+ * Private constructor that prevents unnecessary instantiation
+ *
+ * @throws IllegalStateException when called
+ */
+ private S3SizingFactory()
+ {
+ throw new IllegalStateException(getClass() + " is a static utility class and shall not be instantiated");
+ }
+
+ /**
+ * Returns the {@link Sizing} object based on the {@code sizing} option provided by the user,
+ * or {@link DefaultSizing} as the default sizing for S3-based data layers
+ *
+ * @param replicationFactor the replication factor
+ * @param options the {@link S3DataSourceClientConfig} options
+ * @param consistencyLevel the ConsistencyLevel to use
+ * @param keyspace the keyspace
+ * @param table the table
+ * @param datacenter the DataCenter to use
+ * @param s3BackupReader the S3 backup reader instance to use for table size calculation
+ * @param clusterName the cluster name for S3 backup identification
+ * @return the {@link Sizing} object based on the {@code sizing} option provided by the user
+ */
+ public static Sizing create(ReplicationFactor replicationFactor,
+ S3DataSourceClientConfig options,
+ ConsistencyLevel consistencyLevel,
+ String keyspace,
+ String table,
+ String datacenter,
+ BackupReader s3BackupReader,
+ String clusterName)
+ {
+ if (SIZING_DYNAMIC.equalsIgnoreCase(options.sizing()))
+ {
+ TableSizeProvider tableSizeProvider = new S3TableSizeProvider(s3BackupReader, clusterName);
+ return new DynamicSizing(tableSizeProvider, consistencyLevel, replicationFactor,
+ keyspace, table, datacenter,
+ options.maxPartitionSize(), options.numCores());
+ }
+ else if (options.sizing() == null || options.sizing().isEmpty() || SIZING_DEFAULT.equalsIgnoreCase(options.sizing()))
+ {
+ return new DefaultSizing(options.numCores());
+ }
+ throw new RuntimeException(String.format("Invalid sizing option provided '%s'", options.sizing()));
+ }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/CassandraDataLayer.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/CassandraDataLayer.java
index 99cbc6827..157547a27 100644
--- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/CassandraDataLayer.java
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/CassandraDataLayer.java
@@ -1049,7 +1049,7 @@ protected void await(CountDownLatch latch)
}
}
- static void aliasLastModifiedTimestamp(List requestedFeatures, String alias)
+ public static void aliasLastModifiedTimestamp(List requestedFeatures, String alias)
{
SchemaFeature featureAlias = new SchemaFeature()
{
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/CassandraSchemaConfig.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/CassandraSchemaConfig.java
new file mode 100644
index 000000000..1a39844ac
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/CassandraSchemaConfig.java
@@ -0,0 +1,387 @@
+/*
+ * 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.io.Serializable;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.spark.utils.MapUtils;
+
+/**
+ * Configuration for Cassandra schema-related settings.
+ * This class contains settings needed to build CQL schema and connect to a Cassandra table.
+ * It is shared across batch and streaming configurations.
+ */
+public class CassandraSchemaConfig implements Serializable
+{
+ private static final long serialVersionUID = 1L;
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(CassandraSchemaConfig.class);
+
+ // Option keys
+ public static final String CLUSTER_NAME_KEY = "clusterName";
+ public static final String CLUSTER_ID_KEY = "clusterId";
+ public static final String KEYSPACE_KEY = "keyspace";
+ public static final String TABLE_KEY = "table";
+ public static final String DATACENTER_KEY = "datacenter";
+ public static final String DC_KEY = "dc"; // Alias for datacenter
+ public static final String TABLE_CREATE_STMT_KEY = "tableCreateStmt";
+ public static final String CASSANDRA_VERSION_KEY = "cassandraVersion";
+ public static final String UDTS_KEY = "udts";
+ public static final String REPLICATION_STRATEGY_KEY = "replicationStrategy";
+ public static final String REPLICATION_FACTOR_KEY = "replicationFactor";
+
+ // Defaults
+ public static final String DEFAULT_REPLICATION_STRATEGY = "NetworkTopologyStrategy";
+ public static final String DEFAULT_REPLICATION_FACTOR = "usw2:3";
+ public static final String DEFAULT_CASSANDRA_VERSION = CassandraVersion.FOURZERO.name();
+
+ @NotNull
+ private final String clusterName;
+ @NotNull
+ private final String keyspace;
+ @NotNull
+ private final String table;
+ @Nullable
+ private final String datacenter;
+ @NotNull
+ private final String tableCreateStmt;
+ @NotNull
+ private final String cassandraVersion;
+ @NotNull
+ private final String udts;
+ @NotNull
+ private final String replicationStrategy;
+ @NotNull
+ private final String replicationFactor;
+
+ private CassandraSchemaConfig(@NotNull String clusterName,
+ @NotNull String keyspace,
+ @NotNull String table,
+ @Nullable String datacenter,
+ @NotNull String tableCreateStmt,
+ @NotNull String cassandraVersion,
+ @NotNull String udts,
+ @NotNull String replicationStrategy,
+ @NotNull String replicationFactor)
+ {
+ this.clusterName = clusterName;
+ this.keyspace = keyspace;
+ this.table = table;
+ this.datacenter = datacenter;
+ this.tableCreateStmt = tableCreateStmt;
+ this.cassandraVersion = cassandraVersion;
+ this.udts = udts;
+ this.replicationStrategy = replicationStrategy;
+ this.replicationFactor = replicationFactor;
+ }
+
+ /**
+ * Create a CassandraSchemaConfig from a map of options.
+ *
+ * Cluster identity is a single string downstream: it flows into the {@link
+ * org.apache.cassandra.spark.data.backup.BackupReader} contract as one {@code clusterName}
+ * parameter, and the implementation is free to interpret it however it
+ * needs (UUID, human-readable name, etc.). For caller convenience, two option keys are
+ * accepted at the DataSource boundary and aliased onto that single field: {@code clusterId}
+ * and {@code clusterName}. At least one must be provided; when both are set, {@code
+ * clusterId} wins.
+ *
+ * @param options Configuration options map (case-insensitive keys supported via MapUtils)
+ * @return New CassandraSchemaConfig instance
+ * @throws IllegalArgumentException if required options are missing
+ */
+ public static CassandraSchemaConfig create(Map options)
+ {
+ String clusterNameOpt = MapUtils.getOrDefault(options, CLUSTER_NAME_KEY, null);
+ String clusterIdOpt = MapUtils.getOrDefault(options, CLUSTER_ID_KEY, null);
+
+ if ((clusterNameOpt == null || clusterNameOpt.trim().isEmpty())
+ && (clusterIdOpt == null || clusterIdOpt.trim().isEmpty()))
+ {
+ throw new IllegalArgumentException(
+ "At least one of 'clusterName' or 'clusterId' must be provided");
+ }
+
+ // clusterId, if provided, wins. Otherwise pass clusterName through unmodified — specific backup reader
+ // implementations may interpret it however they need.
+ String resolvedClusterId;
+ if (clusterIdOpt != null && !clusterIdOpt.trim().isEmpty())
+ {
+ resolvedClusterId = clusterIdOpt;
+ if (clusterNameOpt != null && !clusterNameOpt.trim().isEmpty())
+ {
+ LOGGER.info("Both 'clusterName' ({}) and 'clusterId' ({}) provided; "
+ + "'clusterId' takes priority",
+ clusterNameOpt, clusterIdOpt);
+ }
+ }
+ else
+ {
+ resolvedClusterId = clusterNameOpt;
+ }
+
+ String keyspace = MapUtils.getOrThrow(options, KEYSPACE_KEY, "keyspace");
+ String table = MapUtils.getOrThrow(options, TABLE_KEY, "table");
+
+ // Support both "datacenter" and "dc" keys (case-insensitive)
+ String datacenter = MapUtils.getOrDefault(options, DATACENTER_KEY, null);
+ if (datacenter == null || datacenter.isEmpty())
+ {
+ datacenter = MapUtils.getOrDefault(options, DC_KEY, null);
+ }
+
+ String tableCreateStmt = MapUtils.getOrThrow(options, TABLE_CREATE_STMT_KEY, "tableCreateStmt");
+ String cassandraVersion = MapUtils.getOrDefault(options, CASSANDRA_VERSION_KEY, DEFAULT_CASSANDRA_VERSION);
+ String udts = MapUtils.getOrDefault(options, UDTS_KEY, "");
+ String replicationStrategy = MapUtils.getOrDefault(options,
+ REPLICATION_STRATEGY_KEY, DEFAULT_REPLICATION_STRATEGY);
+ String replicationFactor = MapUtils.getOrDefault(options,
+ REPLICATION_FACTOR_KEY, DEFAULT_REPLICATION_FACTOR);
+
+ return new CassandraSchemaConfig(resolvedClusterId, keyspace, table, datacenter,
+ tableCreateStmt, cassandraVersion, udts, replicationStrategy, replicationFactor);
+ }
+
+ /**
+ * Get the cluster identifier. Resolved eagerly at config creation: the {@code clusterId}
+ * option wins when present, otherwise {@code clusterName} is passed through verbatim. The
+ * concrete {@link org.apache.cassandra.spark.data.backup.BackupReader} decides how to
+ * interpret the value (UUID, human-readable name, etc.).
+ *
+ * @return the resolved cluster identifier
+ */
+ @NotNull
+ public String clusterName()
+ {
+ return clusterName;
+ }
+
+ /**
+ * Get the Cassandra keyspace name.
+ *
+ * @return the keyspace name
+ */
+ @NotNull
+ public String keyspace()
+ {
+ return keyspace;
+ }
+
+ /**
+ * Get the Cassandra table name.
+ *
+ * @return the table name
+ */
+ @NotNull
+ public String table()
+ {
+ return table;
+ }
+
+ /**
+ * Get the datacenter name.
+ *
+ * @return Datacenter name or null if not specified
+ */
+ @Nullable
+ public String datacenter()
+ {
+ return datacenter;
+ }
+
+ /**
+ * Get the CREATE TABLE statement for the Cassandra table.
+ *
+ * @return the CREATE TABLE DDL string
+ */
+ @NotNull
+ public String tableCreateStmt()
+ {
+ return tableCreateStmt;
+ }
+
+ /**
+ * Get the Cassandra version as a string.
+ *
+ * @return the Cassandra version string (e.g. {@code "FOURZERO"})
+ */
+ @NotNull
+ public String cassandraVersionString()
+ {
+ return cassandraVersion;
+ }
+
+ /**
+ * Get the Cassandra version as an enum.
+ *
+ * @return the Cassandra version enum
+ */
+ @NotNull
+ public CassandraVersion cassandraVersion()
+ {
+ return CassandraVersion.valueOf(cassandraVersion);
+ }
+
+ /**
+ * Get the raw UDT definitions string.
+ * UDTs are separated by newlines.
+ *
+ * @return the raw UDT definitions string (never {@code null}; may be empty)
+ */
+ @NotNull
+ public String udts()
+ {
+ return udts;
+ }
+
+ /**
+ * Parse UDT definitions into a set of individual UDT statements.
+ *
+ * @return Set of UDT definition strings
+ */
+ @NotNull
+ public Set parsedUdts()
+ {
+ return Arrays.stream(udts.split("\n"))
+ .filter(StringUtils::isNotEmpty)
+ .collect(Collectors.toSet());
+ }
+
+ /**
+ * Get the replication strategy name.
+ *
+ * @return the replication strategy name (e.g. {@code SimpleStrategy}, {@code NetworkTopologyStrategy})
+ */
+ @NotNull
+ public String replicationStrategy()
+ {
+ return replicationStrategy;
+ }
+
+ /**
+ * Get the raw replication factor string.
+ * Format depends on strategy: "3" for SimpleStrategy, "dc1:3,dc2:3" for NetworkTopologyStrategy.
+ *
+ * @return the raw replication-factor string in its strategy-specific format
+ */
+ @NotNull
+ public String replicationFactorString()
+ {
+ return replicationFactor;
+ }
+
+ /**
+ * Parse the replication factor configuration and create a ReplicationFactor object.
+ * The replicationFactor string format is "datacenter:factor,datacenter:factor"
+ * e.g., "usw2:3,euw1:3" for NetworkTopologyStrategy or "3" for SimpleStrategy.
+ *
+ * @return ReplicationFactor object based on the configured strategy and factors
+ * @throws IllegalArgumentException if the format is invalid
+ */
+ @NotNull
+ public ReplicationFactor getParsedReplicationFactor()
+ {
+ ReplicationFactor.ReplicationStrategy strategy =
+ ReplicationFactor.ReplicationStrategy.getEnum(replicationStrategy);
+
+ if (strategy == ReplicationFactor.ReplicationStrategy.SimpleStrategy)
+ {
+ try
+ {
+ int factor = Integer.parseInt(replicationFactor);
+ return ReplicationFactor.simpleStrategy(factor);
+ }
+ catch (NumberFormatException e)
+ {
+ throw new IllegalArgumentException(
+ "For SimpleStrategy, replicationFactor must be a single integer, got: " + replicationFactor, e);
+ }
+ }
+ else if (strategy == ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy)
+ {
+ Map dcFactors = new HashMap<>();
+
+ if (replicationFactor.trim().isEmpty())
+ {
+ throw new IllegalArgumentException("Replication factor cannot be empty for NetworkTopologyStrategy");
+ }
+
+ String[] pairs = replicationFactor.split(",");
+ for (String pair : pairs)
+ {
+ String trimmedPair = pair.trim();
+ if (trimmedPair.isEmpty())
+ {
+ continue;
+ }
+
+ String[] parts = trimmedPair.split(":");
+ if (parts.length != 2)
+ {
+ throw new IllegalArgumentException(
+ "Invalid replication factor format. Expected 'datacenter:factor', got: " + trimmedPair);
+ }
+ try
+ {
+ String dc = parts[0].trim();
+ String factorStr = parts[1].trim();
+
+ if (dc.isEmpty())
+ {
+ throw new IllegalArgumentException("Datacenter name cannot be empty in: " + trimmedPair);
+ }
+ if (factorStr.isEmpty())
+ {
+ throw new IllegalArgumentException("Replication factor cannot be empty in: " + trimmedPair);
+ }
+
+ int factor = Integer.parseInt(factorStr);
+ dcFactors.put(dc, factor);
+ }
+ catch (NumberFormatException e)
+ {
+ throw new IllegalArgumentException("Invalid replication factor number in: " + trimmedPair, e);
+ }
+ }
+ return new ReplicationFactor(strategy, dcFactors);
+ }
+ else if (strategy == ReplicationFactor.ReplicationStrategy.LocalStrategy)
+ {
+ return new ReplicationFactor(strategy, new HashMap<>());
+ }
+ else
+ {
+ throw new IllegalStateException("Unknown ReplicationStrategy: " + strategy);
+ }
+ }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/DataLayer.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/DataLayer.java
index 394913acd..bd5816fe9 100644
--- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/DataLayer.java
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/DataLayer.java
@@ -25,6 +25,7 @@
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
+import java.util.OptionalLong;
import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.stream.Collectors;
@@ -236,6 +237,22 @@ public abstract SSTablesSupplier sstables(int partitionId,
*/
public abstract String jobId();
+ /**
+ * Calculate the total size of all SSTable Data.db files across all partitions.
+ * This method provides a default implementation that returns empty. Specific DataLayer
+ * implementations should override this method if they have efficient ways to
+ * calculate the total size.
+ *
+ * @return OptionalLong containing the total size in bytes of all SSTable Data.db files,
+ * or empty if size information is not available
+ */
+ public OptionalLong calculateTotalSSTableSize()
+ {
+ // Default implementation returns empty (unknown size)
+ // Specific DataLayer implementations can override this for better estimates
+ return OptionalLong.empty();
+ }
+
public StreamScanner openCompactionScanner(int partitionId, List partitionKeyFilters, SSTableTimeRangeFilter sstableTimeRangeFilter)
{
return openCompactionScanner(partitionId, partitionKeyFilters, sstableTimeRangeFilter, null);
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/S3CassandraDataLayer.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/S3CassandraDataLayer.java
new file mode 100644
index 000000000..a5eab7a11
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/S3CassandraDataLayer.java
@@ -0,0 +1,1714 @@
+/*
+ * 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 com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.collect.Range;
+
+import org.apache.cassandra.analytics.stats.Stats;
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.bridge.CassandraBridgeFactory;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.clients.ExecutorHolder;
+import org.apache.cassandra.spark.config.SchemaFeature;
+import org.apache.cassandra.spark.config.SchemaFeatureSet;
+import org.apache.cassandra.spark.data.backup.BackupReader;
+import org.apache.cassandra.spark.data.backup.BackupReaderRegistry;
+import org.apache.cassandra.spark.sparksql.RowBuilder;
+import org.apache.cassandra.spark.sparksql.SnapshotTimestampDecorator;
+import org.apache.cassandra.spark.data.partitioner.CassandraInstance;
+import org.apache.cassandra.spark.data.partitioner.CassandraRing;
+import org.apache.cassandra.spark.data.partitioner.ConsistencyLevel;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.data.partitioner.TokenPartitioner;
+import org.apache.cassandra.spark.sparksql.SparkCustomMetricsStats;
+import org.apache.cassandra.spark.utils.TimeProvider;
+import org.apache.cassandra.spark.utils.S3SnapshotTimeProvider;
+import org.apache.cassandra.spark.utils.ScalaFunctions;
+import org.apache.cassandra.spark.utils.streaming.BufferingInputStream;
+import org.apache.cassandra.spark.utils.streaming.CassandraFileSource;
+import org.apache.cassandra.spark.utils.streaming.StreamConsumer;
+import org.apache.cassandra.spark.common.S3SizingFactory;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.util.ShutdownHookManager;
+
+import org.apache.commons.lang.NotImplementedException;
+
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.OptionalLong;
+
+import org.apache.cassandra.bridge.TokenRange;
+import org.apache.cassandra.spark.utils.RangeUtils;
+
+/**
+ * S3-backed CassandraDataLayer. The concrete backup format is provided by a pluggable
+ * {@link BackupReader} resolved via {@link BackupReaderRegistry} using the
+ * {@code backupReaderType} option (no default; callers must register a factory).
+ *
+ * Assumes that Murmur3Partitioner is used. The backup reader is expected to return a list of
+ * Cassandra instances per individual vnode.
+ */
+public class S3CassandraDataLayer extends PartitionedDataLayer implements Serializable
+{
+ private static final long serialVersionUID = 1997L;
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(S3CassandraDataLayer.class);
+
+ /**
+ * JVM-wide intern cache that canonicalizes {@link BackupReader} instances per executor, so
+ * all tasks reading the same manifest+S3 identity share a single reader (and its
+ * implementation-specific cache, e.g. {@code sstableInfoCache}) instead of one copy per
+ * deserialized task.
+ *
+ * Key: {@code (cluster, keyspace, table, datacenter, earliestEpoch, latestEpoch,
+ * manifestFingerprint, s3Region, s3Bucket, s3EndpointOverride, s3CredentialsFingerprint,
+ * s3HttpMaxConcurrency)}. The fingerprint (SHA-256 over sorted {@code (nodeId, epoch)}
+ * pairs) is the authoritative manifest identity — without it, two manifest sets sharing
+ * the same {@code (min, max)} epochs alias and silently read stale SSTables (real failure
+ * mode with 3+ nodes when a middle node rolls independently). S3 identity fields prevent a
+ * caller with a different {@code s3Config} from reading through the wrong endpoint.
+ *
+ * Credentials in the key isolate IAM principals and force a fresh reader after static-key
+ * rotation. For prod (EMR instance role / IRSA / IMDSv2) the access keys are null and the
+ * fingerprint collapses to a constant; STS rotation happens inside the SDK and does not
+ * invalidate the key.
+ *
+ * Values are weak ({@link CacheBuilder#weakValues()}), so canonical readers are GC'd once
+ * no layer references them. {@link Cache#get(Object, java.util.concurrent.Callable)} is
+ * the atomic install-or-return primitive and pins the returned value across the call.
+ *
+ * BackupReader read methods receive the task {@link Stats}, preserving S3 GET/HEAD metric
+ * attribution when tasks share a canonical reader.
+ */
+ private static final class ReaderInternCache
+ {
+ // weakValues: entries auto-evict once no layer references the canonical reader.
+ // Reachability is the correct lifecycle signal here; do not add time-based eviction.
+ private static final Cache CACHE =
+ CacheBuilder.newBuilder().weakValues().build();
+
+ private static BackupReader canonicalize(String clusterName,
+ String keyspace,
+ String table,
+ String datacenter,
+ long earliestSnapshotEpochSecond,
+ long latestSnapshotEpochSecond,
+ @NotNull BackupReader fresh)
+ {
+ // Bypass when manifest identity isn't fully materialized: production sets all three
+ // (both epochs and fingerprint) inside initializeS3BackupReader. Hitting any of
+ // these branches means we'd otherwise install a partially-keyed entry.
+ if (earliestSnapshotEpochSecond <= 0 || latestSnapshotEpochSecond <= 0)
+ {
+ return fresh;
+ }
+ S3ClientConfig fingerprintConfig = fresh.s3Config();
+ String fingerprintBucket = fresh.bucket();
+ if (fingerprintConfig == null || fingerprintBucket == null)
+ {
+ // Mock readers in reflection-driven tests land here.
+ return fresh;
+ }
+ String manifestFingerprint = fresh.getManifestFingerprint(clusterName);
+ if (manifestFingerprint == null || manifestFingerprint.isEmpty())
+ {
+ return fresh;
+ }
+
+ Key key = Key.from(clusterName, keyspace, table, datacenter,
+ earliestSnapshotEpochSecond, latestSnapshotEpochSecond,
+ manifestFingerprint,
+ fingerprintConfig, fingerprintBucket);
+
+ try
+ {
+ // Cache.get(key, loader) is atomic install-or-return; losing-candidate fresh
+ // readers never publish and are GC-eligible immediately on return.
+ BackupReader canonical = CACHE.get(key, () -> {
+ LOGGER.info("ReaderInternCache: installed canonical BackupReader "
+ + "cluster={} keyspace={} table={} datacenter={} earliestEpoch={} latestEpoch={} "
+ + "manifestFingerprint={} region={} bucket={} endpoint={} maxConcurrency={} identity={}",
+ clusterName, keyspace, table, datacenter,
+ earliestSnapshotEpochSecond, latestSnapshotEpochSecond,
+ manifestFingerprint,
+ fingerprintConfig.s3Region(), fingerprintBucket,
+ fingerprintConfig.s3EndpointOverride(),
+ fingerprintConfig.s3HttpMaxConcurrency(),
+ System.identityHashCode(fresh));
+ return fresh;
+ });
+
+ if (canonical != fresh)
+ {
+ LOGGER.debug("ReaderInternCache: reused canonical BackupReader "
+ + "cluster={} keyspace={} table={} datacenter={} earliestEpoch={} latestEpoch={} "
+ + "manifestFingerprint={} region={} bucket={} canonicalIdentity={} discardedFreshIdentity={}",
+ clusterName, keyspace, table, datacenter,
+ earliestSnapshotEpochSecond, latestSnapshotEpochSecond,
+ manifestFingerprint,
+ fingerprintConfig.s3Region(), fingerprintBucket,
+ System.identityHashCode(canonical), System.identityHashCode(fresh));
+
+ // Defense in depth against a future Key regression that aliases buckets.
+ if (!fingerprintBucket.equals(canonical.bucket()))
+ {
+ LOGGER.error("ReaderInternCache: bucket mismatch on canonical reader for "
+ + "key={}. Canonical bucket={} fresh bucket={}. Replacing canonical "
+ + "with fresh reader to avoid wrong-bucket reads.",
+ key, canonical.bucket(), fingerprintBucket);
+ CACHE.put(key, fresh);
+ return fresh;
+ }
+ }
+ return canonical;
+ }
+ catch (java.util.concurrent.ExecutionException e)
+ {
+ // Loader does not throw checked exceptions; unreachable today.
+ throw new RuntimeException("ReaderInternCache loader unexpectedly threw", e.getCause());
+ }
+ }
+
+ @VisibleForTesting
+ static void clearForTesting()
+ {
+ CACHE.invalidateAll();
+ // Drain weak-ref eviction queue so sizeForTesting() is stable.
+ CACHE.cleanUp();
+ }
+
+ @VisibleForTesting
+ static long sizeForTesting()
+ {
+ CACHE.cleanUp();
+ return CACHE.size();
+ }
+
+ private static final class Key
+ {
+ private final String clusterName;
+ private final String keyspace;
+ private final String table;
+ private final String datacenter;
+ private final long earliestSnapshotEpochSecond;
+ private final long latestSnapshotEpochSecond;
+ // SHA-256 over sorted (nodeId, autosnapEpoch) pairs. Disambiguates manifest sets
+ // that share the same (min, max) epochs but differ on a middle node's rotation.
+ private final String manifestFingerprint;
+ private final String s3Region;
+ private final String s3Bucket;
+ @Nullable
+ private final String s3EndpointOverride;
+ // "|", mirroring S3ClientCache.getCacheKey. Null/empty
+ // access keys normalize to "default", empty secrets to hash 0; raw secret never
+ // enters the key. Prod (EMR/IRSA/IMDSv2) collapses to a constant "default|0".
+ private final String s3CredentialsFingerprint;
+ private final int s3HttpMaxConcurrency;
+
+ private Key(String clusterName, String keyspace, String table, String datacenter,
+ long earliestSnapshotEpochSecond, long latestSnapshotEpochSecond,
+ String manifestFingerprint,
+ String s3Region, String s3Bucket, @Nullable String s3EndpointOverride,
+ String s3CredentialsFingerprint, int s3HttpMaxConcurrency)
+ {
+ this.clusterName = clusterName;
+ this.keyspace = keyspace;
+ this.table = table;
+ this.datacenter = datacenter;
+ this.earliestSnapshotEpochSecond = earliestSnapshotEpochSecond;
+ this.latestSnapshotEpochSecond = latestSnapshotEpochSecond;
+ this.manifestFingerprint = manifestFingerprint;
+ this.s3Region = s3Region;
+ this.s3Bucket = s3Bucket;
+ this.s3EndpointOverride = s3EndpointOverride;
+ this.s3CredentialsFingerprint = s3CredentialsFingerprint;
+ this.s3HttpMaxConcurrency = s3HttpMaxConcurrency;
+ }
+
+ static Key from(String clusterName, String keyspace, String table, String datacenter,
+ long earliestSnapshotEpochSecond, long latestSnapshotEpochSecond,
+ String manifestFingerprint,
+ S3ClientConfig s3Config, String bucket)
+ {
+ return new Key(clusterName, keyspace, table, datacenter,
+ earliestSnapshotEpochSecond, latestSnapshotEpochSecond,
+ manifestFingerprint,
+ s3Config.s3Region(), bucket, s3Config.s3EndpointOverride(),
+ credentialsFingerprint(s3Config),
+ s3Config.s3HttpMaxConcurrency());
+ }
+
+ // Mirrors S3ClientCache.getCacheKey credential portion: accessKey|secretHash.
+ private static String credentialsFingerprint(S3ClientConfig s3Config)
+ {
+ String accessKey = s3Config.s3AccessKeyId();
+ String secret = s3Config.s3SecretAccessKey();
+ String normalizedAccessKey = (accessKey != null && !accessKey.isEmpty()) ? accessKey : "default";
+ int secretHash = (secret != null && !secret.isEmpty()) ? secret.hashCode() : 0;
+ return normalizedAccessKey + "|" + secretHash;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o)
+ {
+ return true;
+ }
+ if (!(o instanceof Key))
+ {
+ return false;
+ }
+ Key other = (Key) o;
+ return earliestSnapshotEpochSecond == other.earliestSnapshotEpochSecond
+ && latestSnapshotEpochSecond == other.latestSnapshotEpochSecond
+ && s3HttpMaxConcurrency == other.s3HttpMaxConcurrency
+ && Objects.equals(clusterName, other.clusterName)
+ && Objects.equals(keyspace, other.keyspace)
+ && Objects.equals(table, other.table)
+ && Objects.equals(datacenter, other.datacenter)
+ && Objects.equals(manifestFingerprint, other.manifestFingerprint)
+ && Objects.equals(s3Region, other.s3Region)
+ && Objects.equals(s3Bucket, other.s3Bucket)
+ && Objects.equals(s3EndpointOverride, other.s3EndpointOverride)
+ && Objects.equals(s3CredentialsFingerprint, other.s3CredentialsFingerprint);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(clusterName, keyspace, table, datacenter,
+ earliestSnapshotEpochSecond, latestSnapshotEpochSecond,
+ manifestFingerprint,
+ s3Region, s3Bucket, s3EndpointOverride,
+ s3CredentialsFingerprint, s3HttpMaxConcurrency);
+ }
+
+ @Override
+ public String toString()
+ {
+ // Error-logging only; raw credentials never appear (already a hash).
+ return "ReaderInternCache.Key{cluster=" + clusterName
+ + " keyspace=" + keyspace
+ + " table=" + table
+ + " dc=" + datacenter
+ + " earliestEpoch=" + earliestSnapshotEpochSecond
+ + " latestEpoch=" + latestSnapshotEpochSecond
+ + " manifestFingerprint=" + manifestFingerprint
+ + " region=" + s3Region
+ + " bucket=" + s3Bucket
+ + " endpoint=" + s3EndpointOverride
+ + " credsFingerprint=" + s3CredentialsFingerprint
+ + " maxConcurrency=" + s3HttpMaxConcurrency
+ + "}";
+ }
+ }
+ }
+
+ /**
+ * Test-only: clear the JVM-wide reader intern cache between tests. Production must never
+ * call this — it will not free memory (canonical readers stay referenced by live layers)
+ * and the next deserialization will install a duplicate.
+ */
+ @VisibleForTesting
+ public static void clearReaderInternCacheForTesting()
+ {
+ ReaderInternCache.clearForTesting();
+ }
+
+ @VisibleForTesting
+ public static long readerInternCacheSizeForTesting()
+ {
+ return ReaderInternCache.sizeForTesting();
+ }
+
+ /**
+ * Test-only entry to {@link ReaderInternCache#canonicalize}, bypassing layer construction.
+ * Layer constructors register a Spark shutdown hook that pins the layer (and reader) for
+ * JVM lifetime, which would defeat weak-value GC assertions.
+ *
+ * @param clusterName logical cluster identity
+ * @param keyspace Cassandra keyspace
+ * @param table Cassandra table
+ * @param datacenter datacenter
+ * @param earliestSnapshotEpochSecond earliest contributing snapshot epoch (seconds)
+ * @param latestSnapshotEpochSecond latest contributing snapshot epoch (seconds)
+ * @param fresh freshly-constructed candidate reader to canonicalize
+ * @return the canonical {@link BackupReader} (may be {@code fresh}, or a previously interned instance)
+ */
+ @VisibleForTesting
+ public static BackupReader canonicalizeForTesting(String clusterName,
+ String keyspace,
+ String table,
+ String datacenter,
+ long earliestSnapshotEpochSecond,
+ long latestSnapshotEpochSecond,
+ BackupReader fresh)
+ {
+ return ReaderInternCache.canonicalize(clusterName, keyspace, table, datacenter,
+ earliestSnapshotEpochSecond, latestSnapshotEpochSecond,
+ fresh);
+ }
+
+ private String clusterName;
+ private String keyspace;
+ private String table;
+ private String s3Region;
+ private String s3Bucket;
+ @Nullable
+ private String s3EndpointOverride;
+ @Nullable
+ private String s3AccessKeyId;
+ @Nullable
+ private String s3SecretAccessKey;
+
+ protected transient CassandraBridge bridge;
+
+ private CassandraRing ring;
+ private TokenPartitioner tokenPartitioner;
+ protected CqlTable cqlTable;
+
+ @Nullable
+ protected String lastModifiedTimestampField;
+ @Nullable
+ protected String snapshotTimestampField;
+ protected List requestedFeatures;
+ protected int sstableS3ReadTimeoutSeconds;
+ protected long latestSnapshotEpochSecond;
+
+ // Data.db ranged-GET buffer sizes. Carried as instance fields (not just on S3DataSourceClientConfig)
+ // so they survive Spark serialization to executors.
+ private long dataChunkBufferSize = org.apache.cassandra.spark.utils.Properties.DEFAULT_S3_DATA_CHUNK_BUFFER_SIZE;
+ private long dataMaxBufferSize = org.apache.cassandra.spark.utils.Properties.DEFAULT_S3_DATA_MAX_BUFFER_SIZE;
+
+ // Switch for Data.db ranged-GET delivery. Default true: Data.db reads use the
+ // AsyncResponseTransformer.toPublisher() streaming path. When false,
+ // AsyncResponseTransformer.toBytes() is used (single materialized byte[] per ranged GET).
+ // Non-Data file types and mutable metadata reads always use their existing paths regardless of this
+ // flag. Carried as instance field so it survives Spark serialization to executors.
+ private boolean sstableDataPublisherReadEnabled = true;
+
+ // SSTable metadata cache sizes forwarded to {@code SSTableCache} via JVM sysprops. Carried here so
+ // executor-side deserialization can re-apply them; defaults mirror S3DataSourceClientConfig.
+ private int sstableCacheSummaryMaxEntries = 32768;
+ private int sstableCacheIndexMaxEntries = 16384;
+ private int sstableCacheStatsMaxEntries = 16384;
+ private int sstableCacheFilterMaxEntries = 16384;
+ private int sstableCacheCompressionInfoMaxEntries = 16384;
+
+ private boolean sstableTokenIndexEnabled = false;
+ private int sstableTokenIndexPrebuildPartitions = 0;
+ private int sstableTokenIndexPrebuildPerTaskConcurrency = 4;
+ private transient SSTableTokenIndex sstableTokenIndex;
+
+ private BackupReader s3BackupReader = null;
+ protected transient TimeProvider timeProvider;
+ private transient Stats stats;
+ private S3DataSourceClientConfig s3Config;
+ // Selects the BackupReaderFactory. Non-final so readObject can reassign it on executors.
+ private String backupReaderType;
+
+ public S3CassandraDataLayer(@NotNull S3DataSourceClientConfig config)
+ {
+ super(config.consistencyLevel(), config.datacenter());
+ this.s3Config = config;
+ this.clusterName = config.clusterName();
+ this.keyspace = config.keyspace();
+ this.table = config.table();
+ this.s3Region = config.s3Region();
+ this.s3Bucket = config.s3Bucket();
+ this.s3EndpointOverride = config.s3EndpointOverride();
+ this.s3AccessKeyId = config.s3AccessKeyId();
+ this.s3SecretAccessKey = config.s3SecretAccessKey();
+ this.sstableS3ReadTimeoutSeconds = config.sstableS3ReadTimeoutSeconds();
+ this.dataChunkBufferSize = config.s3DataChunkBufferSize();
+ this.dataMaxBufferSize = config.s3DataMaxBufferSize();
+ this.sstableDataPublisherReadEnabled = config.sstableDataPublisherReadEnabled();
+ this.sstableCacheSummaryMaxEntries = config.sstableCacheSummaryMaxEntries();
+ this.sstableCacheIndexMaxEntries = config.sstableCacheIndexMaxEntries();
+ this.sstableCacheStatsMaxEntries = config.sstableCacheStatsMaxEntries();
+ this.sstableCacheFilterMaxEntries = config.sstableCacheFilterMaxEntries();
+ this.sstableCacheCompressionInfoMaxEntries = config.sstableCacheCompressionInfoMaxEntries();
+ this.sstableTokenIndexEnabled = config.sstableTokenIndexEnabled();
+ this.sstableTokenIndexPrebuildPartitions = config.sstableTokenIndexPrebuildPartitions();
+ this.sstableTokenIndexPrebuildPerTaskConcurrency = config.sstableTokenIndexPrebuildPerTaskConcurrency();
+ this.backupReaderType = config.backupReaderType();
+
+ // Driver-side apply; executor side is covered from readObject / Kryo Serializer.read.
+ applySSTableCacheSystemProperties();
+
+ LOGGER.info("Initializing S3CassandraDataLayer for cluster={}, keyspace={}, table={}, "
+ + "dataChunkBufferSize={} bytes, dataMaxBufferSize={} bytes, "
+ + "sstableDataPublisherReadEnabled={}",
+ clusterName, keyspace, table, dataChunkBufferSize, dataMaxBufferSize,
+ sstableDataPublisherReadEnabled);
+
+ // Initialize stats before initializing s3BackupReader such that stats can be passed to s3BackupReader
+ this.stats = new SparkCustomMetricsStats();
+
+ initializeS3BackupReader();
+
+ // list Cassandra instances in S3 bucket
+ final List instances = s3BackupReader.instances(clusterName, config.keyspace(), config.table(), config.datacenter());
+ // build CassandraRing and TokenPartitioner. Prefer rack-aware authoritative replica
+ // placement from the BackupReader; fall back to the naive (rack-unaware) ring when
+ // none is available. Exceptions from the reader signal a genuine integrity issue and
+ // must surface — see BackupReader#buildRackAwareReplicas for the contract.
+ final Partitioner partitioner = Partitioner.Murmur3Partitioner;
+ final ReplicationFactor rf = config.getParsedReplicationFactor();
+ java.util.Optional>> authoritative =
+ s3BackupReader.buildRackAwareReplicas(clusterName, config.keyspace(), config.table(),
+ config.datacenter(), partitioner, rf, instances);
+ if (authoritative.isPresent())
+ {
+ LOGGER.info("S3CassandraDataLayer: using rack-aware authoritative ring for cluster={}, keyspace={}, table={}, dc={}",
+ clusterName, config.keyspace(), config.table(), config.datacenter());
+ this.ring = new CassandraRing(partitioner, config.keyspace(), rf, instances, authoritative.get());
+ }
+ else
+ {
+ LOGGER.warn("S3CassandraDataLayer: rack-aware replicas unavailable for cluster={}, keyspace={}, table={}, dc={}; "
+ + "falling back to naive (rack-unaware) ring derivation. Cross-DC replicas and local racks "
+ + "will not be honored.",
+ clusterName, config.keyspace(), config.table(), config.datacenter());
+ this.ring = new CassandraRing(partitioner, config.keyspace(), rf, instances);
+ }
+
+ // Calculate effective number of cores using dynamic sizing. config.numberSplits() honors the
+ // optional `number_splits` DataSource option; -1 (DEFAULT_NUM_SPLITS) falls back to the
+ // (defaultParallelism, numCores) formula in TokenPartitioner.
+ int effectiveNumberOfCores = getSizing(rf, config).getEffectiveNumberOfCores();
+ this.tokenPartitioner = new TokenPartitioner(ring,
+ config.numberSplits(),
+ config.defaultParallelism(),
+ effectiveNumberOfCores);
+
+ // build cqlTable based on tableCreateStmt and provided udts.
+ this.bridge = CassandraBridgeFactory.get(config.cassandraVersion());
+ this.cqlTable = bridge().buildSchema(config.tableCreateStmt(), config.keyspace(), rf, partitioner, config.parsedUdts());
+ this.lastModifiedTimestampField = config.lastModifiedTimestampField();
+ this.snapshotTimestampField = config.snapshotTimestampField();
+ this.requestedFeatures = config.requestedFeatures();
+ if (this.lastModifiedTimestampField != null)
+ {
+ CassandraDataLayer.aliasLastModifiedTimestamp(this.requestedFeatures, this.lastModifiedTimestampField);
+ }
+ final long earliestEpoch = s3BackupReader.getSnapshotEpochSecond(clusterName, config.keyspace(), config.table(), config.datacenter());
+ this.latestSnapshotEpochSecond = s3BackupReader.getLatestSnapshotEpochSecond(clusterName, config.keyspace(), config.table(), config.datacenter());
+ this.timeProvider = new S3SnapshotTimeProvider(earliestEpoch);
+ injectSnapshotTimestamp(this.requestedFeatures, this.latestSnapshotEpochSecond, this.snapshotTimestampField);
+
+ // Always assign s3BackupReader via the intern cache so future executor-side refactors
+ // can't reintroduce per-task readers. Must run after epochs are populated above.
+ this.s3BackupReader = ReaderInternCache.canonicalize(clusterName, config.keyspace(), config.table(),
+ config.datacenter(),
+ earliestEpoch, this.latestSnapshotEpochSecond,
+ this.s3BackupReader);
+
+ // Register shutdown hook to clean up S3 resources
+ ShutdownHookManager.addShutdownHook(org.apache.spark.util.ShutdownHookManager.TEMP_DIR_SHUTDOWN_PRIORITY(),
+ ScalaFunctions.wrapLambda(this::shutdownHook));
+ }
+
+ @Override
+ public CassandraBridge bridge()
+ {
+ return bridge;
+ }
+
+ @Override
+ public List requestedFeatures()
+ {
+ return requestedFeatures;
+ }
+
+ // For deserialization
+ @VisibleForTesting
+ // CHECKSTYLE IGNORE: Constructor with many parameters
+ public S3CassandraDataLayer(@NotNull final String clusterName,
+ @NotNull final String keyspace,
+ @NotNull final String table,
+ @NotNull String datacenter,
+ @NotNull String s3Region,
+ @NotNull String s3Bucket,
+ @Nullable String s3EndpointOverride,
+ @Nullable String s3AccessKeyId,
+ @Nullable String s3SecretAccessKey,
+ int sstableS3ReadTimeoutSeconds,
+ @NotNull final TokenPartitioner tokenPartitioner,
+ @NotNull CassandraVersion version,
+ @NotNull final CassandraRing ring,
+ @NotNull final CqlTable cqlTable,
+ @Nullable ConsistencyLevel consistencyLevel,
+ @Nullable String lastModifiedTimestampField,
+ @Nullable String snapshotTimestampField,
+ long latestSnapshotEpochSecond,
+ List requestedFeatures,
+ TimeProvider timeProvider,
+ @NotNull final BackupReader s3BackupReader)
+ {
+ this(clusterName, keyspace, table, datacenter, s3Region, s3Bucket, s3EndpointOverride,
+ s3AccessKeyId, s3SecretAccessKey, sstableS3ReadTimeoutSeconds, tokenPartitioner,
+ version, ring, cqlTable, consistencyLevel, lastModifiedTimestampField,
+ snapshotTimestampField, latestSnapshotEpochSecond, requestedFeatures, timeProvider,
+ s3BackupReader, /* backupReaderType */ "test");
+ }
+
+ // For deserialization with backupReaderType (Kryo path)
+ @VisibleForTesting
+ // CHECKSTYLE IGNORE: Constructor with many parameters
+ public S3CassandraDataLayer(@NotNull final String clusterName,
+ @NotNull final String keyspace,
+ @NotNull final String table,
+ @NotNull String datacenter,
+ @NotNull String s3Region,
+ @NotNull String s3Bucket,
+ @Nullable String s3EndpointOverride,
+ @Nullable String s3AccessKeyId,
+ @Nullable String s3SecretAccessKey,
+ int sstableS3ReadTimeoutSeconds,
+ @NotNull final TokenPartitioner tokenPartitioner,
+ @NotNull CassandraVersion version,
+ @NotNull final CassandraRing ring,
+ @NotNull final CqlTable cqlTable,
+ @Nullable ConsistencyLevel consistencyLevel,
+ @Nullable String lastModifiedTimestampField,
+ @Nullable String snapshotTimestampField,
+ long latestSnapshotEpochSecond,
+ List requestedFeatures,
+ TimeProvider timeProvider,
+ @NotNull final BackupReader s3BackupReader,
+ @NotNull String backupReaderType)
+ {
+ super(consistencyLevel, datacenter);
+
+ // Initialize stats first to ensure it's never null
+ this.stats = new SparkCustomMetricsStats();
+
+ this.clusterName = clusterName;
+ this.keyspace = keyspace;
+ this.table = table;
+ this.bridge = CassandraBridgeFactory.get(version);
+ this.cqlTable = cqlTable;
+ this.tokenPartitioner = tokenPartitioner;
+ this.s3Region = s3Region;
+ this.s3Bucket = s3Bucket;
+ this.s3EndpointOverride = s3EndpointOverride;
+ this.s3AccessKeyId = s3AccessKeyId;
+ this.s3SecretAccessKey = s3SecretAccessKey;
+ this.sstableS3ReadTimeoutSeconds = sstableS3ReadTimeoutSeconds;
+ this.lastModifiedTimestampField = lastModifiedTimestampField;
+ this.snapshotTimestampField = snapshotTimestampField;
+ this.latestSnapshotEpochSecond = latestSnapshotEpochSecond;
+ this.backupReaderType = backupReaderType;
+ this.requestedFeatures = requestedFeatures;
+ if (lastModifiedTimestampField != null)
+ {
+ CassandraDataLayer.aliasLastModifiedTimestamp(this.requestedFeatures, this.lastModifiedTimestampField);
+ }
+ injectSnapshotTimestamp(this.requestedFeatures, this.latestSnapshotEpochSecond, this.snapshotTimestampField);
+ this.ring = ring;
+ this.timeProvider = timeProvider;
+ // Tests pass latestSnapshotEpochSecond=0 to short-circuit canonicalize (keeps mock
+ // readers isolated). Null-guard timeProvider for the same reason.
+ long earliestSnapshotEpochSecond = (timeProvider != null) ? timeProvider.referenceEpochInSeconds() : 0L;
+ this.s3BackupReader = ReaderInternCache.canonicalize(clusterName, keyspace, table, datacenter,
+ earliestSnapshotEpochSecond, latestSnapshotEpochSecond,
+ s3BackupReader);
+
+ // No shutdown hook here: production never reaches this ctor (Spark task closures use
+ // JDK readObject, which doesn't run constructors; Kryo is unused for this layer).
+ // Registering a hook would pin `this` for JVM lifetime and defeat weakValues() on the
+ // canonical reader. The driver-side primary ctor registers exactly one hook per JVM.
+ }
+
+ /**
+ * Replaces the placeholder {@link SchemaFeatureSet#SNAPSHOT_TIMESTAMP} enum entry in the
+ * requested features list with a custom {@link SchemaFeature} instance that carries the
+ * actual latest snapshot epoch and column alias.
+ *
+ * If {@code snapshotTimestampField} is null, the feature is left as-is (using its default
+ * field name {@code "snapshot_timestamp"}).
+ *
+ * @param requestedFeatures the mutable list of requested features
+ * @param latestSnapshotEpoch the latest autosnap epoch in seconds across all nodes
+ * @param snapshotTimestampField the user-supplied column alias, or null for the default name
+ */
+ static void injectSnapshotTimestamp(List requestedFeatures,
+ long latestSnapshotEpoch,
+ @Nullable String snapshotTimestampField)
+ {
+ int index = requestedFeatures.indexOf(SchemaFeatureSet.SNAPSHOT_TIMESTAMP);
+ if (index < 0)
+ {
+ return;
+ }
+
+ final String alias = snapshotTimestampField != null
+ ? snapshotTimestampField
+ : SchemaFeatureSet.SNAPSHOT_TIMESTAMP.fieldName();
+
+ SchemaFeature injected = new SchemaFeature()
+ {
+ @Override
+ public String optionName()
+ {
+ return SchemaFeatureSet.SNAPSHOT_TIMESTAMP.optionName();
+ }
+
+ @Override
+ public String fieldName()
+ {
+ return alias;
+ }
+
+ @Override
+ public DataType fieldDataType()
+ {
+ return DataTypes.TimestampType;
+ }
+
+ @Override
+ public RowBuilder decorate(RowBuilder builder)
+ {
+ return new SnapshotTimestampDecorator<>(builder, alias, latestSnapshotEpoch);
+ }
+
+ @Override
+ public boolean fieldNullable()
+ {
+ return SchemaFeatureSet.SNAPSHOT_TIMESTAMP.fieldNullable();
+ }
+ };
+ requestedFeatures.set(index, injected);
+ }
+
+ private void initializeS3BackupReader()
+ {
+ if (s3BackupReader == null)
+ {
+ this.s3BackupReader = BackupReaderRegistry.create(this.backupReaderType,
+ this.s3Config.toBackupReaderConfig());
+ this.s3BackupReader.initializeSSTableInfoCache(clusterName, keyspace, table, datacenter);
+ }
+ }
+
+ @Override
+ public CqlTable cqlTable()
+ {
+ return cqlTable;
+ }
+
+ @Override
+ public TimeProvider timeProvider()
+ {
+ return timeProvider;
+ }
+
+ public boolean sstableTokenIndexEnabled()
+ {
+ return sstableTokenIndexEnabled;
+ }
+
+ public int sstableTokenIndexPrebuildPartitions(int sparkDefaultParallelism)
+ {
+ return s3Config.resolveSSTableTokenIndexPrebuildPartitions(sstableCountForTokenIndex(), sparkDefaultParallelism);
+ }
+
+ public int sstableTokenIndexPrebuildPerTaskConcurrency()
+ {
+ return sstableTokenIndexPrebuildPerTaskConcurrency;
+ }
+
+ public S3ClientConfig s3ClientConfig()
+ {
+ return s3Config.s3Config();
+ }
+
+ /**
+ * Returns the {@link BackupReader} type this layer was constructed with.
+ *
+ * @return the configured backup reader type
+ */
+ public String backupReaderType()
+ {
+ return backupReaderType;
+ }
+
+ public String clusterName()
+ {
+ return clusterName;
+ }
+
+ public String datacenter()
+ {
+ return datacenter;
+ }
+
+ public int sstableCountForTokenIndex()
+ {
+ return s3BackupReader.sstables(clusterName, keyspace, table, datacenter).size();
+ }
+
+ public List sstableTokenIndexWorkItems()
+ {
+ Map tokenByNode = new HashMap<>();
+ for (CassandraInstance instance : s3BackupReader.instances(clusterName, keyspace, table, datacenter))
+ {
+ tokenByNode.putIfAbsent(instance.nodeName(), instance.token());
+ }
+ Set activeNodeIds = new HashSet<>(tokenByNode.keySet());
+
+ return s3BackupReader.sstables(clusterName, keyspace, table, datacenter)
+ .entrySet()
+ .stream()
+ .filter(entry -> activeNodeIds.contains(entry.getKey().getNodeId()))
+ .map(entry -> new SSTableSummaryWorkItem(entry.getKey(),
+ tokenByNode.getOrDefault(entry.getKey().getNodeId(), ""),
+ entry.getValue()))
+ .collect(Collectors.toList());
+ }
+
+ public void setSSTableTokenIndex(@Nullable SSTableTokenIndex sstableTokenIndex)
+ {
+ this.sstableTokenIndex = sstableTokenIndex;
+ }
+
+ /**
+ * Lists all SSTables for the given partitionId, token range, and instance.
+ *
+ * For now, token range filtering is not applied and happens later during SSTable reading with SparkRangeFilter.
+ * @param partitionId the partition ID to list SSTables for
+ * @param range the range of tokens to filter SSTables
+ * @param instance the Cassandra instance to list SSTables for
+ * @return a CompletableFuture containing a stream of SSTable objects
+ */
+ @Override
+ public CompletableFuture> listInstance(int partitionId,
+ @NotNull Range range,
+ @NotNull CassandraInstance instance)
+ {
+ // list all Data.db files for the specific instance
+ // and create an S3SSTable object per Data.db file
+ String nodeName = instance.nodeName();
+ TokenRange tokenRange = RangeUtils.toTokenRange(range);
+ // One context per listInstance call; carries only the fields S3SSTable / S3SSTableSource need.
+ S3SSTableContext context = newS3SSTableContext(sstableDataPublisherReadEnabled);
+ Stream sstableStream = s3BackupReader.sstables(clusterName, keyspace, table, instance.dataCenter(), nodeName)
+ .entrySet().stream()
+ .filter(entry -> shouldIncludeSSTable(entry.getKey(), tokenRange))
+ .peek(ssTable -> LOGGER.info("Opening SSTable node={} SSTableKey={}", nodeName,
+ ssTable.getKey()))
+ .map(entry -> {
+ SSTableKey sstableKey = entry.getKey();
+ String sstableFileName = sstableKey.getDataFileName();
+ return new S3SSTable(instance.token(), sstableFileName, entry.getValue(), sstableKey,
+ context);
+ });
+
+ return CompletableFuture.completedFuture(sstableStream);
+ }
+
+ private boolean shouldIncludeSSTable(SSTableKey sstableKey, TokenRange tokenRange)
+ {
+ if (sstableTokenIndex == null)
+ {
+ return true;
+ }
+ boolean include = sstableTokenIndex.include(sstableKey, tokenRange);
+ if (!include)
+ {
+ LOGGER.debug("Pruned SSTable by token index SSTableKey={} tokenRange={}", sstableKey, tokenRange);
+ }
+ return include;
+ }
+
+ @Override
+ public CassandraRing ring()
+ {
+ return ring;
+ }
+
+ @Override
+ public TokenPartitioner tokenPartitioner()
+ {
+ return tokenPartitioner;
+ }
+
+ @Override
+ public ReplicationFactor replicationFactor(String keyspace)
+ {
+ return this.ring.replicationFactor();
+ }
+
+ @Override
+ protected ExecutorService executorService()
+ {
+ return ExecutorHolder.EXECUTOR_SERVICE;
+ }
+
+ public String jobId()
+ {
+ throw new NotImplementedException("Cdc has not been implemented for the S3DataLayer");
+ }
+
+ /**
+ * Returns the {@link Sizing} object based on the {@code sizing} option provided by the user,
+ * or {@link DefaultSizing} as the default sizing
+ *
+ * @param replicationFactor the replication factor
+ * @param options the {@link S3DataSourceClientConfig} options
+ * @return the {@link Sizing} object based on the {@code sizing} option provided by the user
+ */
+ protected Sizing getSizing(ReplicationFactor replicationFactor, S3DataSourceClientConfig options)
+ {
+ return S3SizingFactory.create(replicationFactor, options, consistencyLevel, keyspace, table, datacenter, s3BackupReader, clusterName);
+ }
+
+ /**
+ * Override to provide efficient SSTable size calculation using S3 backup metadata.
+ * This avoids the need to iterate through individual SSTable suppliers and directly
+ * uses the cached SSTable information from S3.
+ *
+ * @return OptionalLong containing the total size in bytes of all SSTable Data.db files,
+ * or empty if calculation fails
+ */
+ @Override
+ public OptionalLong calculateTotalSSTableSize()
+ {
+ try
+ {
+ // Get all SSTables for this table from S3 backup reader cache
+ Map> sstables = s3BackupReader.sstables(clusterName, keyspace, table, datacenter);
+
+ long totalSize = 0;
+
+ // Sum up the Data.db file sizes from all SSTables
+ for (Map componentSizes : sstables.values())
+ {
+ Long dataFileSize = componentSizes.get(FileType.DATA);
+ if (dataFileSize != null)
+ {
+ totalSize += dataFileSize;
+ }
+ }
+
+ return OptionalLong.of(totalSize);
+ }
+ catch (Exception e)
+ {
+ // If S3-specific calculation fails, fall back to default implementation
+ return super.calculateTotalSSTableSize();
+ }
+ }
+
+ /**
+ * Override to provide metrics collection for S3-based Cassandra data reading.
+ * `stats` is always initialized in constructors, so no null check needed.
+ */
+ @Override
+ public Stats stats()
+ {
+ return stats;
+ }
+
+ /** DATA uses the configurable size; other FileTypes fall back to per-FileType then global defaults. */
+ private long bufferSizeForChunk(FileType fileType)
+ {
+ if (fileType == FileType.DATA)
+ {
+ return dataChunkBufferSize;
+ }
+ Long override = org.apache.cassandra.spark.utils.Properties.DEFAULT_CHUNK_BUFFER_OVERRIDE.get(fileType);
+ return override != null ? override : org.apache.cassandra.spark.utils.Properties.DEFAULT_CHUNK_BUFFER_SIZE;
+ }
+
+ private long bufferSizeForMax(FileType fileType)
+ {
+ if (fileType == FileType.DATA)
+ {
+ return dataMaxBufferSize;
+ }
+ Long override = org.apache.cassandra.spark.utils.Properties.DEFAULT_MAX_BUFFER_OVERRIDE.get(fileType);
+ return override != null ? override : org.apache.cassandra.spark.utils.Properties.DEFAULT_MAX_BUFFER_SIZE;
+ }
+
+ @VisibleForTesting
+ long dataChunkBufferSize()
+ {
+ return dataChunkBufferSize;
+ }
+
+ @VisibleForTesting
+ long dataMaxBufferSize()
+ {
+ return dataMaxBufferSize;
+ }
+
+ @VisibleForTesting
+ boolean sstableDataPublisherReadEnabled()
+ {
+ return sstableDataPublisherReadEnabled;
+ }
+
+ @VisibleForTesting
+ void setSstableDataPublisherReadEnabledForTesting(boolean enabled)
+ {
+ this.sstableDataPublisherReadEnabled = enabled;
+ }
+
+ @VisibleForTesting
+ BackupReader s3BackupReaderForTesting()
+ {
+ return s3BackupReader;
+ }
+
+ @VisibleForTesting
+ void setS3BackupReaderForTesting(BackupReader reader)
+ {
+ this.s3BackupReader = reader;
+ }
+
+ /** Snapshot the current layer fields into an {@link S3SSTableContext}. */
+ private S3SSTableContext newS3SSTableContext(boolean publisherReadEnabled)
+ {
+ return new S3SSTableContext(clusterName, datacenter, s3BackupReader,
+ dataChunkBufferSize, dataMaxBufferSize,
+ sstableS3ReadTimeoutSeconds, publisherReadEnabled, stats());
+ }
+
+ /**
+ * Test-only factory mirroring {@code listInstance}'s SSTable construction. Lets tests choose the
+ * captured publisher-read flag at construction time without needing to name {@link S3SSTableContext}.
+ *
+ * @param token Cassandra token used for path resolution
+ * @param fileName SSTable file name
+ * @param componentSizes per-component byte sizes
+ * @param sstableKey identifies the SSTable
+ * @param publisherReadEnabled if {@code true}, Data.db reads use publisher transformer
+ * @return a configured {@link S3SSTable} for tests
+ */
+ @VisibleForTesting
+ public S3SSTable newSSTableForTesting(String token,
+ String fileName,
+ Map componentSizes,
+ SSTableKey sstableKey,
+ boolean publisherReadEnabled)
+ {
+ return new S3SSTable(token, fileName, componentSizes, sstableKey,
+ newS3SSTableContext(publisherReadEnabled));
+ }
+
+ /**
+ * Apply SSTable metadata cache sizes as JVM sysprops for {@code SSTableCache} to read at first load.
+ * Operator-set {@code -D} flags win (existing sysprops are preserved). No-op if {@code SSTableCache}
+ * has already been class-loaded in this JVM.
+ */
+ @VisibleForTesting
+ public void applySSTableCacheSystemProperties()
+ {
+ setSysPropIfUnset("sbr.cache.summary.maxEntries", String.valueOf(sstableCacheSummaryMaxEntries));
+ setSysPropIfUnset("sbr.cache.index.maxEntries", String.valueOf(sstableCacheIndexMaxEntries));
+ setSysPropIfUnset("sbr.cache.stats.maxEntries", String.valueOf(sstableCacheStatsMaxEntries));
+ setSysPropIfUnset("sbr.cache.filter.maxEntries", String.valueOf(sstableCacheFilterMaxEntries));
+ setSysPropIfUnset("sbr.cache.compressionInfo.maxEntries", String.valueOf(sstableCacheCompressionInfoMaxEntries));
+ // S3 backup objects are immutable for a given (path, generation), so cached metadata cannot
+ // go stale within a job and TTL only forces wasteful re-fetches (extra S3 GETs + KMS
+ // decrypts). Set a long TTL on every cache for the S3 path; bounded memory is still enforced
+ // by maximumSize() LRU. Non-S3 readers keep upstream's 15 / 60 min TTLs.
+ String s3CacheExpireMins = String.valueOf(TimeUnit.DAYS.toMinutes(1));
+ setSysPropIfUnset("sbr.cache.summary.expireAfterMins", s3CacheExpireMins);
+ setSysPropIfUnset("sbr.cache.index.expireAfterMins", s3CacheExpireMins);
+ setSysPropIfUnset("sbr.cache.stats.expireAfterMins", s3CacheExpireMins);
+ setSysPropIfUnset("sbr.cache.filter.expireAfterMins", s3CacheExpireMins);
+ setSysPropIfUnset("sbr.cache.compressionInfo.expireAfterMins", s3CacheExpireMins);
+ }
+
+ private static void setSysPropIfUnset(String name, String value)
+ {
+ if (System.getProperty(name) == null)
+ {
+ System.setProperty(name, value);
+ }
+ }
+
+ /**
+ * Shutdown hook to clean up resources used by S3CassandraDataLayer.
+ * Closes the s3BackupReader and releases all cached S3 clients.
+ */
+ protected void shutdownHook()
+ {
+ try
+ {
+ if (s3BackupReader != null)
+ {
+ s3BackupReader.close();
+ }
+ S3ClientCache.closeAll();
+ }
+ catch (Exception exception)
+ {
+ LOGGER.warn("Unable to close S3 resources", exception);
+ }
+ }
+
+ public void close()
+ {
+ try
+ {
+ if (s3BackupReader != null)
+ {
+ s3BackupReader.close();
+ }
+ sstableTokenIndex = null;
+ }
+ catch (Exception exception)
+ {
+ LOGGER.warn("Unable to close S3 Cassandra data layer resources", exception);
+ }
+ }
+
+ /**
+ * Immutable bundle of fields that {@link S3SSTable} and {@link S3SSTableSource} need at runtime.
+ * Carrying these in a separate object lets both classes be {@code static}, so cached SSTable
+ * keys in {@code SSTableCache} no longer pin their owning {@link S3CassandraDataLayer} via a
+ * synthetic outer reference.
+ */
+ static final class S3SSTableContext
+ {
+ final String clusterName;
+ final String datacenter;
+ final BackupReader s3BackupReader;
+ final long dataChunkBufferSize;
+ final long dataMaxBufferSize;
+ final int sstableS3ReadTimeoutSeconds;
+ final boolean sstableDataPublisherReadEnabled;
+ // Stats reference (not a resolved BufferingInputStreamStats) preserves per-open resolution.
+ final Stats stats;
+
+ S3SSTableContext(String clusterName,
+ String datacenter,
+ BackupReader s3BackupReader,
+ long dataChunkBufferSize,
+ long dataMaxBufferSize,
+ int sstableS3ReadTimeoutSeconds,
+ boolean sstableDataPublisherReadEnabled,
+ Stats stats)
+ {
+ this.clusterName = clusterName;
+ this.datacenter = datacenter;
+ this.s3BackupReader = s3BackupReader;
+ this.dataChunkBufferSize = dataChunkBufferSize;
+ this.dataMaxBufferSize = dataMaxBufferSize;
+ this.sstableS3ReadTimeoutSeconds = sstableS3ReadTimeoutSeconds;
+ this.sstableDataPublisherReadEnabled = sstableDataPublisherReadEnabled;
+ this.stats = stats;
+ }
+
+ long bufferSizeForChunk(FileType fileType)
+ {
+ if (fileType == FileType.DATA)
+ {
+ return dataChunkBufferSize;
+ }
+ Long override = org.apache.cassandra.spark.utils.Properties.DEFAULT_CHUNK_BUFFER_OVERRIDE.get(fileType);
+ return override != null ? override : org.apache.cassandra.spark.utils.Properties.DEFAULT_CHUNK_BUFFER_SIZE;
+ }
+
+ long bufferSizeForMax(FileType fileType)
+ {
+ if (fileType == FileType.DATA)
+ {
+ return dataMaxBufferSize;
+ }
+ Long override = org.apache.cassandra.spark.utils.Properties.DEFAULT_MAX_BUFFER_OVERRIDE.get(fileType);
+ return override != null ? override : org.apache.cassandra.spark.utils.Properties.DEFAULT_MAX_BUFFER_SIZE;
+ }
+ }
+
+ public static class S3SSTable extends SSTable
+ {
+ private final String token;
+ private final String fileName;
+ private final Map componentSizes;
+ private final SSTableKey sstableKey;
+ private final S3SSTableContext context;
+ private final ConcurrentMap actualComponentSizes = new ConcurrentHashMap<>();
+
+ // Package-private: only listInstance(...) and newSSTableForTesting(...) construct these.
+ S3SSTable(String token,
+ String fileName,
+ Map componentSizes,
+ SSTableKey sstableKey,
+ @NotNull S3SSTableContext context)
+ {
+ this.token = token;
+ this.fileName = fileName;
+ this.componentSizes = componentSizes;
+ this.sstableKey = sstableKey;
+ this.context = context;
+ }
+
+ @Nullable
+ protected InputStream openInputStream(FileType fileType)
+ {
+ // open an InputStream on the SSTable file component
+ final Long size = componentSizes.get(fileType);
+ if (size == null)
+ {
+ // file doesn't exist
+ return null;
+ }
+ // using the SSTableInputStream allows us to open many SSTables without OOMing
+ // by buffering and requesting more on demand
+ return new BufferingInputStream<>(new S3SSTableSource(this, fileType, size, context),
+ context.stats.bufferingInputStreamStats());
+ }
+
+ @VisibleForTesting
+ public CassandraFileSource newSourceForTesting(FileType fileType, long size)
+ {
+ return new S3SSTableSource(this, fileType, size, context);
+ }
+
+ /**
+ * Cross-package test hook delegating to the protected {@link #openInputStream(FileType)}.
+ * Production code should use the {@code BufferingInputStream} accessors on
+ * {@link SSTable} (e.g. {@code openSummaryStream()}) instead.
+ *
+ * @param fileType SSTable component to open
+ * @return an {@link InputStream} for the requested component
+ */
+ @VisibleForTesting
+ public InputStream openInputStreamForTesting(FileType fileType)
+ {
+ return openInputStream(fileType);
+ }
+
+ public long length(FileType fileType)
+ {
+ Long actualSize = actualComponentSizes.get(fileType);
+ if (actualSize != null)
+ {
+ return actualSize;
+ }
+ final Long size = componentSizes.get(fileType);
+ if (size == null)
+ {
+ throw new IncompleteSSTableException(fileType);
+ }
+ return size;
+ }
+
+ public boolean isMissing(FileType fileType)
+ {
+ if (componentSizes != null)
+ {
+ return !componentSizes.containsKey(fileType);
+ }
+ return !context.s3BackupReader.exists(context.clusterName, context.datacenter, token, sstableKey, fileType, context.stats);
+ }
+
+ public String getDataFileName()
+ {
+ return fileName;
+ }
+
+ public int hashCode()
+ {
+ return Objects.hash(token, fileName, sstableKey);
+ }
+
+ public boolean equals(Object obj)
+ {
+ if (obj == null)
+ {
+ return false;
+ }
+ if (obj == this)
+ {
+ return true;
+ }
+ if (obj.getClass() != getClass())
+ {
+ return false;
+ }
+
+ final S3SSTable rhs = (S3SSTable) obj;
+ return token.equals(rhs.token)
+ && fileName.equals(rhs.fileName)
+ && sstableKey.equals(rhs.sstableKey);
+ }
+ }
+
+ /**
+ * Async data source for streaming bytes from a single SSTable component to BufferingInputStream.
+ * Sibling of {@link S3SSTable} so both can be {@code static} (no synthetic outer reference) while
+ * still sharing the same package-private surface.
+ */
+ private static class S3SSTableSource implements CassandraFileSource
+ {
+ private final S3SSTable ssTable;
+ private final FileType fileType;
+ private final long manifestSize;
+ private final S3SSTableContext context;
+ private volatile long size;
+ private volatile boolean actualSizeResolved;
+
+ S3SSTableSource(S3SSTable ssTable, FileType fileType, long size, S3SSTableContext context)
+ {
+ this.ssTable = ssTable;
+ this.fileType = fileType;
+ this.manifestSize = size;
+ this.context = context;
+ Long actualSize = ssTable.actualComponentSizes.get(fileType);
+ this.actualSizeResolved = actualSize != null;
+ this.size = actualSize == null ? size : actualSize;
+ }
+
+ public void request(long start, long end, StreamConsumer consumer)
+ {
+ // Mutable metadata (Summary.db, Filter.db, Statistics.db) supports
+ // size-drift handling for stale autosnap manifests.
+ if (fileType.isMutableMetadata())
+ {
+ if (actualSizeResolved)
+ {
+ if (start >= size)
+ {
+ consumer.onEnd();
+ return;
+ }
+ context.s3BackupReader.getAsync(context.clusterName,
+ context.datacenter,
+ ssTable.token,
+ ssTable.sstableKey,
+ fileType,
+ start,
+ Math.min(end, size - 1),
+ consumer,
+ context.stats);
+ return;
+ }
+ context.s3BackupReader.getMutableMetadataAsync(context.clusterName,
+ context.datacenter,
+ ssTable.token,
+ ssTable.sstableKey,
+ fileType,
+ start,
+ end,
+ consumer,
+ this::setActualSize,
+ manifestSize,
+ context.stats);
+ return;
+ }
+
+ // Data.db ranged GETs go to the toBytes() path by default and only opt
+ // into the toPublisher() streaming path when sstableDataPublisherReadEnabled=true.
+ // All non-Data immutable components (Index.db, CompressionInfo.db, etc.) continue to
+ // use the streaming path regardless of the flag.
+ if (fileType == FileType.DATA && !context.sstableDataPublisherReadEnabled)
+ {
+ // Timeout semantics for the toBytes() path:
+ // BufferingInputStream's no-activity poll timeout (sstableS3ReadTimeoutSeconds,
+ // default 600s) covers the entire chunk materialization window because onRead is
+ // invoked exactly once per ranged GET (after the byte[] is fully assembled).
+ // Stalled sockets are caught earlier by the AWS SDK's NettyNioAsyncHttpClient
+ // readTimeout (S3ClientCache.READ_TIMEOUT_SECONDS=120s) and surfaced here via
+ // consumer.onError, so the BufferingInputStream poll timeout is the secondary
+ // safety net rather than the primary stall detector.
+ context.s3BackupReader.readAsync(context.clusterName,
+ context.datacenter,
+ ssTable.token,
+ ssTable.sstableKey,
+ fileType,
+ start,
+ end,
+ context.stats)
+ .whenComplete((bytes, throwable) -> {
+ if (throwable != null)
+ {
+ consumer.onError(throwable);
+ return;
+ }
+ try
+ {
+ // BufferingInputStream expects onRead(...) followed by onEnd().
+ // wrap() does not copy: it adopts the byte[] reference.
+ consumer.onRead(org.apache.cassandra.spark.utils.streaming.StreamBuffer.wrap(bytes));
+ consumer.onEnd();
+ }
+ catch (Throwable forwardErr)
+ {
+ consumer.onError(forwardErr);
+ }
+ });
+ return;
+ }
+
+ context.s3BackupReader.getAsync(context.clusterName, context.datacenter, ssTable.token,
+ ssTable.sstableKey, fileType, start, end, consumer, context.stats);
+ }
+
+ public S3SSTable cassandraFile()
+ {
+ return ssTable;
+ }
+
+ public FileType fileType()
+ {
+ return fileType;
+ }
+
+ public long size()
+ {
+ return size;
+ }
+
+ private void setActualSize(long actualSize)
+ {
+ if (actualSize > 0L)
+ {
+ size = actualSize;
+ actualSizeResolved = true;
+ ssTable.actualComponentSizes.put(fileType, actualSize);
+ }
+ }
+
+ @Override
+ public long chunkBufferSize()
+ {
+ return context.bufferSizeForChunk(fileType);
+ }
+
+ @Override
+ public long maxBufferSize()
+ {
+ return context.bufferSizeForMax(fileType);
+ }
+
+ @Nullable
+ @Override
+ public Duration timeout()
+ {
+ return context.sstableS3ReadTimeoutSeconds > 0
+ ? Duration.ofSeconds(context.sstableS3ReadTimeoutSeconds)
+ : null;
+ }
+ }
+
+ // jdk serialization
+
+ @Nullable
+ private static String readNullable(ObjectInputStream in) throws IOException
+ {
+ if (in.readBoolean())
+ {
+ return in.readUTF();
+ }
+ return null;
+ }
+
+ private static void writeNullable(ObjectOutputStream out, @Nullable String string) throws IOException
+ {
+ if (string == null)
+ {
+ out.writeBoolean(false);
+ }
+ else
+ {
+ out.writeBoolean(true);
+ out.writeUTF(string);
+ }
+ }
+
+ private void readObject(final ObjectInputStream in) throws IOException, ClassNotFoundException
+ {
+ LOGGER.info("Falling back to JDK deserialization");
+ this.clusterName = in.readUTF();
+ this.keyspace = in.readUTF();
+ this.table = in.readUTF();
+ this.bridge = CassandraBridgeFactory.get(CassandraVersion.valueOf(in.readUTF()));
+
+ this.s3Region = in.readUTF();
+ this.s3Bucket = in.readUTF();
+ this.s3EndpointOverride = readNullable(in);
+ this.s3AccessKeyId = readNullable(in);
+ this.s3SecretAccessKey = readNullable(in);
+ this.sstableS3ReadTimeoutSeconds = in.readInt();
+ this.dataChunkBufferSize = in.readLong();
+ this.dataMaxBufferSize = in.readLong();
+ this.sstableCacheSummaryMaxEntries = in.readInt();
+ this.sstableCacheIndexMaxEntries = in.readInt();
+ this.sstableCacheStatsMaxEntries = in.readInt();
+ this.sstableCacheFilterMaxEntries = in.readInt();
+ this.sstableCacheCompressionInfoMaxEntries = in.readInt();
+ this.sstableTokenIndexEnabled = in.readBoolean();
+ this.sstableTokenIndexPrebuildPartitions = in.readInt();
+ this.sstableTokenIndexPrebuildPerTaskConcurrency = in.readInt();
+ this.sstableDataPublisherReadEnabled = in.readBoolean();
+ this.backupReaderType = in.readUTF();
+
+ this.cqlTable = bridge.javaDeserialize(in, CqlTable.class); // Delegate (de-)serialization of version-specific objects to the Cassandra Bridge
+ this.tokenPartitioner = (TokenPartitioner) in.readObject();
+ this.ring = (CassandraRing) in.readObject();
+ this.lastModifiedTimestampField = readNullable(in);
+ this.snapshotTimestampField = readNullable(in);
+ this.latestSnapshotEpochSecond = in.readLong();
+ int features = in.readShort();
+ List requestedFeatures = new ArrayList<>(features);
+ for (int feature = 0; feature < features; feature++)
+ {
+ String featureName = in.readUTF();
+ requestedFeatures.add(SchemaFeatureSet.valueOf(featureName.toUpperCase()));
+ }
+ this.requestedFeatures = requestedFeatures;
+ if (this.lastModifiedTimestampField != null)
+ {
+ CassandraDataLayer.aliasLastModifiedTimestamp(this.requestedFeatures, this.lastModifiedTimestampField);
+ }
+ injectSnapshotTimestamp(this.requestedFeatures, this.latestSnapshotEpochSecond, this.snapshotTimestampField);
+ this.timeProvider = new S3SnapshotTimeProvider(in.readLong());
+ // The executor JVM must have the concrete BackupReader implementation on its classpath.
+ BackupReader deserializedReader = (BackupReader) in.readObject();
+ // Canonicalize per task so all tasks on this executor share one reader per manifest.
+ // The discarded fresh reader is reclaimed on the next GC.
+ if (deserializedReader != null)
+ {
+ this.s3BackupReader = ReaderInternCache.canonicalize(this.clusterName, this.keyspace, this.table,
+ this.datacenter,
+ this.timeProvider.referenceEpochInSeconds(),
+ this.latestSnapshotEpochSecond,
+ deserializedReader);
+ }
+
+ // Mirror constructor: re-apply cache sysprops so executor-side cache sizes match driver.
+ applySSTableCacheSystemProperties();
+
+ // Recreate the executor-local Spark metrics sink.
+ this.stats = new SparkCustomMetricsStats();
+ }
+
+ private void writeObject(final ObjectOutputStream out) throws IOException, ClassNotFoundException
+ {
+ LOGGER.info("Falling back to JDK serialization");
+ out.writeUTF(this.clusterName);
+ out.writeUTF(this.keyspace);
+ out.writeUTF(this.table);
+ out.writeUTF(this.version().name());
+ out.writeUTF(this.s3Region);
+ out.writeUTF(this.s3Bucket);
+ writeNullable(out, this.s3EndpointOverride);
+ writeNullable(out, this.s3AccessKeyId);
+ writeNullable(out, this.s3SecretAccessKey);
+ out.writeInt(this.sstableS3ReadTimeoutSeconds);
+ out.writeLong(this.dataChunkBufferSize);
+ out.writeLong(this.dataMaxBufferSize);
+ out.writeInt(this.sstableCacheSummaryMaxEntries);
+ out.writeInt(this.sstableCacheIndexMaxEntries);
+ out.writeInt(this.sstableCacheStatsMaxEntries);
+ out.writeInt(this.sstableCacheFilterMaxEntries);
+ out.writeInt(this.sstableCacheCompressionInfoMaxEntries);
+ out.writeBoolean(this.sstableTokenIndexEnabled);
+ out.writeInt(this.sstableTokenIndexPrebuildPartitions);
+ out.writeInt(this.sstableTokenIndexPrebuildPerTaskConcurrency);
+ out.writeBoolean(this.sstableDataPublisherReadEnabled);
+ out.writeUTF(this.backupReaderType);
+ bridge.javaSerialize(out, this.cqlTable); // Delegate (de-)serialization of version-specific objects to the Cassandra Bridge
+ out.writeObject(this.tokenPartitioner);
+ out.writeObject(this.ring);
+ writeNullable(out, this.lastModifiedTimestampField);
+ writeNullable(out, this.snapshotTimestampField);
+ out.writeLong(this.latestSnapshotEpochSecond);
+ // Write the list of requested features: first write the size, then write the feature names
+ out.writeShort(this.requestedFeatures.size());
+ for (SchemaFeature feature : requestedFeatures)
+ {
+ out.writeUTF(feature.optionName());
+ }
+ out.writeLong(timeProvider.referenceEpochInSeconds());
+ // Ensure s3BackupReader is initialized before serialization
+ if (this.s3BackupReader == null)
+ {
+ initializeS3BackupReader();
+ }
+ // Carry the reader (with its populated manifest cache) over to the executor.
+ out.writeObject(this.s3BackupReader);
+ }
+
+ // Kryo serialization
+
+ public static class Serializer extends com.esotericsoftware.kryo.Serializer
+ {
+ @Override
+ public void write(final Kryo kryo, final Output out, final S3CassandraDataLayer obj)
+ {
+ LOGGER.info("Serializing S3CassandraDataLayer with Kryo");
+ out.writeString(obj.clusterName);
+ out.writeString(obj.keyspace);
+ out.writeString(obj.table);
+ out.writeString(obj.datacenter);
+ out.writeString(obj.s3Region);
+ out.writeString(obj.s3Bucket);
+ kryo.writeObjectOrNull(out, obj.s3EndpointOverride, String.class);
+ kryo.writeObjectOrNull(out, obj.s3AccessKeyId, String.class);
+ kryo.writeObjectOrNull(out, obj.s3SecretAccessKey, String.class);
+ out.writeInt(obj.sstableS3ReadTimeoutSeconds);
+ kryo.writeObject(out, obj.tokenPartitioner);
+ kryo.writeObject(out, obj.version());
+ kryo.writeObject(out, obj.ring);
+ kryo.writeObject(out, obj.cqlTable);
+ kryo.writeObject(out, obj.consistencyLevel);
+ kryo.writeObjectOrNull(out, obj.lastModifiedTimestampField, String.class);
+ kryo.writeObjectOrNull(out, obj.snapshotTimestampField, String.class);
+ out.writeLong(obj.latestSnapshotEpochSecond);
+ // Write the list of requested features: first write the size, then write the feature names
+ S3CassandraDataLayer.Serializer.SchemaFeaturesListWrapper listWrapper = new S3CassandraDataLayer.Serializer.SchemaFeaturesListWrapper();
+ listWrapper.requestedFeatureNames = obj.requestedFeatures.stream()
+ .map(SchemaFeature::optionName)
+ .collect(Collectors.toList());
+ kryo.writeObject(out, listWrapper);
+ out.writeLong(obj.timeProvider.referenceEpochInSeconds());
+ // Ensure s3BackupReader is initialized before serialization
+ if (obj.s3BackupReader == null)
+ {
+ obj.initializeS3BackupReader();
+ }
+ // Polymorphic write so core does not depend on a specific BackupReader subclass.
+ // This Kryo path is currently unused at runtime (closure serialization is Java).
+ kryo.writeClassAndObject(out, obj.s3BackupReader);
+ out.writeString(obj.backupReaderType);
+
+ // Trailing fields (set on the layer post-construction in read()) — the @VisibleForTesting
+ // constructor below is frozen and does not accept them.
+ out.writeLong(obj.dataChunkBufferSize);
+ out.writeLong(obj.dataMaxBufferSize);
+ out.writeInt(obj.sstableCacheSummaryMaxEntries);
+ out.writeInt(obj.sstableCacheIndexMaxEntries);
+ out.writeInt(obj.sstableCacheStatsMaxEntries);
+ out.writeInt(obj.sstableCacheFilterMaxEntries);
+ out.writeInt(obj.sstableCacheCompressionInfoMaxEntries);
+ out.writeBoolean(obj.sstableTokenIndexEnabled);
+ out.writeInt(obj.sstableTokenIndexPrebuildPartitions);
+ out.writeInt(obj.sstableTokenIndexPrebuildPerTaskConcurrency);
+ out.writeBoolean(obj.sstableDataPublisherReadEnabled);
+ }
+
+ @Override
+ public S3CassandraDataLayer read(final Kryo kryo, final Input in, final Class type)
+ {
+ LOGGER.info("Deserializing S3CassandraDataLayer with Kryo");
+ String clusterName = in.readString();
+ String keyspace = in.readString();
+ String table = in.readString();
+ String datacenter = in.readString();
+ String s3Region = in.readString();
+ String s3Bucket = in.readString();
+ String s3EndpointOverride = kryo.readObjectOrNull(in, String.class);
+ String s3AccessKeyId = kryo.readObjectOrNull(in, String.class);
+ String s3SecretAccessKey = kryo.readObjectOrNull(in, String.class);
+ int sstableS3ReadTimeoutSeconds = in.readInt();
+ TokenPartitioner tokenPartitioner = kryo.readObject(in, TokenPartitioner.class);
+ CassandraVersion version = kryo.readObject(in, CassandraVersion.class);
+ CassandraRing ring = kryo.readObject(in, CassandraRing.class);
+ CqlTable cqlTable = kryo.readObject(in, CqlTable.class);
+ ConsistencyLevel consistencyLevel = kryo.readObject(in, ConsistencyLevel.class);
+ String lastModifiedTimestampField = kryo.readObjectOrNull(in, String.class);
+ String snapshotTimestampField = kryo.readObjectOrNull(in, String.class);
+ long latestSnapshotEpochSecond = in.readLong();
+ List requestedFeatures = kryo.readObject(in, S3CassandraDataLayer.Serializer.SchemaFeaturesListWrapper.class).toList();
+ TimeProvider timeProvider = new S3SnapshotTimeProvider(in.readLong());
+ BackupReader s3BackupReader = (BackupReader) kryo.readClassAndObject(in);
+ String backupReaderType = in.readString();
+
+ S3CassandraDataLayer layer = new S3CassandraDataLayer(
+ clusterName, keyspace, table, datacenter, s3Region, s3Bucket,
+ s3EndpointOverride, s3AccessKeyId, s3SecretAccessKey, sstableS3ReadTimeoutSeconds,
+ tokenPartitioner, version, ring, cqlTable, consistencyLevel,
+ lastModifiedTimestampField, snapshotTimestampField, latestSnapshotEpochSecond,
+ requestedFeatures, timeProvider, s3BackupReader, backupReaderType);
+
+ layer.dataChunkBufferSize = in.readLong();
+ layer.dataMaxBufferSize = in.readLong();
+ layer.sstableCacheSummaryMaxEntries = in.readInt();
+ layer.sstableCacheIndexMaxEntries = in.readInt();
+ layer.sstableCacheStatsMaxEntries = in.readInt();
+ layer.sstableCacheFilterMaxEntries = in.readInt();
+ layer.sstableCacheCompressionInfoMaxEntries = in.readInt();
+ layer.sstableTokenIndexEnabled = in.readBoolean();
+ layer.sstableTokenIndexPrebuildPartitions = in.readInt();
+ layer.sstableTokenIndexPrebuildPerTaskConcurrency = in.readInt();
+ layer.sstableDataPublisherReadEnabled = in.readBoolean();
+
+ // Executor-side equivalent of the constructor's apply.
+ layer.applySSTableCacheSystemProperties();
+ return layer;
+ }
+
+ // Wrapper only used internally for Kryo serialization/deserialization
+ private static class SchemaFeaturesListWrapper
+ {
+ public List requestedFeatureNames; // CHECKSTYLE IGNORE: Public mutable field
+
+ public List toList()
+ {
+ return requestedFeatureNames.stream()
+ .map(name -> SchemaFeatureSet.valueOf(name.toUpperCase()))
+ .collect(Collectors.toList());
+ }
+ }
+ }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/S3ClientCache.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/S3ClientCache.java
new file mode 100644
index 000000000..ee3a85a3e
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/S3ClientCache.java
@@ -0,0 +1,586 @@
+/*
+ * 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.net.URI;
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.SparkEnv;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.s3.S3AsyncClient;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.transfer.s3.S3TransferManager;
+
+/**
+ * Unified cache for S3 clients and transfer managers.
+ *
+ * This class consolidates all S3 client creation across the codebase into a single
+ * cached location. Clients are keyed by region|endpoint|accessKeyId|secretHash for
+ * proper isolation across different configurations.
+ *
+ * Key design decisions:
+ *
+ * - No JVM shutdown hooks - problematic in Spark executors. Use explicit {@link #closeAll()}
+ * - Thread-safe with ConcurrentHashMap
+ * - Clients should NOT be closed by callers - cache manages lifecycle
+ *
+ */
+public final class S3ClientCache
+{
+ private static final Logger LOGGER = LoggerFactory.getLogger(S3ClientCache.class);
+ static final int MAX_CONCURRENCY_PER_TASK_SLOT = 16;
+ static final int MAX_CONCURRENCY_FLOOR = 128;
+ static final int MAX_CONCURRENCY_CEILING = 1024;
+ static final int MAX_PENDING_ACQUIRES_PER_CONNECTION = 2;
+ static final int MAX_PENDING_ACQUIRES_FLOOR = 256;
+ static final int MAX_PENDING_ACQUIRES_CEILING = 2048;
+ static final int CONNECTION_ACQUISITION_TIMEOUT_SECONDS = 60;
+ static final int CONNECTION_TIMEOUT_SECONDS = 5;
+ static final int READ_TIMEOUT_SECONDS = 120;
+ static final int CONNECTION_MAX_IDLE_TIME_SECONDS = 60;
+ static final String NETTY_DNS_RESOLVER_CLASS = "io.netty.resolver.dns.DnsAddressResolverGroup";
+ private static final boolean NON_BLOCKING_DNS_RESOLVER_AVAILABLE = isClassAvailable(NETTY_DNS_RESOLVER_CLASS);
+ private static volatile boolean loggedTaskSlotSource = false;
+
+ // Separate caches for different client types
+ private static final ConcurrentHashMap SYNC_CLIENT_CACHE = new ConcurrentHashMap<>();
+ private static final ConcurrentHashMap ASYNC_CLIENT_CACHE = new ConcurrentHashMap<>();
+ private static final ConcurrentHashMap TRANSFER_MANAGER_CACHE = new ConcurrentHashMap<>();
+
+ private S3ClientCache()
+ {
+ // Static utility class
+ }
+
+ // ========================================================================
+ // Public API
+ // ========================================================================
+
+ /**
+ * Get or create a cached synchronous S3Client for the given config.
+ *
+ * @param config S3ClientConfig with region, endpoint, and credentials
+ * @return Cached or newly created S3Client (do NOT close - cache manages lifecycle)
+ */
+ public static S3Client getS3Client(S3ClientConfig config)
+ {
+ String key = getCacheKey(config);
+ return SYNC_CLIENT_CACHE.computeIfAbsent(key, k -> {
+ LOGGER.info("Creating new S3Client for key: {}", k);
+ return buildS3Client(config);
+ });
+ }
+
+ /**
+ * Get or create a cached asynchronous S3AsyncClient for the given config.
+ *
+ * @param config S3ClientConfig with region, endpoint, and credentials
+ * @return Cached or newly created S3AsyncClient (do NOT close - cache manages lifecycle)
+ */
+ public static S3AsyncClient getS3AsyncClient(S3ClientConfig config)
+ {
+ ResolvedAsyncHttpConfig resolved = resolveAsyncHttpConfig(config);
+ String key = getAsyncCacheKey(config, resolved);
+ return ASYNC_CLIENT_CACHE.computeIfAbsent(key, k -> {
+ LOGGER.info("Creating new S3AsyncClient for key: {}", k);
+ return buildS3AsyncClient(config, resolved);
+ });
+ }
+
+ /**
+ * Get or create a cached S3TransferManager for the given config.
+ * TransferManager uses the async client internally for parallel downloads.
+ *
+ * @param config S3ClientConfig with region, endpoint, and credentials
+ * @return Cached or newly created S3TransferManager (do NOT close - cache manages lifecycle)
+ */
+ public static S3TransferManager getTransferManager(S3ClientConfig config)
+ {
+ ResolvedAsyncHttpConfig resolved = resolveAsyncHttpConfig(config);
+ String key = getAsyncCacheKey(config, resolved);
+ return TRANSFER_MANAGER_CACHE.computeIfAbsent(key, k -> {
+ LOGGER.info("Creating new S3TransferManager for key: {}", k);
+ // TransferManager wraps an async client
+ S3AsyncClient asyncClient = getS3AsyncClient(config);
+ return S3TransferManager.builder()
+ .s3Client(asyncClient)
+ .build();
+ });
+ }
+
+ // ========================================================================
+ // Lifecycle Management
+ // ========================================================================
+
+ /**
+ * Close all cached clients and clear the caches.
+ * Should be called from the driver's stop() method.
+ */
+ public static void closeAll()
+ {
+ LOGGER.info("Closing all S3 client caches (sync={}, async={}, transfer={})",
+ SYNC_CLIENT_CACHE.size(), ASYNC_CLIENT_CACHE.size(), TRANSFER_MANAGER_CACHE.size());
+
+ // Collect all unique keys across all caches
+ Set allKeys = new HashSet<>();
+ allKeys.addAll(TRANSFER_MANAGER_CACHE.keySet());
+ allKeys.addAll(ASYNC_CLIENT_CACHE.keySet());
+ allKeys.addAll(SYNC_CLIENT_CACHE.keySet());
+
+ // Close clients for each key (reuses closeClientsForKey logic)
+ for (String key : allKeys)
+ {
+ closeClientsForKey(key);
+ }
+ }
+
+ /**
+ * Close cached clients for a specific config and remove from cache.
+ *
+ * The sync client is keyed only by the base S3 identity. Async clients and transfer managers add
+ * the resolved HTTP signature to their keys, so this evicts all async HTTP variants for the same
+ * base identity rather than only the variant that would be resolved by the current JVM.
+ *
+ * @param config The config whose clients should be closed
+ */
+ public static void close(S3ClientConfig config)
+ {
+ String baseKey = getCacheKey(config);
+ closeClientsForBaseKey(baseKey);
+ }
+
+ // ========================================================================
+ // Key Generation
+ // ========================================================================
+
+ /**
+ * Generate a cache key from the S3 config.
+ * Key format: region|endpoint|accessKeyId|secretHash
+ *
+ * Uses String.hashCode() for secret which is deterministic based on content.
+ * Empty strings for credentials are normalized to "default" to match
+ * getCredentialsProvider() behavior which treats empty strings the same as null.
+ *
+ * @param config S3ClientConfig
+ * @return Cache key string
+ */
+ static String getCacheKey(S3ClientConfig config)
+ {
+ String accessKey = config.s3AccessKeyId();
+ String secret = config.s3SecretAccessKey();
+ // Normalize empty strings to match getCredentialsProvider() behavior
+ String normalizedAccessKey = (accessKey != null && !accessKey.isEmpty()) ? accessKey : "default";
+ int secretHash = (secret != null && !secret.isEmpty()) ? secret.hashCode() : 0;
+ return config.s3Region() + "|" +
+ Objects.toString(config.s3EndpointOverride(), "") + "|" +
+ normalizedAccessKey + "|" +
+ secretHash;
+ }
+
+ static String getAsyncCacheKey(S3ClientConfig config)
+ {
+ return getAsyncCacheKey(config, resolveAsyncHttpConfig(config));
+ }
+
+ private static String getAsyncCacheKey(S3ClientConfig config, ResolvedAsyncHttpConfig resolved)
+ {
+ return getCacheKey(config)
+ + "|async|maxConcurrency=" + resolved.maxConcurrency
+ + "|pending=" + resolved.maxPendingConnectionAcquires
+ + "|acquireTimeout=" + CONNECTION_ACQUISITION_TIMEOUT_SECONDS
+ + "|connectTimeout=" + CONNECTION_TIMEOUT_SECONDS
+ + "|readTimeout=" + READ_TIMEOUT_SECONDS
+ + "|idleTime=" + CONNECTION_MAX_IDLE_TIME_SECONDS
+ + "|tcpKeepAlive=true|dns=nonBlocking";
+ }
+
+ // ========================================================================
+ // Testing Support
+ // ========================================================================
+
+ /**
+ * Reset all caches. For testing only.
+ */
+ static void reset()
+ {
+ closeAll();
+ loggedTaskSlotSource = false;
+ }
+
+ /**
+ * Get total cache size across all client types.
+ *
+ * @return Number of unique keys in the cache (max of sync, async, and transfer manager caches)
+ */
+ static int cacheSize()
+ {
+ return Math.max(SYNC_CLIENT_CACHE.size(),
+ Math.max(ASYNC_CLIENT_CACHE.size(), TRANSFER_MANAGER_CACHE.size()));
+ }
+
+ /**
+ * Check if a key exists in any cache.
+ *
+ * @param config Config to check
+ * @return true if cached
+ */
+ static boolean isCached(S3ClientConfig config)
+ {
+ String baseKey = getCacheKey(config);
+ return hasKeyForBase(SYNC_CLIENT_CACHE.keySet(), baseKey) ||
+ hasKeyForBase(ASYNC_CLIENT_CACHE.keySet(), baseKey) ||
+ hasKeyForBase(TRANSFER_MANAGER_CACHE.keySet(), baseKey);
+ }
+
+ // ========================================================================
+ // Private Implementation
+ // ========================================================================
+
+ private static S3Client buildS3Client(S3ClientConfig config)
+ {
+ AwsCredentialsProvider credentialsProvider = getCredentialsProvider(config);
+
+ software.amazon.awssdk.services.s3.S3ClientBuilder builder = S3Client.builder()
+ .region(Region.of(config.s3Region()))
+ .credentialsProvider(credentialsProvider);
+
+ if (config.s3EndpointOverride() != null && !config.s3EndpointOverride().trim().isEmpty())
+ {
+ builder.endpointOverride(URI.create(config.s3EndpointOverride()))
+ .forcePathStyle(true);
+ }
+
+ return builder.build();
+ }
+
+ private static S3AsyncClient buildS3AsyncClient(S3ClientConfig config, ResolvedAsyncHttpConfig resolved)
+ {
+ AwsCredentialsProvider credentialsProvider = getCredentialsProvider(config);
+ NettyNioAsyncHttpClient.Builder httpClientBuilder = NettyNioAsyncHttpClient.builder()
+ .maxConcurrency(resolved.maxConcurrency)
+ .maxPendingConnectionAcquires(resolved.maxPendingConnectionAcquires)
+ .connectionAcquisitionTimeout(Duration.ofSeconds(CONNECTION_ACQUISITION_TIMEOUT_SECONDS))
+ .connectionTimeout(Duration.ofSeconds(CONNECTION_TIMEOUT_SECONDS))
+ .readTimeout(Duration.ofSeconds(READ_TIMEOUT_SECONDS))
+ .connectionMaxIdleTime(Duration.ofSeconds(CONNECTION_MAX_IDLE_TIME_SECONDS))
+ .tcpKeepAlive(true);
+
+ if (isNonBlockingDnsResolverAvailable())
+ {
+ httpClientBuilder.useNonBlockingDnsResolver(true);
+ }
+ else
+ {
+ LOGGER.info("Netty non-blocking DNS resolver is not available on the classpath; "
+ + "building S3AsyncClient with the AWS SDK default DNS resolver");
+ }
+
+ software.amazon.awssdk.services.s3.S3AsyncClientBuilder builder = S3AsyncClient.builder()
+ .region(Region.of(config.s3Region()))
+ .credentialsProvider(credentialsProvider)
+ .httpClientBuilder(httpClientBuilder);
+
+ if (config.s3EndpointOverride() != null && !config.s3EndpointOverride().trim().isEmpty())
+ {
+ builder.endpointOverride(URI.create(config.s3EndpointOverride()))
+ .forcePathStyle(true);
+ }
+
+ LOGGER.info("Built S3AsyncClient region={} maxConcurrency={} maxPendingConnectionAcquires={} "
+ + "nonBlockingDnsResolver={} (knob={}, taskSlotsSource={}, taskSlots={})",
+ config.s3Region(),
+ resolved.maxConcurrency,
+ resolved.maxPendingConnectionAcquires,
+ isNonBlockingDnsResolverAvailable(),
+ config.s3HttpMaxConcurrency(),
+ resolved.taskSlots.source,
+ resolved.taskSlots.taskSlots);
+ return builder.build();
+ }
+
+ static boolean isNonBlockingDnsResolverAvailable()
+ {
+ return NON_BLOCKING_DNS_RESOLVER_AVAILABLE;
+ }
+
+ private static boolean isClassAvailable(String className)
+ {
+ try
+ {
+ Class.forName(className, false, S3ClientCache.class.getClassLoader());
+ return true;
+ }
+ catch (ClassNotFoundException exception)
+ {
+ return false;
+ }
+ }
+
+ static int resolveMaxConcurrency(S3ClientConfig config)
+ {
+ int maxConcurrency = config.s3HttpMaxConcurrency();
+ if (maxConcurrency > 0)
+ {
+ return maxConcurrency;
+ }
+ return resolveMaxConcurrency(resolveTaskSlotsForAutoSize().taskSlots);
+ }
+
+ static int resolveMaxConcurrency(int taskSlots)
+ {
+ return clamp(taskSlots * MAX_CONCURRENCY_PER_TASK_SLOT,
+ MAX_CONCURRENCY_FLOOR,
+ MAX_CONCURRENCY_CEILING);
+ }
+
+ static ResolvedTaskSlots resolveTaskSlotsForAutoSize()
+ {
+ SparkConf conf = null;
+ try
+ {
+ SparkEnv env = SparkEnv.get();
+ if (env != null)
+ {
+ conf = env.conf();
+ }
+ }
+ catch (Throwable ignored)
+ {
+ // SparkEnv can be unavailable in unit tests or non-Spark callers.
+ }
+ return resolveTaskSlotsForAutoSize(conf,
+ System.getenv("SPARK_EXECUTOR_CORES"),
+ Runtime.getRuntime().availableProcessors());
+ }
+
+ static ResolvedTaskSlots resolveTaskSlotsForAutoSize(SparkConf conf,
+ String sparkExecutorCoresEnv,
+ int availableProcessors)
+ {
+ int taskCpus = 1;
+ if (conf != null)
+ {
+ int executorCores = conf.getInt("spark.executor.cores", -1);
+ taskCpus = Math.max(1, conf.getInt("spark.task.cpus", 1));
+ if (executorCores > 0)
+ {
+ return resolvedTaskSlots("SparkEnv.spark.executor.cores/spark.task.cpus",
+ Math.max(1, executorCores / taskCpus));
+ }
+ }
+
+ if (sparkExecutorCoresEnv != null && !sparkExecutorCoresEnv.isEmpty())
+ {
+ try
+ {
+ int executorCores = Integer.parseInt(sparkExecutorCoresEnv.trim());
+ if (executorCores > 0)
+ {
+ return resolvedTaskSlots("SPARK_EXECUTOR_CORES env",
+ Math.max(1, executorCores / taskCpus));
+ }
+ }
+ catch (NumberFormatException ignored)
+ {
+ // Fall through to JVM-visible CPU count.
+ }
+ }
+
+ return resolvedTaskSlots("Runtime.availableProcessors fallback", Math.max(1, availableProcessors));
+ }
+
+ static int resolveMaxPendingConnectionAcquires(int maxConcurrency)
+ {
+ long pending = (long) maxConcurrency * MAX_PENDING_ACQUIRES_PER_CONNECTION;
+ return clamp(pending, MAX_PENDING_ACQUIRES_FLOOR, MAX_PENDING_ACQUIRES_CEILING);
+ }
+
+ private static ResolvedAsyncHttpConfig resolveAsyncHttpConfig(S3ClientConfig config)
+ {
+ ResolvedTaskSlots taskSlots = resolveTaskSlotsForAutoSize();
+ int maxConcurrency = config.s3HttpMaxConcurrency() > 0
+ ? config.s3HttpMaxConcurrency()
+ : resolveMaxConcurrency(taskSlots.taskSlots);
+ int maxPendingConnectionAcquires = resolveMaxPendingConnectionAcquires(maxConcurrency);
+ return new ResolvedAsyncHttpConfig(taskSlots, maxConcurrency, maxPendingConnectionAcquires);
+ }
+
+ private static ResolvedTaskSlots resolvedTaskSlots(String source, int taskSlots)
+ {
+ logTaskSlotSourceOnce(source, taskSlots);
+ return new ResolvedTaskSlots(source, taskSlots);
+ }
+
+ private static void logTaskSlotSourceOnce(String source, int taskSlots)
+ {
+ if (!loggedTaskSlotSource)
+ {
+ loggedTaskSlotSource = true;
+ LOGGER.info("Resolved S3 async HTTP auto-sizing taskSlots={} from {}", taskSlots, source);
+ }
+ }
+
+ private static int clamp(long value, int floor, int ceiling)
+ {
+ return (int) Math.max(floor, Math.min(ceiling, value));
+ }
+
+ private static AwsCredentialsProvider getCredentialsProvider(S3ClientConfig config)
+ {
+ String accessKeyId = config.s3AccessKeyId();
+ String secretAccessKey = config.s3SecretAccessKey();
+ if (accessKeyId != null && !accessKeyId.isEmpty() &&
+ secretAccessKey != null && !secretAccessKey.isEmpty())
+ {
+ return StaticCredentialsProvider.create(
+ AwsBasicCredentials.create(accessKeyId, secretAccessKey));
+ }
+ return DefaultCredentialsProvider.create();
+ }
+
+ /**
+ * Close all clients for a key and remove from caches.
+ */
+ private static void closeClientsForKey(String key)
+ {
+ // Close transfer manager first (wraps async client)
+ S3TransferManager manager = TRANSFER_MANAGER_CACHE.remove(key);
+ if (manager != null)
+ {
+ try
+ {
+ manager.close();
+ }
+ catch (Exception e)
+ {
+ LOGGER.warn("Error closing S3TransferManager for key {}: {}", key, e.getMessage());
+ }
+ }
+
+ // Close async client
+ S3AsyncClient asyncClient = ASYNC_CLIENT_CACHE.remove(key);
+ if (asyncClient != null)
+ {
+ try
+ {
+ asyncClient.close();
+ }
+ catch (Exception e)
+ {
+ LOGGER.warn("Error closing S3AsyncClient for key {}: {}", key, e.getMessage());
+ }
+ }
+
+ // Close sync client
+ S3Client syncClient = SYNC_CLIENT_CACHE.remove(key);
+ if (syncClient != null)
+ {
+ try
+ {
+ syncClient.close();
+ }
+ catch (Exception e)
+ {
+ LOGGER.warn("Error closing S3Client for key {}: {}", key, e.getMessage());
+ }
+ }
+ }
+
+ private static void closeClientsForBaseKey(String baseKey)
+ {
+ Set keys = new HashSet<>();
+ collectKeysForBase(keys, SYNC_CLIENT_CACHE.keySet(), baseKey);
+ collectKeysForBase(keys, ASYNC_CLIENT_CACHE.keySet(), baseKey);
+ collectKeysForBase(keys, TRANSFER_MANAGER_CACHE.keySet(), baseKey);
+ for (String key : keys)
+ {
+ closeClientsForKey(key);
+ }
+ }
+
+ private static void collectKeysForBase(Set collector, Set keys, String baseKey)
+ {
+ for (String key : keys)
+ {
+ if (isKeyForBase(key, baseKey))
+ {
+ collector.add(key);
+ }
+ }
+ }
+
+ private static boolean hasKeyForBase(Set keys, String baseKey)
+ {
+ for (String key : keys)
+ {
+ if (isKeyForBase(key, baseKey))
+ {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private static boolean isKeyForBase(String key, String baseKey)
+ {
+ return key.equals(baseKey) || key.startsWith(baseKey + "|async|");
+ }
+
+ static final class ResolvedTaskSlots
+ {
+ final String source;
+ final int taskSlots;
+
+ private ResolvedTaskSlots(String source, int taskSlots)
+ {
+ this.source = source;
+ this.taskSlots = taskSlots;
+ }
+ }
+
+ private static final class ResolvedAsyncHttpConfig
+ {
+ final ResolvedTaskSlots taskSlots;
+ final int maxConcurrency;
+ final int maxPendingConnectionAcquires;
+
+ private ResolvedAsyncHttpConfig(ResolvedTaskSlots taskSlots,
+ int maxConcurrency,
+ int maxPendingConnectionAcquires)
+ {
+ this.taskSlots = taskSlots;
+ this.maxConcurrency = maxConcurrency;
+ this.maxPendingConnectionAcquires = maxPendingConnectionAcquires;
+ }
+ }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/S3ClientConfig.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/S3ClientConfig.java
new file mode 100644
index 000000000..b9778ef80
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/S3ClientConfig.java
@@ -0,0 +1,234 @@
+/*
+ * 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.io.Serializable;
+import java.util.Map;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import org.apache.cassandra.spark.utils.MapUtils;
+
+/**
+ * Minimal S3 client configuration for connecting to S3.
+ * This class contains only the essential settings needed to create an S3 client.
+ * It is shared across batch and streaming configurations.
+ */
+public class S3ClientConfig implements Serializable
+{
+ private static final long serialVersionUID = 2L;
+
+ // Option keys
+ public static final String S3_REGION_KEY = "s3-region";
+ public static final String S3_BUCKET_KEY = "s3-bucket";
+ public static final String S3_ENDPOINT_OVERRIDE_KEY = "s3-endpoint-override";
+ public static final String S3_ACCESS_KEY_ID_KEY = "s3-access-key-id";
+ public static final String S3_SECRET_ACCESS_KEY_KEY = "s3-secret-access-key";
+ public static final String S3_HTTP_MAX_CONCURRENCY_KEY = "s3-http-max-concurrency";
+
+ @NotNull
+ private final String s3Region;
+ @NotNull
+ private final String s3Bucket;
+ @Nullable
+ private final String s3EndpointOverride;
+ @Nullable
+ private final String s3AccessKeyId;
+ @Nullable
+ private final String s3SecretAccessKey;
+ private final int s3HttpMaxConcurrency;
+
+ private S3ClientConfig(@NotNull String s3Region,
+ @NotNull String s3Bucket,
+ @Nullable String s3EndpointOverride,
+ @Nullable String s3AccessKeyId,
+ @Nullable String s3SecretAccessKey,
+ int s3HttpMaxConcurrency)
+ {
+ if (s3HttpMaxConcurrency < 0)
+ {
+ throw new IllegalArgumentException("Invalid value for option '" + S3_HTTP_MAX_CONCURRENCY_KEY
+ + "': " + s3HttpMaxConcurrency + " (must be >= 0)");
+ }
+ this.s3Region = s3Region;
+ this.s3Bucket = s3Bucket;
+ this.s3EndpointOverride = s3EndpointOverride;
+ this.s3AccessKeyId = s3AccessKeyId;
+ this.s3SecretAccessKey = s3SecretAccessKey;
+ this.s3HttpMaxConcurrency = s3HttpMaxConcurrency;
+ }
+
+ /**
+ * Create an S3ClientConfig from a map of options.
+ *
+ * @param options Configuration options map (case-insensitive keys supported via MapUtils)
+ * @return New S3ClientConfig instance
+ * @throws IllegalArgumentException if required options (s3-region, s3-bucket) are missing
+ */
+ public static S3ClientConfig create(Map options)
+ {
+ String s3Region = MapUtils.getOrThrow(options, S3_REGION_KEY, "region");
+ String s3Bucket = MapUtils.getOrThrow(options, S3_BUCKET_KEY, "bucket");
+ String s3EndpointOverride = options.get(S3_ENDPOINT_OVERRIDE_KEY);
+ String s3AccessKeyId = options.get(S3_ACCESS_KEY_ID_KEY);
+ String s3SecretAccessKey = options.get(S3_SECRET_ACCESS_KEY_KEY);
+ int s3HttpMaxConcurrency = MapUtils.getInt(options, S3_HTTP_MAX_CONCURRENCY_KEY, 0);
+
+ return new S3ClientConfig(s3Region, s3Bucket, s3EndpointOverride, s3AccessKeyId, s3SecretAccessKey,
+ s3HttpMaxConcurrency);
+ }
+
+ /**
+ * Create an S3ClientConfig with explicit values.
+ *
+ * @param s3Region AWS region (required)
+ * @param s3Bucket S3 bucket name (required)
+ * @param s3EndpointOverride Custom endpoint URL (optional, for LocalStack/MinIO)
+ * @param s3AccessKeyId AWS access key ID (optional, uses default credentials if null)
+ * @param s3SecretAccessKey AWS secret access key (optional, uses default credentials if null)
+ * @return New S3ClientConfig instance
+ */
+ public static S3ClientConfig create(@NotNull String s3Region,
+ @NotNull String s3Bucket,
+ @Nullable String s3EndpointOverride,
+ @Nullable String s3AccessKeyId,
+ @Nullable String s3SecretAccessKey)
+ {
+ return create(s3Region, s3Bucket, s3EndpointOverride, s3AccessKeyId, s3SecretAccessKey, 0);
+ }
+
+ /**
+ * Create an S3ClientConfig with explicit values.
+ *
+ * @param s3Region AWS region (required)
+ * @param s3Bucket S3 bucket name (required)
+ * @param s3EndpointOverride Custom endpoint URL (optional, for LocalStack/MinIO)
+ * @param s3AccessKeyId AWS access key ID (optional, uses default credentials if null)
+ * @param s3SecretAccessKey AWS secret access key (optional, uses default credentials if null)
+ * @param s3HttpMaxConcurrency S3 async HTTP max concurrency; 0 means auto-size from Spark task slots
+ * @return New S3ClientConfig instance
+ */
+ public static S3ClientConfig create(@NotNull String s3Region,
+ @NotNull String s3Bucket,
+ @Nullable String s3EndpointOverride,
+ @Nullable String s3AccessKeyId,
+ @Nullable String s3SecretAccessKey,
+ int s3HttpMaxConcurrency)
+ {
+ return new S3ClientConfig(s3Region, s3Bucket, s3EndpointOverride, s3AccessKeyId, s3SecretAccessKey,
+ s3HttpMaxConcurrency);
+ }
+
+ /**
+ * Get the AWS region for S3 operations.
+ *
+ * @return the AWS region
+ */
+ @NotNull
+ public String s3Region()
+ {
+ return s3Region;
+ }
+
+ /**
+ * Get the S3 bucket name.
+ *
+ * @return the S3 bucket name
+ */
+ @NotNull
+ public String s3Bucket()
+ {
+ return s3Bucket;
+ }
+
+ /**
+ * Get the custom S3 endpoint override URL.
+ * Used for LocalStack, MinIO, or other S3-compatible services.
+ *
+ * @return Endpoint URL or null if using default AWS endpoint
+ */
+ @Nullable
+ public String s3EndpointOverride()
+ {
+ return s3EndpointOverride;
+ }
+
+ /**
+ * Get the AWS access key ID for static credentials.
+ *
+ * @return Access key ID or null if using default credentials provider
+ */
+ @Nullable
+ public String s3AccessKeyId()
+ {
+ return s3AccessKeyId;
+ }
+
+ /**
+ * Get the AWS secret access key for static credentials.
+ *
+ * @return Secret access key or null if using default credentials provider
+ */
+ @Nullable
+ public String s3SecretAccessKey()
+ {
+ return s3SecretAccessKey;
+ }
+
+ /**
+ * Get the async S3 HTTP client max concurrency override.
+ * A value of 0 means the cache should auto-size from Spark executor task slots.
+ *
+ * @return the configured max concurrency, or 0 to auto-size
+ */
+ public int s3HttpMaxConcurrency()
+ {
+ return s3HttpMaxConcurrency;
+ }
+
+ public static class Serializer extends com.esotericsoftware.kryo.Serializer
+ {
+ @Override
+ public void write(Kryo kryo, Output output, S3ClientConfig object)
+ {
+ output.writeString(object.s3Region);
+ output.writeString(object.s3Bucket);
+ output.writeString(object.s3EndpointOverride);
+ output.writeString(object.s3AccessKeyId);
+ output.writeString(object.s3SecretAccessKey);
+ output.writeInt(object.s3HttpMaxConcurrency);
+ }
+
+ @Override
+ public S3ClientConfig read(Kryo kryo, Input input, Class type)
+ {
+ return S3ClientConfig.create(input.readString(),
+ input.readString(),
+ input.readString(),
+ input.readString(),
+ input.readString(),
+ input.readInt());
+ }
+ }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/S3DataSourceClientConfig.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/S3DataSourceClientConfig.java
new file mode 100644
index 000000000..f814ffa46
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/S3DataSourceClientConfig.java
@@ -0,0 +1,625 @@
+/*
+ * 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.io.Serializable;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import org.apache.cassandra.bridge.BigNumberConfigImpl;
+import org.jetbrains.annotations.NotNull;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.spark.config.SchemaFeature;
+import org.apache.cassandra.spark.config.SchemaFeatureSet;
+import org.apache.cassandra.spark.data.backup.BackupReaderConfig;
+import org.apache.cassandra.spark.data.backup.BackupReaderFactory;
+import org.apache.cassandra.spark.data.backup.BackupReaderRegistry;
+import org.apache.cassandra.spark.data.partitioner.ConsistencyLevel;
+import org.apache.cassandra.spark.utils.MapUtils;
+import org.apache.cassandra.spark.utils.Properties;
+
+import static org.apache.cassandra.spark.data.CassandraDataLayer.aliasLastModifiedTimestamp;
+
+/**
+ * Configuration for S3-based Cassandra batch data sources.
+ * Composes shared S3 and Cassandra schema configs with batch-specific settings.
+ */
+public class S3DataSourceClientConfig implements Serializable
+{
+ private static final long serialVersionUID = 2L;
+
+ protected final transient Logger logger = LoggerFactory.getLogger(this.getClass());
+
+ // Batch-specific option keys
+ public static final String DEFAULT_PARALLELISM_KEY = "defaultParallelism";
+ public static final String NUM_CORES_KEY = "numCores";
+ public static final String CONSISTENCY_LEVEL_KEY = "consistencyLevel";
+ public static final String ENABLE_STATS_KEY = "enableStats";
+ public static final String LAST_MODIFIED_COLUMN_NAME_KEY = "lastModifiedColumnName";
+ public static final String SNAPSHOT_TIMESTAMP_COLUMN_NAME_KEY = "snapshotTimestampColumnName";
+ public static final String READ_INDEX_OFFSET_KEY = "readIndexOffset";
+ public static final String SIZING_KEY = "sizing";
+ public static final String SIZING_DEFAULT = "default";
+ public static final String SIZING_DYNAMIC = "dynamic";
+ public static final String NUMBER_SPLITS_KEY = "number_splits";
+ // When calculating sizing for dynamic sizing, each partition is maxPartitionSize GB in size.
+ public static final String MAX_PARTITION_SIZE_KEY = "maxPartitionSize";
+ public static final String SSTABLE_S3_READ_TIMEOUT_KEY = "sstable-s3-read-timeout";
+
+ /**
+ * Required option that selects a {@link BackupReaderFactory} from {@link BackupReaderRegistry}.
+ * There is no default; vendor-specific implementation modules register their factory and
+ * document the corresponding option value.
+ */
+ public static final String BACKUP_READER_TYPE_KEY = "backupReaderType";
+
+ // Data.db ranged-GET buffering (DataSource options). Only Data.db; metadata components keep their
+ // smaller per-FileType defaults in Properties#DEFAULT_CHUNK_BUFFER_OVERRIDE.
+ public static final String S3_DATA_CHUNK_BUFFER_SIZE_KEY = "sstableDataChunkBufferSize";
+ public static final String S3_DATA_MAX_BUFFER_SIZE_KEY = "sstableDataMaxBufferSize";
+
+ // When false, Data.db reads use the AsyncResponseTransformer.toBytes() path
+ // (single materialized byte[] per ranged GET).
+ // When true (default), Data.db reads use the AsyncResponseTransformer.toPublisher() streaming path
+ // (many small ByteBuffer chunks fanned through BufferingInputStream). Non-Data file types
+ // and mutable metadata are not affected.
+ public static final String SSTABLE_DATA_PUBLISHER_READ_ENABLED_KEY = "sstableDataPublisherReadEnabled";
+
+ public static final String SSTABLE_TOKEN_INDEX_ENABLED_KEY = "sstableTokenIndexEnabled";
+ public static final String SSTABLE_TOKEN_INDEX_PREBUILD_PARTITIONS_KEY = "sstableTokenIndexPrebuildPartitions";
+ public static final String SSTABLE_TOKEN_INDEX_PREBUILD_PER_TASK_CONCURRENCY_KEY = "sstableTokenIndexPrebuildPerTaskConcurrency";
+ private static final int DEFAULT_SSTABLES_PER_TOKEN_INDEX_PREBUILD_PARTITION = 10_000;
+ private static final int DEFAULT_SSTABLE_TOKEN_INDEX_PREBUILD_PER_TASK_CONCURRENCY = 4;
+
+ // SSTable metadata cache sizing (forwarded to SSTableCache via sbr.cache.*.maxEntries sysprops).
+ public static final String SSTABLE_CACHE_SUMMARY_MAX_ENTRIES_KEY = "sstableCacheSummaryMaxEntries";
+ public static final String SSTABLE_CACHE_INDEX_MAX_ENTRIES_KEY = "sstableCacheIndexMaxEntries";
+ public static final String SSTABLE_CACHE_STATS_MAX_ENTRIES_KEY = "sstableCacheStatsMaxEntries";
+ public static final String SSTABLE_CACHE_FILTER_MAX_ENTRIES_KEY = "sstableCacheFilterMaxEntries";
+ public static final String SSTABLE_CACHE_COMPRESSION_INFO_MAX_ENTRIES_KEY = "sstableCacheCompressionInfoMaxEntries";
+
+ public static final int DEFAULT_NUM_SPLITS = -1;
+
+ // Composed configs
+ @NotNull
+ private final S3ClientConfig s3Config;
+ @NotNull
+ private final CassandraSchemaConfig schemaConfig;
+
+ // Batch-specific fields
+ private final int defaultParallelism;
+ private final int numCores;
+ private final ConsistencyLevel consistencyLevel;
+ private final Map bigNumberConfigMap;
+ private final boolean enableStats;
+ private final boolean readIndexOffset;
+ private final String sizing;
+ private final int numberSplits;
+ private final int maxPartitionSize;
+ private final List requestedFeatures;
+ private final String lastModifiedTimestampField;
+ private final String snapshotTimestampField;
+ private final int sstableS3ReadTimeoutSeconds;
+ private final long s3DataChunkBufferSize;
+ private final long s3DataMaxBufferSize;
+ private final boolean sstableDataPublisherReadEnabled;
+ private final boolean sstableTokenIndexEnabled;
+ private final int sstableTokenIndexPrebuildPartitions;
+ private final int sstableTokenIndexPrebuildPerTaskConcurrency;
+ private final int sstableCacheSummaryMaxEntries;
+ private final int sstableCacheIndexMaxEntries;
+ private final int sstableCacheStatsMaxEntries;
+ private final int sstableCacheFilterMaxEntries;
+ private final int sstableCacheCompressionInfoMaxEntries;
+ @NotNull
+ private final String backupReaderType;
+
+ protected S3DataSourceClientConfig(Map options)
+ {
+ // Create composed configs
+ this.s3Config = S3ClientConfig.create(options);
+ this.schemaConfig = CassandraSchemaConfig.create(options);
+
+ // Batch-specific options
+ this.defaultParallelism = MapUtils.getInt(options, DEFAULT_PARALLELISM_KEY, 1);
+ this.numCores = MapUtils.getInt(options, NUM_CORES_KEY, 1);
+ this.consistencyLevel = Optional.ofNullable(options.get(MapUtils.lowerCaseKey(CONSISTENCY_LEVEL_KEY)))
+ .map(ConsistencyLevel::valueOf)
+ .orElse(null);
+ this.bigNumberConfigMap = BigNumberConfigImpl.build(options);
+ this.enableStats = MapUtils.getBoolean(options, ENABLE_STATS_KEY, true);
+ this.readIndexOffset = MapUtils.getBoolean(options, READ_INDEX_OFFSET_KEY, true);
+ this.sizing = MapUtils.getOrDefault(options, SIZING_KEY, SIZING_DEFAULT);
+ this.maxPartitionSize = MapUtils.getInt(options, MAX_PARTITION_SIZE_KEY, 1);
+ this.lastModifiedTimestampField = MapUtils.getOrDefault(options, LAST_MODIFIED_COLUMN_NAME_KEY, null);
+ this.snapshotTimestampField = MapUtils.getOrDefault(options, SNAPSHOT_TIMESTAMP_COLUMN_NAME_KEY, null);
+ this.numberSplits = MapUtils.getInt(options, NUMBER_SPLITS_KEY, DEFAULT_NUM_SPLITS, "number of splits");
+ this.requestedFeatures = initRequestedFeatures(options);
+ this.sstableS3ReadTimeoutSeconds = MapUtils.getInt(options, SSTABLE_S3_READ_TIMEOUT_KEY, 600);
+ // Floor at 1 MiB to avoid pathological overrides.
+ this.s3DataChunkBufferSize = Math.max(1024L * 1024L,
+ MapUtils.getLong(options, S3_DATA_CHUNK_BUFFER_SIZE_KEY,
+ Properties.DEFAULT_S3_DATA_CHUNK_BUFFER_SIZE));
+ // Must be >= chunk so BufferingInputStream can enqueue a full chunk.
+ this.s3DataMaxBufferSize = Math.max(this.s3DataChunkBufferSize,
+ MapUtils.getLong(options, S3_DATA_MAX_BUFFER_SIZE_KEY,
+ Properties.DEFAULT_S3_DATA_MAX_BUFFER_SIZE));
+ // Default true: Data.db reads use the AsyncResponseTransformer.toPublisher() streaming path.
+ this.sstableDataPublisherReadEnabled = MapUtils.getBoolean(options, SSTABLE_DATA_PUBLISHER_READ_ENABLED_KEY, true);
+ this.sstableTokenIndexEnabled = MapUtils.getBoolean(options, SSTABLE_TOKEN_INDEX_ENABLED_KEY, false);
+ this.sstableTokenIndexPrebuildPartitions = optionalPositiveInt(options, SSTABLE_TOKEN_INDEX_PREBUILD_PARTITIONS_KEY);
+ this.sstableTokenIndexPrebuildPerTaskConcurrency =
+ Math.max(1, MapUtils.getInt(options,
+ SSTABLE_TOKEN_INDEX_PREBUILD_PER_TASK_CONCURRENCY_KEY,
+ DEFAULT_SSTABLE_TOKEN_INDEX_PREBUILD_PER_TASK_CONCURRENCY));
+ this.sstableCacheSummaryMaxEntries = nonNegativeInt(options, SSTABLE_CACHE_SUMMARY_MAX_ENTRIES_KEY, 32768);
+ this.sstableCacheIndexMaxEntries = nonNegativeInt(options, SSTABLE_CACHE_INDEX_MAX_ENTRIES_KEY, 16384);
+ this.sstableCacheStatsMaxEntries = nonNegativeInt(options, SSTABLE_CACHE_STATS_MAX_ENTRIES_KEY, 16384);
+ this.sstableCacheFilterMaxEntries = nonNegativeInt(options, SSTABLE_CACHE_FILTER_MAX_ENTRIES_KEY, 16384);
+ this.sstableCacheCompressionInfoMaxEntries = nonNegativeInt(options, SSTABLE_CACHE_COMPRESSION_INFO_MAX_ENTRIES_KEY, 16384);
+ this.backupReaderType = requiredBackupReaderType(options);
+ }
+
+ /**
+ * Validates that the {@code backupReaderType} option is present and matches a registered
+ * factory, falling back to a precise error message that lists the registered types if not.
+ * Validating eagerly here surfaces the misconfiguration at config-parse time rather than at
+ * first read.
+ */
+ private static String requiredBackupReaderType(Map options)
+ {
+ String type = options.get(MapUtils.lowerCaseKey(BACKUP_READER_TYPE_KEY));
+ if (type == null || type.trim().isEmpty())
+ {
+ throw new IllegalArgumentException(
+ "Missing required option '" + BACKUP_READER_TYPE_KEY + "'. "
+ + "Register a backup reader factory via BackupReaderRegistry.register(...) at driver "
+ + "startup and set this option to its registered type. Registered types: "
+ + BackupReaderRegistry.registeredTypes());
+ }
+ // Fail fast if the type is unregistered; factoryFor throws a precise message listing
+ // the registered types. Resolved factory is intentionally discarded — the driver only
+ // needs it later at reader-construction time.
+ BackupReaderRegistry.factoryFor(type);
+ return type;
+ }
+
+ /** Parse a non-negative int option; throws with the option key in the message on negatives. */
+ private static int nonNegativeInt(Map options, String key, int defaultValue)
+ {
+ int value = MapUtils.getInt(options, key, defaultValue);
+ if (value < 0)
+ {
+ throw new IllegalArgumentException(
+ "Invalid value for option '" + key + "': " + value + " (must be >= 0; use 0 to disable the cache)");
+ }
+ return value;
+ }
+
+ private static int optionalPositiveInt(Map options, String key)
+ {
+ String value = options.get(MapUtils.lowerCaseKey(key));
+ return value == null ? 0 : Math.max(1, Integer.parseInt(value));
+ }
+
+ public static S3DataSourceClientConfig create(Map options)
+ {
+ return new S3DataSourceClientConfig(options);
+ }
+
+ // ========================================================================
+ // Composed Config Accessors
+ // ========================================================================
+
+ /**
+ * Get the S3 client configuration.
+ *
+ * @return the S3 client configuration
+ */
+ @NotNull
+ public S3ClientConfig s3Config()
+ {
+ return s3Config;
+ }
+
+ /**
+ * Get the Cassandra schema configuration.
+ *
+ * @return the Cassandra schema configuration
+ */
+ @NotNull
+ public CassandraSchemaConfig schemaConfig()
+ {
+ return schemaConfig;
+ }
+
+ // ========================================================================
+ // Batch-Specific Accessors
+ // ========================================================================
+
+ public int defaultParallelism()
+ {
+ return defaultParallelism;
+ }
+
+ public int numCores()
+ {
+ return numCores;
+ }
+
+ public ConsistencyLevel consistencyLevel()
+ {
+ return consistencyLevel;
+ }
+
+ public Map bigNumberConfigMap()
+ {
+ return bigNumberConfigMap;
+ }
+
+ public boolean enableStats()
+ {
+ return enableStats;
+ }
+
+ public boolean readIndexOffset()
+ {
+ return readIndexOffset;
+ }
+
+ public String sizing()
+ {
+ return sizing;
+ }
+
+ public int maxPartitionSize()
+ {
+ return maxPartitionSize;
+ }
+
+ public int numberSplits()
+ {
+ return numberSplits;
+ }
+
+ public List requestedFeatures()
+ {
+ return requestedFeatures;
+ }
+
+ public String lastModifiedTimestampField()
+ {
+ return lastModifiedTimestampField;
+ }
+
+ public String snapshotTimestampField()
+ {
+ return snapshotTimestampField;
+ }
+
+ public int sstableS3ReadTimeoutSeconds()
+ {
+ return sstableS3ReadTimeoutSeconds;
+ }
+
+ /**
+ * Ranged-GET chunk size (bytes) for {@code Data.db}; trades per-GET / KMS overhead vs per-chunk heap.
+ *
+ * @return the per-GET chunk size in bytes
+ */
+ public long s3DataChunkBufferSize()
+ {
+ return s3DataChunkBufferSize;
+ }
+
+ /**
+ * Per-stream buffer cap for Data.db; values above the chunk size allow one in-flight + one draining.
+ *
+ * @return the per-stream buffer cap in bytes
+ */
+ public long s3DataMaxBufferSize()
+ {
+ return s3DataMaxBufferSize;
+ }
+
+ /**
+ * When false, Data.db reads use {@code AsyncResponseTransformer.toBytes()}.
+ * When true (default), Data.db reads use {@code AsyncResponseTransformer.toPublisher()}.
+ * Non-Data file types and mutable metadata reads are unaffected.
+ *
+ * @return {@code true} iff publisher-based reads are enabled for Data.db
+ */
+ public boolean sstableDataPublisherReadEnabled()
+ {
+ return sstableDataPublisherReadEnabled;
+ }
+
+ public boolean sstableTokenIndexEnabled()
+ {
+ return sstableTokenIndexEnabled;
+ }
+
+ public int sstableTokenIndexPrebuildPartitions()
+ {
+ return sstableTokenIndexPrebuildPartitions;
+ }
+
+ public int sstableTokenIndexPrebuildPerTaskConcurrency()
+ {
+ return sstableTokenIndexPrebuildPerTaskConcurrency;
+ }
+
+ public int resolveSSTableTokenIndexPrebuildPartitions(int sstableCount, int sparkDefaultParallelism)
+ {
+ if (sstableTokenIndexPrebuildPartitions > 0)
+ {
+ return sstableTokenIndexPrebuildPartitions;
+ }
+
+ int defaultParallelismCap = Math.max(1, sparkDefaultParallelism / 4);
+ int sstableCountPartitions = Math.max(1, (sstableCount + DEFAULT_SSTABLES_PER_TOKEN_INDEX_PREBUILD_PARTITION - 1)
+ / DEFAULT_SSTABLES_PER_TOKEN_INDEX_PREBUILD_PARTITION);
+ return Math.min(defaultParallelismCap, sstableCountPartitions);
+ }
+
+ public int sstableCacheSummaryMaxEntries()
+ {
+ return sstableCacheSummaryMaxEntries;
+ }
+
+ public int sstableCacheIndexMaxEntries()
+ {
+ return sstableCacheIndexMaxEntries;
+ }
+
+ public int sstableCacheStatsMaxEntries()
+ {
+ return sstableCacheStatsMaxEntries;
+ }
+
+ public int sstableCacheFilterMaxEntries()
+ {
+ return sstableCacheFilterMaxEntries;
+ }
+
+ public int sstableCacheCompressionInfoMaxEntries()
+ {
+ return sstableCacheCompressionInfoMaxEntries;
+ }
+
+ /**
+ * Returns the configured backup reader type (a registered key in {@link BackupReaderRegistry}).
+ *
+ * @return the configured backup reader type
+ */
+ @NotNull
+ public String backupReaderType()
+ {
+ return backupReaderType;
+ }
+
+ /**
+ * Builds a {@link BackupReaderConfig} from this config's {@link S3ClientConfig}. {@code Stats}
+ * are not carried on the config object; instead callers pass the appropriate {@code Stats}
+ * instance per call into
+ * {@link org.apache.cassandra.spark.data.backup.BackupReader} read methods (executor-local
+ * {@code SparkCustomMetricsStats} for reads, {@code DoNothingStats.INSTANCE} for prebuild).
+ *
+ * @return a {@link BackupReaderConfig} populated from this config's S3 client settings
+ */
+ public BackupReaderConfig toBackupReaderConfig()
+ {
+ return BackupReaderConfig.of(s3Config);
+ }
+
+ /**
+ * Driver-side apply of SSTable cache sizes as JVM sysprops; existing values are preserved.
+ * Executor-side propagation lives on {@link S3CassandraDataLayer#applySSTableCacheSystemProperties()}.
+ */
+ public void applySSTableCacheSystemProperties()
+ {
+ setIfUnset("sbr.cache.summary.maxEntries", String.valueOf(sstableCacheSummaryMaxEntries));
+ setIfUnset("sbr.cache.index.maxEntries", String.valueOf(sstableCacheIndexMaxEntries));
+ setIfUnset("sbr.cache.stats.maxEntries", String.valueOf(sstableCacheStatsMaxEntries));
+ setIfUnset("sbr.cache.filter.maxEntries", String.valueOf(sstableCacheFilterMaxEntries));
+ setIfUnset("sbr.cache.compressionInfo.maxEntries", String.valueOf(sstableCacheCompressionInfoMaxEntries));
+ }
+
+ private static void setIfUnset(String name, String value)
+ {
+ if (System.getProperty(name) == null)
+ {
+ System.setProperty(name, value);
+ }
+ }
+
+ // ========================================================================
+ // Convenience Delegate Methods (for backward compatibility)
+ // These delegate to the composed configs for callers that access these
+ // properties directly on S3DataSourceClientConfig.
+ // ========================================================================
+
+ /**
+ * Get the cluster identifier (UUID). Resolved eagerly at config creation.
+ * Delegates to schemaConfig.
+ *
+ * @return the cluster identifier
+ */
+ public String clusterName()
+ {
+ return schemaConfig.clusterName();
+ }
+
+ /**
+ * Get the keyspace name. Delegates to schemaConfig.
+ *
+ * @return the keyspace name
+ */
+ public String keyspace()
+ {
+ return schemaConfig.keyspace();
+ }
+
+ /**
+ * Get the table name. Delegates to schemaConfig.
+ *
+ * @return the table name
+ */
+ public String table()
+ {
+ return schemaConfig.table();
+ }
+
+ /**
+ * Get the datacenter. Delegates to schemaConfig.
+ *
+ * @return the datacenter name, or {@code null} if not specified
+ */
+ public String datacenter()
+ {
+ return schemaConfig.datacenter();
+ }
+
+ /**
+ * Get the CREATE TABLE statement. Delegates to schemaConfig.
+ *
+ * @return the CREATE TABLE DDL string
+ */
+ public String tableCreateStmt()
+ {
+ return schemaConfig.tableCreateStmt();
+ }
+
+ /**
+ * Get the Cassandra version. Delegates to schemaConfig.
+ *
+ * @return the Cassandra version enum
+ */
+ public org.apache.cassandra.bridge.CassandraVersion cassandraVersion()
+ {
+ return schemaConfig.cassandraVersion();
+ }
+
+ /**
+ * Get raw UDT definitions string. Delegates to schemaConfig.
+ *
+ * @return the raw UDT definitions string
+ */
+ public String udts()
+ {
+ return schemaConfig.udts();
+ }
+
+ /**
+ * Get parsed UDTs. Delegates to schemaConfig.
+ *
+ * @return parsed UDT definition strings
+ */
+ public java.util.Set parsedUdts()
+ {
+ return schemaConfig.parsedUdts();
+ }
+
+ /**
+ * Get parsed replication factor. Delegates to schemaConfig.
+ *
+ * @return the parsed replication factor
+ */
+ public ReplicationFactor getParsedReplicationFactor()
+ {
+ return schemaConfig.getParsedReplicationFactor();
+ }
+
+ /**
+ * Get the S3 region. Delegates to s3Config.
+ *
+ * @return the S3 region
+ */
+ public String s3Region()
+ {
+ return s3Config.s3Region();
+ }
+
+ /**
+ * Get the S3 bucket. Delegates to s3Config.
+ *
+ * @return the S3 bucket name
+ */
+ public String s3Bucket()
+ {
+ return s3Config.s3Bucket();
+ }
+
+ /**
+ * Get the S3 endpoint override. Delegates to s3Config.
+ *
+ * @return the S3 endpoint override, or {@code null} if using default
+ */
+ public String s3EndpointOverride()
+ {
+ return s3Config.s3EndpointOverride();
+ }
+
+ /**
+ * Get the S3 access key ID. Delegates to s3Config.
+ *
+ * @return the S3 access key ID, or {@code null} if using default credentials
+ */
+ public String s3AccessKeyId()
+ {
+ return s3Config.s3AccessKeyId();
+ }
+
+ /**
+ * Get the S3 secret access key. Delegates to s3Config.
+ *
+ * @return the S3 secret access key, or {@code null} if using default credentials
+ */
+ public String s3SecretAccessKey()
+ {
+ return s3Config.s3SecretAccessKey();
+ }
+
+ protected List initRequestedFeatures(Map options)
+ {
+ Map optionsCopy = new HashMap<>(options);
+ String lastModifiedColumnName = MapUtils.getOrDefault(options, LAST_MODIFIED_COLUMN_NAME_KEY, null);
+ if (lastModifiedColumnName != null)
+ {
+ optionsCopy.put(SchemaFeatureSet.LAST_MODIFIED_TIMESTAMP.optionName(), "true");
+ }
+ String snapshotTimestampColumnName = MapUtils.getOrDefault(options, SNAPSHOT_TIMESTAMP_COLUMN_NAME_KEY, null);
+ if (snapshotTimestampColumnName != null)
+ {
+ optionsCopy.put(SchemaFeatureSet.SNAPSHOT_TIMESTAMP.optionName(), "true");
+ }
+ List features = SchemaFeatureSet.initializeFromOptions(optionsCopy);
+ if (lastModifiedColumnName != null)
+ {
+ aliasLastModifiedTimestamp(features, lastModifiedColumnName);
+ }
+ return features;
+ }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/S3TableSizeProvider.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/S3TableSizeProvider.java
new file mode 100644
index 000000000..565ff3168
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/S3TableSizeProvider.java
@@ -0,0 +1,86 @@
+/*
+ * 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.Map;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.spark.data.backup.BackupReader;
+
+/**
+ * Implementation of {@link TableSizeProvider} that uses S3 backup metadata to calculate the table size
+ * without requiring sidecar connectivity.
+ */
+public class S3TableSizeProvider implements TableSizeProvider
+{
+ private static final Logger LOGGER = LoggerFactory.getLogger(S3TableSizeProvider.class);
+
+ private final BackupReader s3BackupReader;
+ private final String clusterName;
+
+ public S3TableSizeProvider(BackupReader s3BackupReader, String clusterName)
+ {
+ this.s3BackupReader = s3BackupReader;
+ this.clusterName = clusterName;
+ }
+
+ /**
+ * Returns the total used space for {@code table} across the datacenter by aggregating
+ * the sizes of all SSTable files from the S3 backup.
+ *
+ * @param keyspace the keyspace where the table lives
+ * @param table the table to get the size from
+ * @param datacenter the datacenter
+ * @return the total used space for the table across the datacenter
+ */
+ @Override
+ public long tableSizeInBytes(String keyspace, String table, String datacenter)
+ {
+ try
+ {
+ // Get all SSTables for this table from S3
+ Map> sstables = s3BackupReader.sstables(clusterName, keyspace, table, datacenter);
+
+ long totalSize = 0;
+
+ // Sum up the Data.db file sizes from all SSTables
+ for (Map componentSizes : sstables.values())
+ {
+ Long dataFileSize = componentSizes.get(FileType.DATA);
+ if (dataFileSize != null)
+ {
+ totalSize += dataFileSize;
+ }
+ }
+
+ LOGGER.info("Calculated S3 table size for {}.{} in datacenter {}: {} bytes from {} SSTables",
+ keyspace, table, datacenter, totalSize, sstables.size());
+
+ return totalSize;
+ }
+ catch (Exception ex)
+ {
+ throw new RuntimeException(String.format("Error occurred while determining the S3 table size for table '%s.%s'",
+ keyspace, table), ex);
+ }
+ }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/SSTableIndexKey.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/SSTableIndexKey.java
new file mode 100644
index 000000000..8c7dcd32b
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/SSTableIndexKey.java
@@ -0,0 +1,97 @@
+/*
+ * 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.io.Serializable;
+import java.util.Objects;
+
+/**
+ * Stable SSTable identifier for token-index lookups.
+ *
+ * This intentionally copies the value fields out of {@link SSTableKey} so index lookups never depend
+ * on Java object identity after Spark task and broadcast deserialization.
+ */
+public final class SSTableIndexKey implements Serializable
+{
+ private static final long serialVersionUID = 2026042501L;
+
+ private final String nodeId;
+ private final String keyspace;
+ private final String table;
+ private final String tableId;
+ private final String generationId;
+ private final String crc;
+ private final String fileNameWithoutType;
+
+ private SSTableIndexKey(String nodeId,
+ String keyspace,
+ String table,
+ String tableId,
+ String generationId,
+ String crc,
+ String fileNameWithoutType)
+ {
+ this.nodeId = nodeId;
+ this.keyspace = keyspace;
+ this.table = table;
+ this.tableId = tableId;
+ this.generationId = generationId;
+ this.crc = crc;
+ this.fileNameWithoutType = fileNameWithoutType;
+ }
+
+ public static SSTableIndexKey from(SSTableKey key)
+ {
+ return new SSTableIndexKey(key.getNodeId(),
+ key.getKeyspace(),
+ key.getTable(),
+ key.getTableId(),
+ key.getGenerationId(),
+ key.getCrc(),
+ key.getFileNameWithoutType());
+ }
+
+ @Override
+ public boolean equals(Object obj)
+ {
+ if (this == obj)
+ {
+ return true;
+ }
+ if (obj == null || getClass() != obj.getClass())
+ {
+ return false;
+ }
+ SSTableIndexKey that = (SSTableIndexKey) obj;
+ return Objects.equals(nodeId, that.nodeId)
+ && Objects.equals(keyspace, that.keyspace)
+ && Objects.equals(table, that.table)
+ && Objects.equals(tableId, that.tableId)
+ && Objects.equals(generationId, that.generationId)
+ && Objects.equals(crc, that.crc)
+ && Objects.equals(fileNameWithoutType, that.fileNameWithoutType);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(nodeId, keyspace, table, tableId, generationId, crc, fileNameWithoutType);
+ }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/SSTableKey.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/SSTableKey.java
new file mode 100644
index 000000000..5c63e0f2b
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/SSTableKey.java
@@ -0,0 +1,141 @@
+/*
+ * 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.io.Serializable;
+import java.util.Objects;
+
+/**
+ * A key class that uniquely identifies an SSTable across various nodes.
+ */
+public class SSTableKey implements Serializable
+{
+ private static final long serialVersionUID = 20250620L;
+
+ private final String nodeId;
+ private final String keyspace;
+ private final String table;
+ private final String tableId;
+ private final String generationId;
+ private final String crc;
+ private final String fileNameWithoutType;
+
+ /**
+ * Creates an SSTableKey from the parsed components of an SSTable.
+ *
+ * @param nodeId the node ID where the SSTable resides
+ * @param keyspace the keyspace name
+ * @param table the table name
+ * @param tableId the table UUID
+ * @param generationId the generation ID of the SSTable
+ * @param crc the CRC value of the SSTable
+ * @param fileNameWithoutType the SSTable filename without file type suffix (e.g., "nb-123456-big")
+ */
+ public SSTableKey(String nodeId, String keyspace, String table, String tableId, String generationId, String crc, String fileNameWithoutType)
+ {
+ this.nodeId = Objects.requireNonNull(nodeId, "nodeId cannot be null");
+ this.keyspace = Objects.requireNonNull(keyspace, "keyspace cannot be null");
+ this.table = Objects.requireNonNull(table, "table cannot be null");
+ this.tableId = Objects.requireNonNull(tableId, "tableId cannot be null");
+ this.generationId = Objects.requireNonNull(generationId, "generationId cannot be null");
+ this.crc = Objects.requireNonNull(crc, "crc cannot be null");
+ this.fileNameWithoutType = Objects.requireNonNull(fileNameWithoutType, "fileNameWithoutType cannot be null");
+ }
+
+ // Getters
+ public String getNodeId()
+ {
+ return nodeId;
+ }
+
+ public String getKeyspace()
+ {
+ return keyspace;
+ }
+
+ public String getTable()
+ {
+ return table;
+ }
+
+ public String getTableId()
+ {
+ return tableId;
+ }
+
+ public String getGenerationId()
+ {
+ return generationId;
+ }
+
+ public String getCrc()
+ {
+ return crc;
+ }
+
+ public String getFileNameWithoutType()
+ {
+ return fileNameWithoutType;
+ }
+
+ /**
+ * Generates the data filename for this SSTable.
+ *
+ * @return the data filename in the format fileNameWithoutType-Data.db
+ */
+ public String getDataFileName()
+ {
+ return String.format("%s-%s", fileNameWithoutType, FileType.DATA.getFileSuffix());
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o)
+ {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass())
+ {
+ return false;
+ }
+ SSTableKey that = (SSTableKey) o;
+ return Objects.equals(nodeId, that.nodeId) &&
+ Objects.equals(keyspace, that.keyspace) &&
+ Objects.equals(table, that.table) &&
+ Objects.equals(tableId, that.tableId) &&
+ Objects.equals(generationId, that.generationId) &&
+ Objects.equals(crc, that.crc) &&
+ Objects.equals(fileNameWithoutType, that.fileNameWithoutType);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(nodeId, keyspace, table, tableId, generationId, crc, fileNameWithoutType);
+ }
+
+ @Override
+ public String toString()
+ {
+ return String.format("SSTableKey{nodeId='%s', keyspace='%s', table='%s', tableId='%s', generationId='%s', crc='%s', fileNameWithoutType='%s'}",
+ nodeId, keyspace, table, tableId, generationId, crc, fileNameWithoutType);
+ }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/SSTableSummaryWorkItem.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/SSTableSummaryWorkItem.java
new file mode 100644
index 000000000..e40102a43
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/SSTableSummaryWorkItem.java
@@ -0,0 +1,61 @@
+/*
+ * 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.io.Serializable;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+public final class SSTableSummaryWorkItem implements Serializable
+{
+ private static final long serialVersionUID = 2026042501L;
+
+ private final SSTableKey sstableKey;
+ private final String token;
+ private final Map componentSizes;
+
+ public SSTableSummaryWorkItem(SSTableKey sstableKey, String token, Map componentSizes)
+ {
+ this.sstableKey = sstableKey;
+ this.token = token;
+ this.componentSizes = Collections.unmodifiableMap(new HashMap<>(componentSizes));
+ }
+
+ public SSTableKey sstableKey()
+ {
+ return sstableKey;
+ }
+
+ public SSTableIndexKey indexKey()
+ {
+ return SSTableIndexKey.from(sstableKey);
+ }
+
+ public String token()
+ {
+ return token;
+ }
+
+ public Map componentSizes()
+ {
+ return componentSizes;
+ }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/SSTableTokenBounds.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/SSTableTokenBounds.java
new file mode 100644
index 000000000..65080efce
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/SSTableTokenBounds.java
@@ -0,0 +1,68 @@
+/*
+ * 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.io.Serializable;
+import java.math.BigInteger;
+
+import org.apache.cassandra.bridge.TokenRange;
+
+public final class SSTableTokenBounds implements Serializable
+{
+ private static final long serialVersionUID = 2026042501L;
+
+ // Murmur3 partitioner only: tokens fit in a long. RandomPartitioner is not supported here.
+ private static final BigInteger MIN_TOKEN = BigInteger.valueOf(Long.MIN_VALUE);
+ private static final BigInteger MAX_TOKEN = BigInteger.valueOf(Long.MAX_VALUE);
+
+ private final long firstToken;
+ private final long lastToken;
+
+ public SSTableTokenBounds(long firstToken, long lastToken)
+ {
+ this.firstToken = firstToken;
+ this.lastToken = lastToken;
+ }
+
+ public long firstToken()
+ {
+ return firstToken;
+ }
+
+ public long lastToken()
+ {
+ return lastToken;
+ }
+
+ public boolean overlaps(TokenRange range)
+ {
+ BigInteger first = BigInteger.valueOf(firstToken);
+ BigInteger last = BigInteger.valueOf(lastToken);
+ // firstToken > lastToken represents a ring wrap-around (the same convention as
+ // RangeUtils.calculateTokenRanges); model it as the two segments [first, MAX] and
+ // [MIN, last] and report overlap if the query range hits either.
+ if (firstToken > lastToken)
+ {
+ return range.isConnected(TokenRange.closed(first, MAX_TOKEN))
+ || range.isConnected(TokenRange.closed(MIN_TOKEN, last));
+ }
+ return range.isConnected(TokenRange.closed(first, last));
+ }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/SSTableTokenIndex.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/SSTableTokenIndex.java
new file mode 100644
index 000000000..3d5d7e05b
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/SSTableTokenIndex.java
@@ -0,0 +1,96 @@
+/*
+ * 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.io.Serializable;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.cassandra.bridge.TokenRange;
+
+public final class SSTableTokenIndex implements Serializable
+{
+ private static final long serialVersionUID = 2026042501L;
+
+ private final Map boundsBySSTable;
+ private final int successCount;
+ private final int missingCount;
+ private final int errorCount;
+
+ private SSTableTokenIndex(Map boundsBySSTable,
+ int successCount,
+ int missingCount,
+ int errorCount)
+ {
+ this.boundsBySSTable = Collections.unmodifiableMap(boundsBySSTable);
+ this.successCount = successCount;
+ this.missingCount = missingCount;
+ this.errorCount = errorCount;
+ }
+
+ public static SSTableTokenIndex fromShards(List shards)
+ {
+ Map merged = new HashMap<>();
+ int successCount = 0;
+ int missingCount = 0;
+ int errorCount = 0;
+ for (TokenIndexShard shard : shards)
+ {
+ merged.putAll(shard.boundsBySSTable());
+ successCount += shard.successCount();
+ missingCount += shard.missingCount();
+ errorCount += shard.errorCount();
+ }
+ return new SSTableTokenIndex(merged, successCount, missingCount, errorCount);
+ }
+
+ public boolean include(SSTableKey key, TokenRange range)
+ {
+ SSTableTokenBounds bounds = boundsBySSTable.get(SSTableIndexKey.from(key));
+ return bounds == null || bounds.overlaps(range);
+ }
+
+ public int size()
+ {
+ return boundsBySSTable.size();
+ }
+
+ public int successCount()
+ {
+ return successCount;
+ }
+
+ public int missingCount()
+ {
+ return missingCount;
+ }
+
+ public int errorCount()
+ {
+ return errorCount;
+ }
+
+ public long estimatedSizeInBytes()
+ {
+ return 64L * Math.max(1, boundsBySSTable.size());
+ }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/SSTableTokenIndexBuilder.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/SSTableTokenIndexBuilder.java
new file mode 100644
index 000000000..d1f19181e
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/SSTableTokenIndexBuilder.java
@@ -0,0 +1,498 @@
+/*
+ * 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.io.ByteArrayInputStream;
+import java.io.EOFException;
+import java.io.InputStream;
+import java.math.BigInteger;
+import java.net.SocketException;
+import java.net.SocketTimeoutException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.cassandra.analytics.stats.Stats;
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.bridge.CassandraBridgeFactory;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.bridge.SSTableSummary;
+import org.apache.cassandra.spark.data.backup.BackupReader;
+import org.apache.cassandra.spark.data.backup.BackupReaderConfig;
+import org.apache.cassandra.spark.data.backup.BackupReaderFactory;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.jetbrains.annotations.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.core.exception.SdkException;
+
+public final class SSTableTokenIndexBuilder
+{
+ private static final Logger LOGGER = LoggerFactory.getLogger(SSTableTokenIndexBuilder.class);
+ private static final int DEFAULT_MIN_INDEX_INTERVAL = 128;
+ private static final int DEFAULT_MAX_INDEX_INTERVAL = 2048;
+ // Retry budget for Summary.db prebuild reads. Tuned for S3/KMS throttling, which is the
+ // dominant failure mode under high prebuild fan-out: KMS rate-limit windows tend to recover
+ // on the order of seconds, so an initial 500 ms (jittered) and a 5 s cap give two meaningful
+ // re-reads beyond the SDK's own retries without pinning per-task threads for too long.
+ private static final int SUMMARY_READ_MAX_ATTEMPTS = 3;
+ private static final long SUMMARY_READ_INITIAL_BACKOFF_MILLIS = 500L;
+ private static final long SUMMARY_READ_MAX_BACKOFF_MILLIS = 5_000L;
+ private static final int DETAILED_FAILURE_LOG_LIMIT = 5;
+ private static final int FAILURE_LOG_INTERVAL = 10_000;
+
+ private SSTableTokenIndexBuilder()
+ {
+ }
+
+ public static TokenIndexShard buildShard(Iterator workItems,
+ BackupReaderFactory backupReaderFactory,
+ BackupReaderConfig backupReaderConfig,
+ String clusterName,
+ String datacenter,
+ CassandraVersion cassandraVersion,
+ int concurrency)
+ {
+ BackupReader reader = backupReaderFactory.create(backupReaderConfig);
+ CassandraBridge bridge = CassandraBridgeFactory.get(cassandraVersion);
+ int maxInFlight = Math.max(1, concurrency);
+ ExecutorService executor = Executors.newFixedThreadPool(maxInFlight);
+ ExecutorCompletionService completionService = new ExecutorCompletionService<>(executor);
+ int inFlight = 0;
+ FailureReporter failureReporter = new FailureReporter(clusterName, datacenter);
+ try
+ {
+ Map boundsBySSTable = new HashMap<>();
+ int missingCount = 0;
+ int errorCount = 0;
+ while (workItems.hasNext() || inFlight > 0)
+ {
+ while (workItems.hasNext() && inFlight < maxInFlight)
+ {
+ SSTableSummaryWorkItem workItem = workItems.next();
+ completionService.submit(scanWorkItem(reader, bridge, clusterName, datacenter, workItem, failureReporter));
+ inFlight++;
+ }
+
+ TokenIndexRecord record = take(completionService, failureReporter);
+ inFlight--;
+ if (record.bounds == null)
+ {
+ if (record.missing)
+ {
+ missingCount++;
+ }
+ else
+ {
+ errorCount++;
+ }
+ }
+ else
+ {
+ boundsBySSTable.put(record.key, record.bounds);
+ }
+ }
+ failureReporter.logShardSummary(boundsBySSTable.size(), missingCount, errorCount);
+ return new TokenIndexShard(boundsBySSTable, missingCount, errorCount);
+ }
+ finally
+ {
+ executor.shutdownNow();
+ reader.close();
+ }
+ }
+
+ private static Callable scanWorkItem(BackupReader reader,
+ CassandraBridge bridge,
+ String clusterName,
+ String datacenter,
+ SSTableSummaryWorkItem workItem,
+ FailureReporter failureReporter)
+ {
+ return () -> {
+ if (!workItem.componentSizes().containsKey(FileType.SUMMARY))
+ {
+ return TokenIndexRecord.missing(workItem.indexKey());
+ }
+ try
+ {
+ SummaryOnlySSTable ssTable = new SummaryOnlySSTable(reader, clusterName, datacenter, workItem);
+ SSTableSummary summary = executeWithRetry(() -> bridge.getSSTableSummary(Partitioner.Murmur3Partitioner,
+ ssTable,
+ DEFAULT_MIN_INDEX_INTERVAL,
+ DEFAULT_MAX_INDEX_INTERVAL),
+ SUMMARY_READ_MAX_ATTEMPTS,
+ SUMMARY_READ_INITIAL_BACKOFF_MILLIS,
+ SUMMARY_READ_MAX_BACKOFF_MILLIS);
+ return TokenIndexRecord.success(workItem.indexKey(),
+ new SSTableTokenBounds(toLong(summary.firstToken),
+ toLong(summary.lastToken)));
+ }
+ catch (Exception exception)
+ {
+ failureReporter.record(workItem, exception);
+ return TokenIndexRecord.error(workItem.indexKey());
+ }
+ };
+ }
+
+ private static TokenIndexRecord take(ExecutorCompletionService completionService,
+ FailureReporter failureReporter)
+ {
+ try
+ {
+ Future future = completionService.take();
+ return future.get();
+ }
+ catch (InterruptedException exception)
+ {
+ Thread.currentThread().interrupt();
+ throw new IllegalStateException("Interrupted while building SSTable token index shard", exception);
+ }
+ catch (ExecutionException exception)
+ {
+ failureReporter.record(null, exception);
+ return TokenIndexRecord.error(null);
+ }
+ }
+
+ /**
+ * Retry helper used by the prebuild path. Doubles an unjittered "schedule" on each failure
+ * and sleeps for a full-jitter sample uniformly in {@code [0, schedule]}. Doing the
+ * doubling on the unjittered base preserves an exponential ceiling; sleeping on the
+ * jittered sample spreads the herd of in-flight workers across the throttling window.
+ * See AWS SDK v2 {@code BackoffStrategy.fullJitter} for the same approach.
+ */
+ static T executeWithRetry(Callable action,
+ int maxAttempts,
+ long initialBackoffMillis,
+ long maxBackoffMillis) throws Exception
+ {
+ int attempts = Math.max(1, maxAttempts);
+ long scheduleMillis = Math.max(0L, initialBackoffMillis);
+ long ceilingMillis = Math.max(scheduleMillis, Math.max(0L, maxBackoffMillis));
+ Exception lastException = null;
+ for (int attempt = 1; attempt <= attempts; attempt++)
+ {
+ try
+ {
+ return action.call();
+ }
+ catch (Exception exception)
+ {
+ lastException = exception;
+ if (attempt == attempts || !isRetryableSummaryFailure(exception))
+ {
+ throw exception;
+ }
+ try
+ {
+ sleep(jitter(scheduleMillis));
+ }
+ catch (InterruptedException interrupted)
+ {
+ Thread.currentThread().interrupt();
+ interrupted.addSuppressed(exception);
+ throw interrupted;
+ }
+ // Advance the unjittered schedule so the next sleep window keeps growing
+ // even when this iteration sampled near zero.
+ scheduleMillis = Math.min(Math.max(scheduleMillis * 2L, 1L), ceilingMillis);
+ }
+ }
+ // Math.max(1, maxAttempts) above guarantees at least one iteration that assigns
+ // lastException before throwing, so this branch is unreachable in practice.
+ throw lastException == null ? new IllegalStateException("executeWithRetry exited without throwing or returning")
+ : lastException;
+ }
+
+ private static long jitter(long scheduleMillis)
+ {
+ if (scheduleMillis <= 0L)
+ {
+ return 0L;
+ }
+ // Full jitter: uniform in [0, scheduleMillis]. Spreads simultaneous failures across
+ // the throttling recovery window instead of issuing all retries at the same instant.
+ return ThreadLocalRandom.current().nextLong(scheduleMillis + 1L);
+ }
+
+ private static void sleep(long backoffMillis) throws InterruptedException
+ {
+ if (backoffMillis > 0L)
+ {
+ Thread.sleep(backoffMillis);
+ }
+ }
+
+ private static boolean isRetryableSummaryFailure(Throwable throwable)
+ {
+ // Keep this allow-list narrow. Summary parser failures (including EOF) are deterministic
+ // for a resolved object/range and retrying them only adds S3/KMS load. Let the AWS SDK mark
+ // service-side failures retryable, and only add local network socket failures here.
+ if (containsCause(throwable, EOFException.class))
+ {
+ return false;
+ }
+ return containsRetryableSdkException(throwable)
+ || containsCause(throwable, SocketException.class)
+ || containsCause(throwable, SocketTimeoutException.class);
+ }
+
+ private static boolean containsRetryableSdkException(Throwable throwable)
+ {
+ Throwable current = unwrapCompletion(throwable);
+ while (current.getCause() != null && current.getCause() != current)
+ {
+ if (current instanceof SdkException && ((SdkException) current).retryable())
+ {
+ return true;
+ }
+ current = unwrapCompletion(current.getCause());
+ }
+ return current instanceof SdkException && ((SdkException) current).retryable();
+ }
+
+ private static boolean containsCause(Throwable throwable, Class extends Throwable> causeClass)
+ {
+ Throwable current = unwrapCompletion(throwable);
+ while (current.getCause() != null && current.getCause() != current)
+ {
+ if (causeClass.isInstance(current))
+ {
+ return true;
+ }
+ current = unwrapCompletion(current.getCause());
+ }
+ return causeClass.isInstance(current);
+ }
+
+ private static Throwable rootCause(Throwable throwable)
+ {
+ Throwable current = unwrapCompletion(throwable);
+ while (current.getCause() != null && current.getCause() != current)
+ {
+ current = unwrapCompletion(current.getCause());
+ }
+ return current;
+ }
+
+ private static Throwable unwrapCompletion(Throwable throwable)
+ {
+ if ((throwable instanceof CompletionException || throwable instanceof ExecutionException) && throwable.getCause() != null)
+ {
+ return throwable.getCause();
+ }
+ return throwable;
+ }
+
+ private static long toLong(BigInteger token)
+ {
+ // SSTableTokenBounds stores Murmur3 tokens as long. Fail fast if a non-Murmur3 token
+ // ever reaches here instead of silently truncating to the low 64 bits.
+ return token.longValueExact();
+ }
+
+ private static final class FailureReporter
+ {
+ private final String clusterName;
+ private final String datacenter;
+ private final AtomicInteger failureCount = new AtomicInteger();
+ private final ConcurrentMap failuresByException = new ConcurrentHashMap<>();
+
+ private FailureReporter(String clusterName, String datacenter)
+ {
+ this.clusterName = clusterName;
+ this.datacenter = datacenter;
+ }
+
+ private void record(@Nullable SSTableSummaryWorkItem workItem, Throwable throwable)
+ {
+ Throwable rootCause = rootCause(throwable);
+ String exceptionClass = rootCause.getClass().getName();
+ failuresByException.computeIfAbsent(exceptionClass, key -> new AtomicInteger()).incrementAndGet();
+
+ int count = failureCount.incrementAndGet();
+ if (count <= DETAILED_FAILURE_LOG_LIMIT)
+ {
+ LOGGER.warn("Failed to prebuild SSTable token index from Summary.db "
+ + "cluster={} datacenter={} failureCount={} sstableKey={} token={} "
+ + "summarySizeBytes={} exceptionClass={} message={}",
+ clusterName,
+ datacenter,
+ count,
+ workItem == null ? null : workItem.sstableKey(),
+ workItem == null ? null : workItem.token(),
+ workItem == null ? null : workItem.componentSizes().get(FileType.SUMMARY),
+ exceptionClass,
+ rootCause.getMessage(),
+ rootCause);
+ }
+ else if (count % FAILURE_LOG_INTERVAL == 0)
+ {
+ LOGGER.warn("Failed to prebuild SSTable token index from Summary.db "
+ + "cluster={} datacenter={} failureCount={} sstableKey={} token={} "
+ + "summarySizeBytes={} exceptionClass={} message={}",
+ clusterName,
+ datacenter,
+ count,
+ workItem == null ? null : workItem.sstableKey(),
+ workItem == null ? null : workItem.token(),
+ workItem == null ? null : workItem.componentSizes().get(FileType.SUMMARY),
+ exceptionClass,
+ rootCause.getMessage());
+ }
+ }
+
+ private void logShardSummary(int successCount, int missingCount, int errorCount)
+ {
+ if (errorCount > 0)
+ {
+ LOGGER.warn("Completed SSTable token index shard with fail-open Summary.db errors "
+ + "cluster={} datacenter={} indexed={} missing={} errors={} failureTypes={}",
+ clusterName,
+ datacenter,
+ successCount,
+ missingCount,
+ errorCount,
+ failureCounts());
+ }
+ }
+
+ private Map failureCounts()
+ {
+ Map counts = new HashMap<>();
+ failuresByException.forEach((exceptionClass, count) -> counts.put(exceptionClass, count.get()));
+ return Collections.unmodifiableMap(counts);
+ }
+ }
+
+ private static final class TokenIndexRecord
+ {
+ @Nullable
+ private final SSTableIndexKey key;
+ @Nullable
+ private final SSTableTokenBounds bounds;
+ private final boolean missing;
+
+ private TokenIndexRecord(@Nullable SSTableIndexKey key, @Nullable SSTableTokenBounds bounds, boolean missing)
+ {
+ this.key = key;
+ this.bounds = bounds;
+ this.missing = missing;
+ }
+
+ private static TokenIndexRecord success(SSTableIndexKey key, SSTableTokenBounds bounds)
+ {
+ return new TokenIndexRecord(key, bounds, false);
+ }
+
+ private static TokenIndexRecord missing(SSTableIndexKey key)
+ {
+ return new TokenIndexRecord(key, null, true);
+ }
+
+ private static TokenIndexRecord error(@Nullable SSTableIndexKey key)
+ {
+ return new TokenIndexRecord(key, null, false);
+ }
+ }
+
+ private static final class SummaryOnlySSTable extends SSTable
+ {
+ private final BackupReader reader;
+ private final String clusterName;
+ private final String datacenter;
+ private final SSTableSummaryWorkItem workItem;
+ private final ConcurrentMap actualSizes = new ConcurrentHashMap<>();
+
+ private SummaryOnlySSTable(BackupReader reader,
+ String clusterName,
+ String datacenter,
+ SSTableSummaryWorkItem workItem)
+ {
+ this.reader = reader;
+ this.clusterName = clusterName;
+ this.datacenter = datacenter;
+ this.workItem = workItem;
+ }
+
+ @Nullable
+ @Override
+ protected InputStream openInputStream(FileType fileType)
+ {
+ Long size = workItem.componentSizes().get(fileType);
+ if (size == null || size <= 0)
+ {
+ return null;
+ }
+ byte[] bytes = reader.readMutableMetadataAsync(clusterName,
+ datacenter,
+ workItem.token(),
+ workItem.sstableKey(),
+ fileType,
+ size,
+ Stats.DoNothingStats.INSTANCE).join();
+ actualSizes.put(fileType, (long) bytes.length);
+ return new ByteArrayInputStream(bytes);
+ }
+
+ @Override
+ public long length(FileType fileType)
+ {
+ Long actualSize = actualSizes.get(fileType);
+ if (actualSize != null)
+ {
+ return actualSize;
+ }
+ Long size = workItem.componentSizes().get(fileType);
+ if (size == null)
+ {
+ throw new IncompleteSSTableException(fileType);
+ }
+ return size;
+ }
+
+ @Override
+ public boolean isMissing(FileType fileType)
+ {
+ return !workItem.componentSizes().containsKey(fileType);
+ }
+
+ @Override
+ public String getDataFileName()
+ {
+ return workItem.sstableKey().getDataFileName();
+ }
+ }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/TokenIndexShard.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/TokenIndexShard.java
new file mode 100644
index 000000000..94c8174a1
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/TokenIndexShard.java
@@ -0,0 +1,65 @@
+/*
+ * 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.io.Serializable;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+public final class TokenIndexShard implements Serializable
+{
+ private static final long serialVersionUID = 2026042501L;
+
+ private final Map boundsBySSTable;
+ private final int successCount;
+ private final int missingCount;
+ private final int errorCount;
+
+ public TokenIndexShard(Map boundsBySSTable,
+ int missingCount,
+ int errorCount)
+ {
+ this.boundsBySSTable = Collections.unmodifiableMap(new HashMap<>(boundsBySSTable));
+ this.successCount = boundsBySSTable.size();
+ this.missingCount = missingCount;
+ this.errorCount = errorCount;
+ }
+
+ public Map boundsBySSTable()
+ {
+ return boundsBySSTable;
+ }
+
+ public int successCount()
+ {
+ return successCount;
+ }
+
+ public int missingCount()
+ {
+ return missingCount;
+ }
+
+ public int errorCount()
+ {
+ return errorCount;
+ }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/backup/BackupReader.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/backup/BackupReader.java
new file mode 100644
index 000000000..3f5c98417
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/backup/BackupReader.java
@@ -0,0 +1,302 @@
+/*
+ * 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.backup;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.LongConsumer;
+
+import com.google.common.collect.RangeMap;
+import org.jetbrains.annotations.NotNull;
+
+import org.apache.cassandra.analytics.stats.Stats;
+import org.apache.cassandra.spark.data.FileType;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.apache.cassandra.spark.data.S3ClientConfig;
+import org.apache.cassandra.spark.data.SSTableKey;
+import org.apache.cassandra.spark.data.partitioner.CassandraInstance;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.utils.streaming.StreamConsumer;
+
+/**
+ * Pluggable backup-reader API. Implementations encapsulate the format-specific knowledge for a
+ * given backup implementation. Concrete factories are registered with {@link BackupReaderRegistry} on the
+ * driver and selected via the {@code backupReaderType} option; the data layer programs strictly
+ * against this interface.
+ *
+ * Implementations are {@link Serializable} so the data layer can ship a configured reader to
+ * executors inside the Spark task closure, and {@link AutoCloseable} so the data layer can
+ * release reader-local resources on shutdown.
+ */
+public interface BackupReader extends Serializable, AutoCloseable
+{
+ /**
+ * Populates the reader's per-(cluster, keyspace, table) cache by walking the backup manifest.
+ *
+ * @param clusterName logical cluster identity (UUID or human-readable name)
+ * @param keyspace Cassandra keyspace
+ * @param table Cassandra table
+ * @param datacenter datacenter to read from
+ */
+ void initializeSSTableInfoCache(String clusterName, String keyspace, String table, String datacenter)
+ throws IllegalArgumentException;
+
+ /**
+ * Lists Cassandra instances present in the backup for the given dataset slice.
+ *
+ * @param clusterName logical cluster identity
+ * @param keyspace Cassandra keyspace
+ * @param table Cassandra table
+ * @param datacenter datacenter to read from
+ * @return Cassandra instances contributing to this slice
+ */
+ List instances(String clusterName, String keyspace, String table, String datacenter);
+
+ /**
+ * Discovers SSTables across all nodes for the given dataset slice.
+ *
+ * @param clusterName logical cluster identity
+ * @param keyspace Cassandra keyspace
+ * @param table Cassandra table
+ * @param datacenter datacenter to read from
+ * @return map of SSTable key to per-file-type sizes
+ */
+ Map> sstables(String clusterName, String keyspace, String table, String datacenter);
+
+ /**
+ * Discovers SSTables for a single node.
+ *
+ * @param clusterName logical cluster identity
+ * @param keyspace Cassandra keyspace
+ * @param table Cassandra table
+ * @param datacenter datacenter to read from
+ * @param nodeId node identifier whose SSTables to enumerate
+ * @return map of SSTable key to per-file-type sizes for the given node
+ */
+ Map> sstables(String clusterName,
+ String keyspace,
+ String table,
+ String datacenter,
+ String nodeId);
+
+ /**
+ * Buffered ranged GET; completes with the full {@code [start, end]} byte range.
+ *
+ * @param clusterName logical cluster identity
+ * @param datacenter datacenter to read from
+ * @param token Cassandra token (used for path resolution by some implementations)
+ * @param sstableKey identifies the SSTable
+ * @param fileType which SSTable component to read
+ * @param start inclusive byte offset
+ * @param end inclusive byte offset
+ * @param stats per-task stats sink for S3 operation metrics
+ * @return future completing with the read bytes
+ */
+ CompletableFuture readAsync(String clusterName,
+ String datacenter,
+ String token,
+ SSTableKey sstableKey,
+ FileType fileType,
+ long start,
+ long end,
+ Stats stats);
+
+ /**
+ * Buffered read for {@linkplain FileType#isMutableMetadata mutable-metadata} components whose
+ * on-disk size may change over time.
+ *
+ * @param clusterName logical cluster identity
+ * @param datacenter datacenter to read from
+ * @param token Cassandra token
+ * @param sstableKey identifies the SSTable
+ * @param fileType mutable-metadata component to read
+ * @param manifestSize sizing hint from the manifest (may differ from actual on-disk size)
+ * @param stats per-task stats sink
+ * @return future completing with the read bytes
+ */
+ CompletableFuture readMutableMetadataAsync(String clusterName,
+ String datacenter,
+ String token,
+ SSTableKey sstableKey,
+ FileType fileType,
+ long manifestSize,
+ Stats stats);
+
+ /**
+ * Streaming ranged GET. Chunks are pushed to {@code consumer} and the future completes on termination.
+ *
+ * @param clusterName logical cluster identity
+ * @param datacenter datacenter to read from
+ * @param token Cassandra token
+ * @param sstableKey identifies the SSTable
+ * @param fileType SSTable component to read
+ * @param start inclusive byte offset
+ * @param end inclusive byte offset
+ * @param consumer receives streamed chunks
+ * @param stats per-task stats sink
+ * @return future completing when the stream terminates
+ */
+ CompletableFuture getAsync(String clusterName,
+ String datacenter,
+ String token,
+ SSTableKey sstableKey,
+ FileType fileType,
+ long start,
+ long end,
+ @NotNull StreamConsumer consumer,
+ Stats stats);
+
+ /**
+ * Streaming variant of {@link #readMutableMetadataAsync}.
+ *
+ * @param clusterName logical cluster identity
+ * @param datacenter datacenter to read from
+ * @param token Cassandra token
+ * @param sstableKey identifies the SSTable
+ * @param fileType mutable-metadata component to read
+ * @param start inclusive byte offset
+ * @param end inclusive byte offset
+ * @param consumer receives streamed chunks
+ * @param actualSizeConsumer invoked once with the resolved size when the GET response is received
+ * @param manifestSize sizing hint from the manifest
+ * @param stats per-task stats sink
+ * @return future completing when the stream terminates
+ */
+ CompletableFuture getMutableMetadataAsync(String clusterName,
+ String datacenter,
+ String token,
+ SSTableKey sstableKey,
+ FileType fileType,
+ long start,
+ long end,
+ @NotNull StreamConsumer consumer,
+ LongConsumer actualSizeConsumer,
+ long manifestSize,
+ Stats stats);
+
+ /**
+ * Checks whether a specific SSTable component exists in the backup.
+ *
+ * @param clusterName logical cluster identity
+ * @param datacenter datacenter to read from
+ * @param token Cassandra token
+ * @param sstableKey identifies the SSTable
+ * @param fileType SSTable component
+ * @param stats per-task stats sink
+ * @return {@code true} iff the component exists in the backup
+ */
+ boolean exists(String clusterName,
+ String datacenter,
+ String token,
+ SSTableKey sstableKey,
+ FileType fileType,
+ Stats stats);
+
+ /**
+ * Earliest per-node snapshot epoch (seconds) across the contributing nodes.
+ *
+ * @param clusterName logical cluster identity
+ * @param keyspace Cassandra keyspace
+ * @param table Cassandra table
+ * @param datacenter datacenter to read from
+ * @return epoch seconds of the earliest contributing snapshot
+ */
+ long getSnapshotEpochSecond(String clusterName, String keyspace, String table, String datacenter);
+
+ /**
+ * Latest per-node snapshot epoch (seconds) across the contributing nodes.
+ *
+ * @param clusterName logical cluster identity
+ * @param keyspace Cassandra keyspace
+ * @param table Cassandra table
+ * @param datacenter datacenter to read from
+ * @return epoch seconds of the latest contributing snapshot
+ */
+ long getLatestSnapshotEpochSecond(String clusterName, String keyspace, String table, String datacenter);
+
+ /**
+ * Per-cluster manifest fingerprint. Used by the data layer's intern cache to disambiguate
+ * manifests that share {@code (earliestEpoch, latestEpoch)} bounds but differ in per-node
+ * contributions.
+ *
+ * @param clusterName logical cluster identity
+ * @return stable fingerprint string identifying the materialized manifest set
+ */
+ String getManifestFingerprint(String clusterName);
+
+ /**
+ * Returns the underlying {@link S3ClientConfig}. Used for intern-cache identity.
+ *
+ * @return the S3 client configuration this reader was constructed with
+ */
+ S3ClientConfig s3Config();
+
+ /**
+ * Returns the S3 bucket the reader was constructed against. Used for intern-cache identity.
+ *
+ * @return the S3 bucket name
+ */
+ String bucket();
+
+ /**
+ * Returns an authoritative per-range replica mapping for the given dataset slice, or
+ * {@link Optional#empty()} if this reader has no rack-aware source to draw from. The
+ * returned {@link RangeMap} is keyed by {@code (start, end]} sub-ranges that tile the
+ * full token ring and is consumed by the 5-arg
+ * {@link org.apache.cassandra.spark.data.partitioner.CassandraRing} constructor; each
+ * {@link CassandraInstance} in a replica list MUST be element-equal to one in
+ * {@code instances} (same {@code nodeName} / {@code dataCenter} / {@code token}).
+ *
+ * Returning {@link Optional#empty()} is a benign signal — callers must fall back to the
+ * rack-unaware naive derivation that the 4-arg {@code CassandraRing} constructor
+ * performs (see CASSANALYTICS-79). Implementations should reserve exceptions for
+ * actually-broken state (parse error, range gap/overlap, unknown replica id, missing
+ * coverage) where silently degrading to naive would mask data integrity issues.
+ *
+ * @param clusterName logical cluster identity (UUID or human-readable name)
+ * @param keyspace Cassandra keyspace
+ * @param table Cassandra table
+ * @param datacenter datacenter to read from
+ * @param partitioner partitioner whose {@code (minToken, maxToken]} bounds the ring
+ * @param rf per-DC replication factor configuration
+ * @param instances Cassandra instances contributing to this slice (same set the
+ * caller passes to {@code CassandraRing})
+ * @return authoritative per-range replicas, or {@link Optional#empty()} for fallback
+ */
+ default Optional>> buildRackAwareReplicas(
+ String clusterName,
+ String keyspace,
+ String table,
+ String datacenter,
+ Partitioner partitioner,
+ ReplicationFactor rf,
+ List instances)
+ {
+ return Optional.empty();
+ }
+
+ /** Closes any reader-local resources. Should be a no-op for resources owned by shared pools. */
+ @Override
+ void close();
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/backup/BackupReaderConfig.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/backup/BackupReaderConfig.java
new file mode 100644
index 000000000..82eba1b62
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/backup/BackupReaderConfig.java
@@ -0,0 +1,82 @@
+/*
+ * 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.backup;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Map;
+
+import org.apache.cassandra.spark.data.S3ClientConfig;
+
+/**
+ * Configuration bundle passed to {@link BackupReaderFactory#create(BackupReaderConfig)}. Holds
+ * the inputs an implementation needs to instantiate a {@link BackupReader} (S3 client config and
+ * arbitrary string-keyed custom properties for vendor-specific knobs).
+ *
+ * Task metrics sinks are supplied on individual {@link BackupReader} read calls.
+ */
+public final class BackupReaderConfig implements Serializable
+{
+ private static final long serialVersionUID = 2L;
+
+ private final S3ClientConfig s3Config;
+ private final Map customProperties;
+
+ private BackupReaderConfig(S3ClientConfig s3Config, Map customProperties)
+ {
+ this.s3Config = s3Config;
+ this.customProperties = customProperties != null ? customProperties : Collections.emptyMap();
+ }
+
+ /**
+ * Convenience factory: no custom properties.
+ *
+ * @param s3Config S3 client configuration
+ * @return a config carrying only {@code s3Config} (custom properties empty)
+ */
+ public static BackupReaderConfig of(S3ClientConfig s3Config)
+ {
+ return new BackupReaderConfig(s3Config, Collections.emptyMap());
+ }
+
+ /**
+ * Factory with custom properties pre-populated.
+ *
+ * @param s3Config S3 client configuration
+ * @param customProperties vendor-specific string-keyed knobs; may be {@code null} (treated as empty)
+ * @return a config carrying {@code s3Config} and the supplied properties
+ */
+ public static BackupReaderConfig of(S3ClientConfig s3Config, Map customProperties)
+ {
+ return new BackupReaderConfig(s3Config, customProperties);
+ }
+
+ /** @return the S3 client configuration */
+ public S3ClientConfig s3Config()
+ {
+ return s3Config;
+ }
+
+ /** @return Specific custom properties (never {@code null}; may be empty) */
+ public Map customProperties()
+ {
+ return customProperties;
+ }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/backup/BackupReaderFactory.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/backup/BackupReaderFactory.java
new file mode 100644
index 000000000..9502b5593
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/backup/BackupReaderFactory.java
@@ -0,0 +1,40 @@
+/*
+ * 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.backup;
+
+import java.io.Serializable;
+
+/**
+ * {@link Serializable} factory used to construct {@link BackupReader} instances. Extending
+ * {@code Serializable} is load-bearing: the prebuild path captures a factory into a Spark
+ * closure and Spark's closure serializer (Java-only) ships it to executors, where it runs to
+ * produce a freshly-constructed reader. Implementations must not capture non-serializable state.
+ */
+@FunctionalInterface
+public interface BackupReaderFactory extends Serializable
+{
+ /**
+ * Creates a {@link BackupReader} from the given configuration.
+ *
+ * @param config configuration bundle
+ * @return a configured {@link BackupReader} instance
+ */
+ BackupReader create(BackupReaderConfig config);
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/backup/BackupReaderRegistry.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/backup/BackupReaderRegistry.java
new file mode 100644
index 000000000..f625a297c
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/backup/BackupReaderRegistry.java
@@ -0,0 +1,133 @@
+/*
+ * 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.backup;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Driver-side registry of {@link BackupReaderFactory} implementations keyed by a short string
+ * type name. Drivers register their factory once at startup; the data layer and prebuild path
+ * then look it up by the {@code backupReaderType} option. Executors receive the factory through
+ * closure serialization and never consult this registry.
+ *
+ * There is no default type. {@link #factoryFor(String)} and
+ * {@link #create(String, BackupReaderConfig)} throw {@link IllegalArgumentException} listing the
+ * currently-registered types when no match is found.
+ */
+public final class BackupReaderRegistry
+{
+ private static final Logger LOGGER = LoggerFactory.getLogger(BackupReaderRegistry.class);
+
+ // Skip-list map so registered-types listings (used in error messages) are alphabetized.
+ private static final ConcurrentSkipListMap FACTORIES = new ConcurrentSkipListMap<>();
+
+ private BackupReaderRegistry()
+ {
+ }
+
+ /**
+ * Registers a factory for the given type. Re-registering the same type replaces the previous
+ * factory (the most recent registration wins).
+ *
+ * @param type short type name; must be non-null and non-empty
+ * @param factory factory implementation; must be non-null
+ * @throws IllegalArgumentException if {@code type} is null/empty or {@code factory} is null
+ */
+ public static void register(String type, BackupReaderFactory factory)
+ {
+ if (type == null || type.isEmpty())
+ {
+ throw new IllegalArgumentException("backupReaderType cannot be null or empty");
+ }
+ if (factory == null)
+ {
+ throw new IllegalArgumentException("BackupReaderFactory cannot be null");
+ }
+ BackupReaderFactory previous = FACTORIES.put(type, factory);
+ if (previous != null)
+ {
+ LOGGER.warn("BackupReaderRegistry: replacing existing factory for type={}", type);
+ }
+ else
+ {
+ LOGGER.info("BackupReaderRegistry: registered factory for type={}", type);
+ }
+ }
+
+ /**
+ * Returns the factory registered for the given type, or throws if none is registered.
+ *
+ * @param type short type name to resolve
+ * @return the registered factory
+ * @throws IllegalArgumentException if no factory is registered for {@code type}
+ */
+ public static BackupReaderFactory factoryFor(String type)
+ {
+ BackupReaderFactory factory = FACTORIES.get(type);
+ if (factory == null)
+ {
+ throw new IllegalArgumentException(formatMissingTypeMessage(type));
+ }
+ return factory;
+ }
+
+ /**
+ * Resolves the factory for {@code type} and invokes it with {@code config}.
+ *
+ * @param type short type name to resolve
+ * @param config configuration bundle passed to the factory
+ * @return a freshly-constructed {@link BackupReader}
+ * @throws IllegalArgumentException if no factory is registered for {@code type}
+ */
+ public static BackupReader create(String type, BackupReaderConfig config)
+ {
+ BackupReader reader = factoryFor(type).create(config);
+ LOGGER.info("BackupReaderRegistry: created backup reader of type {}", type);
+ return reader;
+ }
+
+ /**
+ * Returns an unmodifiable view of the registered type names.
+ *
+ * @return alphabetically-ordered, unmodifiable set of registered type names
+ */
+ public static Set registeredTypes()
+ {
+ return Collections.unmodifiableSet(new TreeMap<>(FACTORIES).keySet());
+ }
+
+ private static String formatMissingTypeMessage(String requestedType)
+ {
+ Set known = registeredTypes();
+ return String.format(
+ "no backup reader registered for type '%s'; "
+ + "call BackupReaderRegistry.register(...) at driver startup, "
+ + "or set the 'backupReaderType' option to a registered type. "
+ + "Registered types: %s",
+ requestedType,
+ known.isEmpty() ? "[]" : known.toString());
+ }
+}
diff --git a/cassandra-analytics-core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/cassandra-analytics-core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
index 9e699ada4..01c7d0c9c 100644
--- a/cassandra-analytics-core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
+++ b/cassandra-analytics-core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -1,2 +1,3 @@
org.apache.cassandra.spark.sparksql.CassandraDataSource
org.apache.cassandra.spark.sparksql.CassandraDataSink
+org.apache.cassandra.spark.sparksql.S3CassandraDataSource
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraPartitionReaderFactory.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraPartitionReaderFactory.java
index b2a7df7ab..0b1a34150 100644
--- a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraPartitionReaderFactory.java
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraPartitionReaderFactory.java
@@ -25,8 +25,11 @@
import org.slf4j.LoggerFactory;
import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.cassandra.spark.data.S3CassandraDataLayer;
+import org.apache.cassandra.spark.data.SSTableTokenIndex;
import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
import org.apache.spark.TaskContext;
+import org.apache.spark.broadcast.Broadcast;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.connector.read.InputPartition;
import org.apache.spark.sql.connector.read.PartitionReader;
@@ -39,14 +42,17 @@ class CassandraPartitionReaderFactory implements PartitionReaderFactory
final DataLayer dataLayer;
final StructType requiredSchema;
final List partitionKeyFilters;
+ final Broadcast sstableTokenIndexBroadcast;
CassandraPartitionReaderFactory(DataLayer dataLayer,
StructType requiredSchema,
- List partitionKeyFilters)
+ List partitionKeyFilters,
+ Broadcast sstableTokenIndexBroadcast)
{
this.dataLayer = dataLayer;
this.requiredSchema = requiredSchema;
this.partitionKeyFilters = partitionKeyFilters;
+ this.sstableTokenIndexBroadcast = sstableTokenIndexBroadcast;
}
@Override
@@ -64,6 +70,10 @@ public PartitionReader createReader(InputPartition partition)
+ "Using TaskContext to determine the partitionId type={}, partitionId={}",
partition.getClass().getName(), partitionId);
}
+ if (sstableTokenIndexBroadcast != null && dataLayer instanceof S3CassandraDataLayer)
+ {
+ ((S3CassandraDataLayer) dataLayer).setSSTableTokenIndex(sstableTokenIndexBroadcast.value());
+ }
return new SparkRowIterator(partitionId, dataLayer, requiredSchema, partitionKeyFilters);
}
}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraPartitioning.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraPartitioning.java
index ac8d68732..a1730e1c4 100644
--- a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraPartitioning.java
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraPartitioning.java
@@ -19,42 +19,24 @@
package org.apache.cassandra.spark.sparksql;
-import java.util.Arrays;
-import java.util.List;
-import java.util.stream.Collectors;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
-import org.apache.cassandra.spark.data.CqlField;
-import org.apache.cassandra.spark.data.DataLayer;
-import org.apache.spark.sql.connector.read.partitioning.ClusteredDistribution;
-import org.apache.spark.sql.connector.read.partitioning.Distribution;
-import org.apache.spark.sql.connector.read.partitioning.Partitioning;
-
-class CassandraPartitioning implements Partitioning
+/**
+ * Reports the number of Spark partitions to the planner via {@link UnknownPartitioning}.
+ *
+ * Spark 3.3+ replaced the older {@code Partitioning#satisfy(Distribution)} contract with
+ * concrete partitioning types. The Cassandra reader can report its partition count, but not a
+ * key grouping Spark can safely use for storage-partitioned joins.
+ *
+ *
Each {@link CassandraInputPartition} is a token range. Although each row's token is derived
+ * from the partition key, rows in the same token range usually have many distinct partition keys
+ * and many distinct tokens. That does not satisfy {@code KeyGroupedPartitioning}'s contract that
+ * every row in one Spark partition evaluates to the same partition value.
+ */
+class CassandraPartitioning extends UnknownPartitioning
{
- final DataLayer dataLayer;
-
- CassandraPartitioning(DataLayer dataLayer)
- {
- this.dataLayer = dataLayer;
- }
-
- @Override
- public int numPartitions()
- {
- return dataLayer.partitionCount();
- }
-
- @Override
- public boolean satisfy(Distribution distribution)
+ CassandraPartitioning(int numPartitions)
{
- if (distribution instanceof ClusteredDistribution)
- {
- String[] clusteredCols = ((ClusteredDistribution) distribution).clusteredColumns;
- List partitionKeys = dataLayer.cqlTable().partitionKeys().stream()
- .map(CqlField::name)
- .collect(Collectors.toList());
- return Arrays.asList(clusteredCols).containsAll(partitionKeys);
- }
- return false;
+ super(numPartitions);
}
}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraScanBuilder.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraScanBuilder.java
index 2f821398d..ff115eca9 100644
--- a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraScanBuilder.java
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraScanBuilder.java
@@ -24,42 +24,75 @@
import java.util.AbstractMap;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
+import java.util.OptionalLong;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import org.apache.cassandra.spark.data.CqlField;
import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.cassandra.spark.data.S3CassandraDataLayer;
+import org.apache.cassandra.spark.data.SSTableTokenIndex;
import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.cassandra.spark.sparksql.metrics.TotalSummaryReadDuration;
+import org.apache.cassandra.spark.sparksql.metrics.TotalOpenedSSTableDuration;
+import org.apache.cassandra.spark.sparksql.metrics.TotalCorruptSSTableCount;
+import org.apache.cassandra.spark.sparksql.metrics.TotalMutableMetadataDriftCount;
+import org.apache.cassandra.spark.sparksql.metrics.TotalMutableMetadataHeadFallbackCount;
+import org.apache.cassandra.spark.sparksql.metrics.TotalSkippedSSTableCount;
+import org.apache.cassandra.spark.sparksql.metrics.TotalS3HeadObjectDuration;
+import org.apache.cassandra.spark.sparksql.metrics.TotalS3GetObjectDuration;
import org.apache.cassandra.spark.utils.FilterUtils;
+import org.apache.spark.sql.connector.metric.CustomMetric;
import org.apache.spark.sql.connector.read.Batch;
import org.apache.spark.sql.connector.read.InputPartition;
import org.apache.spark.sql.connector.read.PartitionReaderFactory;
import org.apache.spark.sql.connector.read.Scan;
import org.apache.spark.sql.connector.read.ScanBuilder;
+import org.apache.spark.sql.connector.read.Statistics;
import org.apache.spark.sql.connector.read.SupportsPushDownFilters;
import org.apache.spark.sql.connector.read.SupportsPushDownRequiredColumns;
import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.SupportsReportStatistics;
import org.apache.spark.sql.connector.read.partitioning.Partitioning;
import org.apache.spark.sql.sources.Filter;
import org.apache.spark.sql.types.StructType;
import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+import org.apache.spark.broadcast.Broadcast;
+import org.jetbrains.annotations.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
-class CassandraScanBuilder implements ScanBuilder, Scan, Batch, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportPartitioning
+class CassandraScanBuilder implements ScanBuilder, Scan, Batch, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportPartitioning,
+ SupportsReportStatistics
{
+ private static final Logger LOGGER = LoggerFactory.getLogger(CassandraScanBuilder.class);
+
final DataLayer dataLayer;
final StructType schema;
final CaseInsensitiveStringMap options;
StructType requiredSchema = null;
Filter[] pushedFilters = new Filter[0];
+ @Nullable
+ private final Broadcast sstableTokenIndexBroadcast;
CassandraScanBuilder(DataLayer dataLayer, StructType schema, CaseInsensitiveStringMap options)
+ {
+ this(dataLayer, schema, options, null);
+ }
+
+ CassandraScanBuilder(DataLayer dataLayer,
+ StructType schema,
+ CaseInsensitiveStringMap options,
+ @Nullable Broadcast sstableTokenIndexBroadcast)
{
this.dataLayer = dataLayer;
this.schema = schema;
this.options = options;
+ this.sstableTokenIndexBroadcast = sstableTokenIndexBroadcast;
}
@Override
@@ -95,7 +128,7 @@ public Filter[] pushedFilters()
@Override
public StructType readSchema()
{
- return requiredSchema;
+ return requiredSchema == null ? schema : requiredSchema;
}
@Override
@@ -115,13 +148,22 @@ public InputPartition[] planInputPartitions()
@Override
public PartitionReaderFactory createReaderFactory()
{
- return new CassandraPartitionReaderFactory(dataLayer, requiredSchema, buildPartitionKeyFilters());
+ if (sstableTokenIndexBroadcast == null && dataLayer instanceof S3CassandraDataLayer
+ && ((S3CassandraDataLayer) dataLayer).sstableTokenIndexEnabled())
+ {
+ LOGGER.warn("SSTable token index is enabled but no prebuilt read context was provided. "
+ + "Proceeding without token-index pruning.");
+ }
+ return new CassandraPartitionReaderFactory(dataLayer, readSchema(), buildPartitionKeyFilters(), sstableTokenIndexBroadcast);
}
@Override
public Partitioning outputPartitioning()
{
- return new CassandraPartitioning(dataLayer);
+ // See CassandraPartitioning for why we report UnknownPartitioning (token-range Spark
+ // partitions are not key-grouped by Cassandra partition key, so KeyGroupedPartitioning
+ // is the wrong contract here).
+ return new CassandraPartitioning(dataLayer.partitionCount());
}
private List buildPartitionKeyFilters()
@@ -146,4 +188,30 @@ private PartitionKeyFilter buildFilter(List keys)
AbstractMap.SimpleEntry filterKey = dataLayer.bridge().getPartitionKey(dataLayer.cqlTable(), dataLayer.partitioner(), keys);
return PartitionKeyFilter.create(filterKey.getKey(), filterKey.getValue());
}
+
+ @Override
+ public Statistics estimateStatistics()
+ {
+ OptionalLong estimatedSizeInBytes = dataLayer.calculateTotalSSTableSize();
+ return new CassandraSourceStatistics(
+ estimatedSizeInBytes,
+ OptionalLong.empty(), // numRows - not calculated for now
+ Collections.emptyMap() // columnStats - empty for now
+ );
+ }
+
+ @Override
+ public CustomMetric[] supportedCustomMetrics()
+ {
+ return new CustomMetric[] {
+ new TotalSummaryReadDuration(),
+ new TotalOpenedSSTableDuration(),
+ new TotalCorruptSSTableCount(),
+ new TotalSkippedSSTableCount(),
+ new TotalS3HeadObjectDuration(),
+ new TotalS3GetObjectDuration(),
+ new TotalMutableMetadataDriftCount(),
+ new TotalMutableMetadataHeadFallbackCount()
+ };
+ }
}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraSourceStatistics.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraSourceStatistics.java
new file mode 100644
index 000000000..42bc1951d
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraSourceStatistics.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.sparksql;
+
+import java.util.Map;
+import java.util.OptionalLong;
+
+import org.apache.spark.sql.connector.expressions.NamedReference;
+import org.apache.spark.sql.connector.read.Statistics;
+import org.apache.spark.sql.connector.read.colstats.ColumnStatistics;
+
+public class CassandraSourceStatistics implements Statistics
+{
+ private final OptionalLong sizeInBytes;
+ private final OptionalLong numRows;
+ private final Map colstats;
+
+ CassandraSourceStatistics(OptionalLong sizeInBytes, OptionalLong numRows, Map colstats)
+ {
+ this.sizeInBytes = sizeInBytes;
+ this.numRows = numRows;
+ this.colstats = colstats;
+ }
+
+ @Override
+ public OptionalLong sizeInBytes()
+ {
+ return sizeInBytes;
+ }
+
+ @Override
+ public OptionalLong numRows()
+ {
+ return numRows;
+ }
+
+ @Override
+ public Map columnStats()
+ {
+ return colstats;
+ }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraTable.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraTable.java
index f78e4c5b5..bb9a82f4d 100644
--- a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraTable.java
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraTable.java
@@ -25,22 +25,35 @@
import com.google.common.collect.ImmutableList;
import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.cassandra.spark.data.SSTableTokenIndex;
+import org.apache.spark.broadcast.Broadcast;
import org.apache.spark.sql.connector.catalog.SupportsRead;
import org.apache.spark.sql.connector.catalog.Table;
import org.apache.spark.sql.connector.catalog.TableCapability;
import org.apache.spark.sql.connector.read.ScanBuilder;
import org.apache.spark.sql.types.StructType;
import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+import org.jetbrains.annotations.Nullable;
class CassandraTable implements Table, SupportsRead
{
private final DataLayer dataLayer;
private final StructType schema;
+ @Nullable
+ private final Broadcast sstableTokenIndexBroadcast;
CassandraTable(DataLayer dataLayer, StructType schema)
+ {
+ this(dataLayer, schema, null);
+ }
+
+ CassandraTable(DataLayer dataLayer,
+ StructType schema,
+ @Nullable Broadcast sstableTokenIndexBroadcast)
{
this.dataLayer = dataLayer;
this.schema = schema;
+ this.sstableTokenIndexBroadcast = sstableTokenIndexBroadcast;
}
@Override
@@ -58,12 +71,14 @@ public StructType schema()
@Override
public Set capabilities()
{
+ // Keep upstream's [BATCH_READ, MICRO_BATCH_READ] capabilities; the S3 batch reader path
+ // simply ignores MICRO_BATCH_READ because it doesn't register a streaming source.
return new HashSet<>(ImmutableList.of(TableCapability.BATCH_READ, TableCapability.MICRO_BATCH_READ));
}
@Override
public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options)
{
- return new CassandraScanBuilder(dataLayer, schema, options);
+ return new CassandraScanBuilder(dataLayer, schema, options, sstableTokenIndexBroadcast);
}
}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraTableProvider.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraTableProvider.java
index a2e86b10f..cc26e5bd6 100644
--- a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraTableProvider.java
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraTableProvider.java
@@ -19,34 +19,58 @@
package org.apache.cassandra.spark.sparksql;
+import java.util.Locale;
import java.util.Map;
+import java.util.TreeMap;
import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.cassandra.spark.data.SSTableTokenIndex;
+import org.apache.spark.broadcast.Broadcast;
import org.apache.spark.sql.connector.catalog.Table;
import org.apache.spark.sql.connector.catalog.TableProvider;
import org.apache.spark.sql.connector.expressions.Transform;
import org.apache.spark.sql.sources.DataSourceRegister;
import org.apache.spark.sql.types.StructType;
import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+import org.jetbrains.annotations.Nullable;
public abstract class CassandraTableProvider implements TableProvider, DataSourceRegister
{
private DataLayer dataLayer;
+ private String dataLayerOptionsKey;
public abstract DataLayer getDataLayer(CaseInsensitiveStringMap options);
- DataLayer getDataLayerInternal(CaseInsensitiveStringMap options)
+ @Nullable
+ public Broadcast getSSTableTokenIndexBroadcast(CaseInsensitiveStringMap options)
{
+ return null;
+ }
+
+ synchronized DataLayer getDataLayerInternal(CaseInsensitiveStringMap options)
+ {
+ String optionsKey = normalizedOptionsKey(options);
DataLayer dataLayer = this.dataLayer;
- if (dataLayer != null)
+ if (dataLayer != null && optionsKey.equals(dataLayerOptionsKey))
{
return dataLayer;
}
dataLayer = getDataLayer(options);
this.dataLayer = dataLayer;
+ this.dataLayerOptionsKey = optionsKey;
return dataLayer;
}
+ private static String normalizedOptionsKey(CaseInsensitiveStringMap options)
+ {
+ TreeMap normalized = new TreeMap<>();
+ for (Map.Entry entry : options.entrySet())
+ {
+ normalized.put(entry.getKey().toLowerCase(Locale.ROOT), entry.getValue());
+ }
+ return normalized.toString();
+ }
+
@Override
public StructType inferSchema(CaseInsensitiveStringMap options)
{
@@ -56,6 +80,7 @@ public StructType inferSchema(CaseInsensitiveStringMap options)
@Override
public Table getTable(StructType schema, Transform[] partitioning, Map properties)
{
- return new CassandraTable(getDataLayerInternal(new CaseInsensitiveStringMap(properties)), schema);
+ CaseInsensitiveStringMap options = new CaseInsensitiveStringMap(properties);
+ return new CassandraTable(getDataLayerInternal(options), schema, getSSTableTokenIndexBroadcast(options));
}
}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/S3CassandraDataSource.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/S3CassandraDataSource.java
new file mode 100644
index 000000000..f22a383b3
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/S3CassandraDataSource.java
@@ -0,0 +1,97 @@
+/*
+ * 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.sparksql;
+
+import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.cassandra.spark.data.S3CassandraDataLayer;
+import org.apache.cassandra.spark.data.S3DataSourceClientConfig;
+import org.apache.cassandra.spark.data.SSTableTokenIndex;
+
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.connector.catalog.SessionConfigSupport;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+import org.jetbrains.annotations.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+// S3CassandraDataSource that implements Spark 3 CassandraTableProvider
+public class S3CassandraDataSource extends CassandraTableProvider implements SessionConfigSupport
+{
+ private static final Logger LOGGER = LoggerFactory.getLogger(S3CassandraDataSource.class);
+
+ /**
+ * Driver-local handle produced by {@link S3CassandraTokenIndexPrebuilder}. This is intentionally
+ * not a durable DataSource option: saved/replayed plans and remote client/server boundaries must
+ * rebuild the context in the JVM that plans the read.
+ */
+ public static final String READ_CONTEXT_ID_KEY = "s3CassandraReadContextId";
+
+ @Override
+ public String shortName()
+ {
+ return "s3CassandraBulkRead";
+ }
+
+ /*
+ * Spark will propagate session configs spark.datasource.s3CassandraBulkRead.* to the getTable options.
+ * */
+ @Override
+ public String keyPrefix()
+ {
+ return shortName();
+ }
+
+ @Override
+ public DataLayer getDataLayer(CaseInsensitiveStringMap options)
+ {
+ S3CassandraPrebuiltReadContext context = getPrebuiltReadContext(options);
+ if (context != null)
+ {
+ return context.dataLayer();
+ }
+ S3DataSourceClientConfig config = S3DataSourceClientConfig.create(options);
+ return new S3CassandraDataLayer(config);
+ }
+
+ @Nullable
+ @Override
+ public Broadcast getSSTableTokenIndexBroadcast(CaseInsensitiveStringMap options)
+ {
+ S3CassandraPrebuiltReadContext context = getPrebuiltReadContext(options);
+ return context == null ? null : context.sstableTokenIndexBroadcast();
+ }
+
+ @Nullable
+ private S3CassandraPrebuiltReadContext getPrebuiltReadContext(CaseInsensitiveStringMap options)
+ {
+ String contextId = options.get(READ_CONTEXT_ID_KEY);
+ if (contextId == null || contextId.trim().isEmpty())
+ {
+ return null;
+ }
+ S3CassandraPrebuiltReadContext context = S3CassandraPrebuiltReadContextRegistry.get(contextId);
+ if (context == null)
+ {
+ throw new IllegalArgumentException("No S3 Cassandra prebuilt read context found for id=" + contextId);
+ }
+ LOGGER.info("Resolved S3 Cassandra prebuilt read context id={}", contextId);
+ return context;
+ }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/S3CassandraPrebuiltReadContext.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/S3CassandraPrebuiltReadContext.java
new file mode 100644
index 000000000..420313c00
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/S3CassandraPrebuiltReadContext.java
@@ -0,0 +1,86 @@
+/*
+ * 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.sparksql;
+
+import org.apache.cassandra.spark.data.S3CassandraDataLayer;
+import org.apache.cassandra.spark.data.SSTableTokenIndex;
+import org.apache.spark.broadcast.Broadcast;
+import org.jetbrains.annotations.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public final class S3CassandraPrebuiltReadContext implements AutoCloseable
+{
+ private static final Logger LOGGER = LoggerFactory.getLogger(S3CassandraPrebuiltReadContext.class);
+
+ private final String id;
+ private final S3CassandraDataLayer dataLayer;
+ @Nullable
+ private final Broadcast sstableTokenIndexBroadcast;
+ private boolean closed;
+
+ S3CassandraPrebuiltReadContext(String id,
+ S3CassandraDataLayer dataLayer,
+ @Nullable Broadcast sstableTokenIndexBroadcast)
+ {
+ this.id = id;
+ this.dataLayer = dataLayer;
+ this.sstableTokenIndexBroadcast = sstableTokenIndexBroadcast;
+ }
+
+ public String id()
+ {
+ return id;
+ }
+
+ S3CassandraDataLayer dataLayer()
+ {
+ return dataLayer;
+ }
+
+ @Nullable
+ Broadcast sstableTokenIndexBroadcast()
+ {
+ return sstableTokenIndexBroadcast;
+ }
+
+ @Override
+ public synchronized void close()
+ {
+ if (closed)
+ {
+ return;
+ }
+ closed = true;
+ try
+ {
+ S3CassandraPrebuiltReadContextRegistry.remove(id);
+ if (sstableTokenIndexBroadcast != null)
+ {
+ sstableTokenIndexBroadcast.destroy(false);
+ }
+ }
+ finally
+ {
+ dataLayer.close();
+ }
+ LOGGER.info("Closed S3 Cassandra prebuilt read context id={}", id);
+ }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/S3CassandraPrebuiltReadContextRegistry.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/S3CassandraPrebuiltReadContextRegistry.java
new file mode 100644
index 000000000..45a7afcb3
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/S3CassandraPrebuiltReadContextRegistry.java
@@ -0,0 +1,56 @@
+/*
+ * 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.sparksql;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.jetbrains.annotations.Nullable;
+
+public final class S3CassandraPrebuiltReadContextRegistry
+{
+ private static final Map CONTEXTS = new ConcurrentHashMap<>();
+
+ private S3CassandraPrebuiltReadContextRegistry()
+ {
+ }
+
+ static S3CassandraPrebuiltReadContext register(S3CassandraPrebuiltReadContext context)
+ {
+ CONTEXTS.put(context.id(), context);
+ return context;
+ }
+
+ @Nullable
+ public static S3CassandraPrebuiltReadContext get(String id)
+ {
+ return CONTEXTS.get(id);
+ }
+
+ static void remove(String id)
+ {
+ CONTEXTS.remove(id);
+ }
+
+ static void clear()
+ {
+ CONTEXTS.clear();
+ }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/S3CassandraTokenIndexPrebuilder.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/S3CassandraTokenIndexPrebuilder.java
new file mode 100644
index 000000000..981dbb748
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/S3CassandraTokenIndexPrebuilder.java
@@ -0,0 +1,114 @@
+/*
+ * 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.sparksql;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.spark.data.S3CassandraDataLayer;
+import org.apache.cassandra.spark.data.S3DataSourceClientConfig;
+import org.apache.cassandra.spark.data.SSTableSummaryWorkItem;
+import org.apache.cassandra.spark.data.SSTableTokenIndex;
+import org.apache.cassandra.spark.data.SSTableTokenIndexBuilder;
+import org.apache.cassandra.spark.data.TokenIndexShard;
+import org.apache.cassandra.spark.data.backup.BackupReaderConfig;
+import org.apache.cassandra.spark.data.backup.BackupReaderFactory;
+import org.apache.cassandra.spark.data.backup.BackupReaderRegistry;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+import org.jetbrains.annotations.Nullable;
+
+public final class S3CassandraTokenIndexPrebuilder
+{
+ private static final Logger LOGGER = LoggerFactory.getLogger(S3CassandraTokenIndexPrebuilder.class);
+
+ private S3CassandraTokenIndexPrebuilder()
+ {
+ }
+
+ public static S3CassandraPrebuiltReadContext prepare(SparkSession sparkSession, Map options)
+ {
+ return prepare(sparkSession, new CaseInsensitiveStringMap(options));
+ }
+
+ public static S3CassandraPrebuiltReadContext prepare(SparkSession sparkSession, CaseInsensitiveStringMap options)
+ {
+ S3DataSourceClientConfig config = S3DataSourceClientConfig.create(options);
+ S3CassandraDataLayer dataLayer = new S3CassandraDataLayer(config);
+ Broadcast broadcast = dataLayer.sstableTokenIndexEnabled()
+ ? buildBroadcast(sparkSession, dataLayer)
+ : null;
+ String id = UUID.randomUUID().toString();
+ S3CassandraPrebuiltReadContext context = new S3CassandraPrebuiltReadContext(id, dataLayer, broadcast);
+ S3CassandraPrebuiltReadContextRegistry.register(context);
+ LOGGER.info("Registered S3 Cassandra prebuilt read context id={} tokenIndexEnabled={} hasBroadcast={}",
+ id, dataLayer.sstableTokenIndexEnabled(), broadcast != null);
+ return context;
+ }
+
+ @Nullable
+ private static Broadcast buildBroadcast(SparkSession sparkSession, S3CassandraDataLayer dataLayer)
+ {
+ long startNanos = System.nanoTime();
+ JavaSparkContext sparkContext = JavaSparkContext.fromSparkContext(sparkSession.sparkContext());
+ List workItems = dataLayer.sstableTokenIndexWorkItems();
+ int partitionCount = dataLayer.sstableTokenIndexPrebuildPartitions(sparkContext.defaultParallelism());
+ int concurrency = dataLayer.sstableTokenIndexPrebuildPerTaskConcurrency();
+ // Capture the factory + a stats-less config into the Spark closure. Executors instantiate
+ // the reader via the factory; stats are reinstalled inside buildShard.
+ BackupReaderFactory backupReaderFactory = BackupReaderRegistry.factoryFor(dataLayer.backupReaderType());
+ BackupReaderConfig backupReaderConfig = BackupReaderConfig.of(dataLayer.s3ClientConfig());
+ String clusterName = dataLayer.clusterName();
+ String datacenter = dataLayer.datacenter();
+ CassandraVersion cassandraVersion = dataLayer.version();
+ LOGGER.info("Building SSTable token index sstableCount={} prebuildPartitions={} perTaskConcurrency={} backupReaderType={}",
+ workItems.size(), partitionCount, concurrency, dataLayer.backupReaderType());
+
+ List shards = sparkContext.parallelize(workItems, partitionCount)
+ .mapPartitions(items -> Collections.singletonList(
+ SSTableTokenIndexBuilder.buildShard(items,
+ backupReaderFactory,
+ backupReaderConfig,
+ clusterName,
+ datacenter,
+ cassandraVersion,
+ concurrency)).iterator())
+ .collect();
+ SSTableTokenIndex tokenIndex = SSTableTokenIndex.fromShards(shards);
+ Broadcast broadcast = sparkContext.broadcast(tokenIndex);
+
+ long elapsedNanos = System.nanoTime() - startNanos;
+ double elapsedSeconds = elapsedNanos / 1_000_000_000.0D;
+ double summariesPerSecond = elapsedSeconds == 0 ? workItems.size() : workItems.size() / elapsedSeconds;
+ LOGGER.info("Built SSTable token index sstableCount={} indexed={} missing={} errors={} "
+ + "elapsedSeconds={} summariesPerSecond={} estimatedBroadcastBytes={}",
+ workItems.size(), tokenIndex.successCount(), tokenIndex.missingCount(), tokenIndex.errorCount(),
+ elapsedSeconds, summariesPerSecond, tokenIndex.estimatedSizeInBytes());
+ return broadcast;
+ }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/SparkCustomMetricsStats.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/SparkCustomMetricsStats.java
new file mode 100644
index 000000000..f39f4180f
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/SparkCustomMetricsStats.java
@@ -0,0 +1,235 @@
+/*
+ * 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.sparksql;
+
+import java.math.BigInteger;
+import java.util.List;
+import java.util.concurrent.atomic.LongAdder;
+
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import org.apache.cassandra.analytics.stats.Stats;
+import org.apache.cassandra.spark.data.FileType;
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.cassandra.spark.sparksql.filters.SparkRangeFilter;
+
+/**
+ * Stats implementation that collects summary read duration timing for metrics reporting.
+ * This extends the base Stats class and accumulates summary read timing data that can be
+ * accessed by the Spark metrics system.
+ *
+ * Uses LongAdder for optimal performance under concurrent access. LongAdder internally
+ * implements thread-local accumulation cells that eliminate contention between executor
+ * threads while maintaining perfect accuracy.
+ */
+public class SparkCustomMetricsStats extends Stats
+{
+ private final LongAdder totalSummaryReadDuration = new LongAdder();
+ private final LongAdder totalOpenedSSTableDuration = new LongAdder();
+ private final LongAdder totalCorruptSSTableCount = new LongAdder();
+ private final LongAdder totalSkippedSSTableCount = new LongAdder();
+ private final LongAdder totalS3HeadObjectDuration = new LongAdder();
+ private final LongAdder totalS3GetObjectDuration = new LongAdder();
+ private final LongAdder totalMutableMetadataDriftCount = new LongAdder();
+ private final LongAdder totalMutableMetadataHeadFallbackCount = new LongAdder();
+
+ /**
+ * Get the total accumulated summary read duration in nanoseconds.
+ * Aggregates all thread-local cells maintained by LongAdder.
+ *
+ * @return total summary read duration in nanoseconds
+ */
+ public long getTotalSummaryReadDurationNanos()
+ {
+ return totalSummaryReadDuration.sum();
+ }
+
+ /**
+ * Get the total accumulated SSTable open duration in nanoseconds.
+ * Aggregates all thread-local cells maintained by LongAdder.
+ *
+ * @return total SSTable open duration in nanoseconds
+ */
+ public long getTotalOpenedSSTableDurationNanos()
+ {
+ return totalOpenedSSTableDuration.sum();
+ }
+
+ /**
+ * Get the total count of corrupt SSTables encountered.
+ * Aggregates all thread-local cells maintained by LongAdder.
+ *
+ * @return total count of corrupt SSTables
+ */
+ public long getTotalCorruptSSTableCount()
+ {
+ return totalCorruptSSTableCount.sum();
+ }
+
+ /**
+ * Get the total count of skipped SSTables.
+ * Aggregates all thread-local cells maintained by LongAdder.
+ *
+ * @return total count of skipped SSTables
+ */
+ public long getTotalSkippedSSTableCount()
+ {
+ return totalSkippedSSTableCount.sum();
+ }
+
+ /**
+ * Get the total accumulated S3 headObject duration in nanoseconds.
+ * Aggregates all thread-local cells maintained by LongAdder.
+ *
+ * @return total S3 headObject duration in nanoseconds
+ */
+ public long getTotalS3HeadObjectDurationNanos()
+ {
+ return totalS3HeadObjectDuration.sum();
+ }
+
+ /**
+ * Get the total accumulated S3 getObject duration in nanoseconds.
+ * Aggregates all thread-local cells maintained by LongAdder.
+ *
+ * @return total S3 getObject duration in nanoseconds
+ */
+ public long getTotalS3GetObjectDurationNanos()
+ {
+ return totalS3GetObjectDuration.sum();
+ }
+
+ /**
+ * Get the total count of mutable metadata size drift observations.
+ *
+ * @return total count of mutable metadata drift observations
+ */
+ public long getTotalMutableMetadataDriftCount()
+ {
+ return totalMutableMetadataDriftCount.sum();
+ }
+
+ /**
+ * Get the total count of mutable metadata HEAD fallback reads.
+ *
+ * @return total count of mutable metadata HEAD fallback reads
+ */
+ public long getTotalMutableMetadataHeadFallbackCount()
+ {
+ return totalMutableMetadataHeadFallbackCount.sum();
+ }
+
+ /**
+ * Accumulate summary read duration timing.
+ *
+ * Uses LongAdder.add() which automatically distributes writes across thread-local
+ * cells under contention, providing excellent performance when multiple executor
+ * threads are concurrently opening SSTable readers.
+ */
+ @Override
+ public void readSummaryDb(SSTable ssTable, long timeNanos)
+ {
+ totalSummaryReadDuration.add(timeNanos);
+ }
+
+ /**
+ * Accumulate SSTable open duration timing.
+ *
+ * Uses LongAdder.add() which automatically distributes writes across thread-local
+ * cells under contention, providing excellent performance when multiple executor
+ * threads are concurrently opening SSTables.
+ */
+ @Override
+ public void openedSSTable(SSTable ssTable, long timeNanos)
+ {
+ totalOpenedSSTableDuration.add(timeNanos);
+ }
+
+ /**
+ * Accumulate corrupt SSTable count.
+ *
+ * Uses LongAdder.add() which automatically distributes writes across thread-local
+ * cells under contention, providing excellent performance when multiple executor
+ * threads are concurrently encountering corrupt SSTables.
+ */
+ @Override
+ public void corruptSSTable(Throwable throwable, String keyspace, String table, SSTable ssTable)
+ {
+ totalCorruptSSTableCount.add(1);
+ }
+
+ /**
+ * Accumulate skipped SSTable count.
+ *
+ * Uses LongAdder.add() which automatically distributes writes across thread-local
+ * cells under contention, providing excellent performance when multiple executor
+ * threads are concurrently skipping SSTables.
+ */
+ @Override
+ public void skippedSSTable(@Nullable SparkRangeFilter sparkRangeFilter,
+ @NotNull List partitionKeyFilters,
+ @NotNull BigInteger firstToken,
+ @NotNull BigInteger lastToken)
+ {
+ totalSkippedSSTableCount.add(1);
+ }
+
+ /**
+ * Accumulate S3 headObject call duration.
+ *
+ * Uses LongAdder.add() which automatically distributes writes across thread-local
+ * cells under contention, providing excellent performance when multiple executor
+ * threads are concurrently performing S3 headObject operations.
+ */
+ @Override
+ public void s3HeadObjectOperation(long timeNanos)
+ {
+ totalS3HeadObjectDuration.add(timeNanos);
+ }
+
+ /**
+ * Accumulate S3 getObject call duration.
+ *
+ * Uses LongAdder.add() which automatically distributes writes across thread-local
+ * cells under contention, providing excellent performance when multiple executor
+ * threads are concurrently performing S3 getObject operations.
+ */
+ @Override
+ public void s3GetObjectOperation(long timeNanos)
+ {
+ totalS3GetObjectDuration.add(timeNanos);
+ }
+
+ @Override
+ public void s3MutableMetadataDriftDetected(FileType fileType, long manifestSize, long actualSize)
+ {
+ totalMutableMetadataDriftCount.add(1);
+ }
+
+ @Override
+ public void s3MutableMetadataHeadFallback(FileType fileType)
+ {
+ totalMutableMetadataHeadFallbackCount.add(1);
+ }
+
+ // All other Stats methods are no-ops, pending further implementations.
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/SparkRowIterator.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/SparkRowIterator.java
index d076b3423..645c1a456 100644
--- a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/SparkRowIterator.java
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/SparkRowIterator.java
@@ -30,8 +30,18 @@
import org.apache.cassandra.spark.config.SchemaFeature;
import org.apache.cassandra.spark.data.DataLayer;
import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.cassandra.spark.sparksql.metrics.TaskTotalSummaryReadDuration;
+import org.apache.cassandra.spark.sparksql.metrics.TaskTotalOpenedSSTableDuration;
+import org.apache.cassandra.spark.sparksql.metrics.TaskTotalCorruptSSTableCount;
+import org.apache.cassandra.spark.sparksql.metrics.TaskTotalMutableMetadataDriftCount;
+import org.apache.cassandra.spark.sparksql.metrics.TaskTotalMutableMetadataHeadFallbackCount;
+import org.apache.cassandra.spark.sparksql.metrics.TaskTotalSkippedSSTableCount;
+import org.apache.cassandra.spark.sparksql.metrics.TaskTotalS3HeadObjectDuration;
+import org.apache.cassandra.spark.sparksql.metrics.TaskTotalS3GetObjectDuration;
+import org.apache.cassandra.analytics.stats.Stats;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.connector.metric.CustomTaskMetric;
import org.apache.spark.sql.connector.read.PartitionReader;
import org.apache.spark.sql.types.StructType;
import org.jetbrains.annotations.NotNull;
@@ -84,4 +94,31 @@ public GenericInternalRow rowBuilder(Object[] valueArray)
{
return new GenericInternalRow(valueArray);
}
+
+ /**
+ * Returns custom metrics collected during task execution.
+ * This is called by Spark to collect task-level metrics for aggregation.
+ */
+ @Override
+ public CustomTaskMetric[] currentMetricsValues()
+ {
+ Stats stats = this.it.stats();
+ if (stats instanceof SparkCustomMetricsStats)
+ {
+ SparkCustomMetricsStats metricsStats = (SparkCustomMetricsStats) stats;
+ return new CustomTaskMetric[] {
+ TaskTotalSummaryReadDuration.from(metricsStats.getTotalSummaryReadDurationNanos()),
+ TaskTotalOpenedSSTableDuration.from(metricsStats.getTotalOpenedSSTableDurationNanos()),
+ TaskTotalCorruptSSTableCount.from(metricsStats.getTotalCorruptSSTableCount()),
+ TaskTotalSkippedSSTableCount.from(metricsStats.getTotalSkippedSSTableCount()),
+ TaskTotalS3HeadObjectDuration.from(metricsStats.getTotalS3HeadObjectDurationNanos()),
+ TaskTotalS3GetObjectDuration.from(metricsStats.getTotalS3GetObjectDurationNanos()),
+ TaskTotalMutableMetadataDriftCount.from(metricsStats.getTotalMutableMetadataDriftCount()),
+ TaskTotalMutableMetadataHeadFallbackCount.from(metricsStats.getTotalMutableMetadataHeadFallbackCount())
+ };
+ }
+
+ // Return empty array if no metrics available
+ return new CustomTaskMetric[0];
+ }
}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalCorruptSSTableCount.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalCorruptSSTableCount.java
new file mode 100644
index 000000000..7be13eb50
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalCorruptSSTableCount.java
@@ -0,0 +1,70 @@
+/*
+ * 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.sparksql.metrics;
+
+import java.util.concurrent.atomic.LongAdder;
+
+import org.apache.spark.sql.connector.metric.CustomTaskMetric;
+
+public class TaskTotalCorruptSSTableCount implements CustomTaskMetric
+{
+ private final long value;
+
+ private TaskTotalCorruptSSTableCount(long value)
+ {
+ this.value = value;
+ }
+
+ @Override
+ public String name()
+ {
+ return TotalCorruptSSTableCount.NAME;
+ }
+
+ @Override
+ public long value()
+ {
+ return value;
+ }
+
+ /**
+ * Create a TaskTotalCorruptSSTableCount metric from accumulated count data.
+ * This should be called at the end of a task to capture the total corrupt SSTable count.
+ *
+ * @param totalCorruptSSTableCount total count of corrupt SSTable files encountered
+ * @return TaskTotalCorruptSSTableCount instance with the count
+ */
+ public static TaskTotalCorruptSSTableCount from(long totalCorruptSSTableCount)
+ {
+ return new TaskTotalCorruptSSTableCount(totalCorruptSSTableCount);
+ }
+
+ /**
+ * Create a TaskTotalCorruptSSTableCount metric from a LongAdder accumulator.
+ * Convenience method for working with thread-local accumulation counters.
+ *
+ * @param totalCorruptSSTableCount LongAdder containing accumulated corrupt SSTable count
+ * @return TaskTotalCorruptSSTableCount instance with the count
+ */
+ public static TaskTotalCorruptSSTableCount from(LongAdder totalCorruptSSTableCount)
+ {
+ return from(totalCorruptSSTableCount.sum());
+ }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalMutableMetadataDriftCount.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalMutableMetadataDriftCount.java
new file mode 100644
index 000000000..e421f382d
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalMutableMetadataDriftCount.java
@@ -0,0 +1,49 @@
+/*
+ * 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.sparksql.metrics;
+
+import org.apache.spark.sql.connector.metric.CustomTaskMetric;
+
+public class TaskTotalMutableMetadataDriftCount implements CustomTaskMetric
+{
+ private final long value;
+
+ private TaskTotalMutableMetadataDriftCount(long value)
+ {
+ this.value = value;
+ }
+
+ @Override
+ public String name()
+ {
+ return TotalMutableMetadataDriftCount.NAME;
+ }
+
+ @Override
+ public long value()
+ {
+ return value;
+ }
+
+ public static TaskTotalMutableMetadataDriftCount from(long totalMutableMetadataDriftCount)
+ {
+ return new TaskTotalMutableMetadataDriftCount(totalMutableMetadataDriftCount);
+ }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalMutableMetadataHeadFallbackCount.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalMutableMetadataHeadFallbackCount.java
new file mode 100644
index 000000000..d813bcb3c
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalMutableMetadataHeadFallbackCount.java
@@ -0,0 +1,49 @@
+/*
+ * 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.sparksql.metrics;
+
+import org.apache.spark.sql.connector.metric.CustomTaskMetric;
+
+public class TaskTotalMutableMetadataHeadFallbackCount implements CustomTaskMetric
+{
+ private final long value;
+
+ private TaskTotalMutableMetadataHeadFallbackCount(long value)
+ {
+ this.value = value;
+ }
+
+ @Override
+ public String name()
+ {
+ return TotalMutableMetadataHeadFallbackCount.NAME;
+ }
+
+ @Override
+ public long value()
+ {
+ return value;
+ }
+
+ public static TaskTotalMutableMetadataHeadFallbackCount from(long totalMutableMetadataHeadFallbackCount)
+ {
+ return new TaskTotalMutableMetadataHeadFallbackCount(totalMutableMetadataHeadFallbackCount);
+ }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalOpenedSSTableDuration.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalOpenedSSTableDuration.java
new file mode 100644
index 000000000..af89f1146
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalOpenedSSTableDuration.java
@@ -0,0 +1,72 @@
+/*
+ * 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.sparksql.metrics;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.LongAdder;
+
+import org.apache.spark.sql.connector.metric.CustomTaskMetric;
+
+public class TaskTotalOpenedSSTableDuration implements CustomTaskMetric
+{
+ private final long value;
+
+ private TaskTotalOpenedSSTableDuration(long value)
+ {
+ this.value = value;
+ }
+
+ @Override
+ public String name()
+ {
+ return TotalOpenedSSTableDuration.NAME;
+ }
+
+ @Override
+ public long value()
+ {
+ return value;
+ }
+
+ /**
+ * Create a TaskTotalOpenedSSTableDuration metric from accumulated timing data.
+ * This should be called at the end of a task to capture the total SSTable open time.
+ *
+ * @param totalOpenedSSTableNanos total time spent opening SSTable files in nanoseconds
+ * @return TaskTotalOpenedSSTableDuration instance with the time converted to milliseconds
+ */
+ public static TaskTotalOpenedSSTableDuration from(long totalOpenedSSTableNanos)
+ {
+ long millis = totalOpenedSSTableNanos > 0 ? TimeUnit.NANOSECONDS.toMillis(totalOpenedSSTableNanos) : 0;
+ return new TaskTotalOpenedSSTableDuration(millis);
+ }
+
+ /**
+ * Create a TaskTotalOpenedSSTableDuration metric from a LongAdder accumulator.
+ * Convenience method for working with thread-local accumulation counters.
+ *
+ * @param totalOpenedSSTableNanos LongAdder containing accumulated SSTable open time in nanoseconds
+ * @return TaskTotalOpenedSSTableDuration instance with the time converted to milliseconds
+ */
+ public static TaskTotalOpenedSSTableDuration from(LongAdder totalOpenedSSTableNanos)
+ {
+ return from(totalOpenedSSTableNanos.sum());
+ }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalS3GetObjectDuration.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalS3GetObjectDuration.java
new file mode 100644
index 000000000..3229a91ad
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalS3GetObjectDuration.java
@@ -0,0 +1,72 @@
+/*
+ * 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.sparksql.metrics;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.LongAdder;
+
+import org.apache.spark.sql.connector.metric.CustomTaskMetric;
+
+public class TaskTotalS3GetObjectDuration implements CustomTaskMetric
+{
+ private final long value;
+
+ private TaskTotalS3GetObjectDuration(long value)
+ {
+ this.value = value;
+ }
+
+ @Override
+ public String name()
+ {
+ return TotalS3GetObjectDuration.NAME;
+ }
+
+ @Override
+ public long value()
+ {
+ return value;
+ }
+
+ /**
+ * Create a TaskTotalS3GetObjectDuration metric from accumulated timing data.
+ * This should be called at the end of a task to capture the total S3 getObject time.
+ *
+ * @param totalS3GetObjectNanos total time spent on S3 getObject operations in nanoseconds
+ * @return TaskTotalS3GetObjectDuration instance with the time converted to milliseconds
+ */
+ public static TaskTotalS3GetObjectDuration from(long totalS3GetObjectNanos)
+ {
+ long millis = totalS3GetObjectNanos > 0 ? TimeUnit.NANOSECONDS.toMillis(totalS3GetObjectNanos) : 0;
+ return new TaskTotalS3GetObjectDuration(millis);
+ }
+
+ /**
+ * Create a TaskTotalS3GetObjectDuration metric from a LongAdder accumulator.
+ * Convenience method for working with thread-local accumulation counters.
+ *
+ * @param totalS3GetObjectNanos LongAdder containing accumulated S3 getObject time in nanoseconds
+ * @return TaskTotalS3GetObjectDuration instance with the time converted to milliseconds
+ */
+ public static TaskTotalS3GetObjectDuration from(LongAdder totalS3GetObjectNanos)
+ {
+ return from(totalS3GetObjectNanos.sum());
+ }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalS3HeadObjectDuration.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalS3HeadObjectDuration.java
new file mode 100644
index 000000000..7e579787a
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalS3HeadObjectDuration.java
@@ -0,0 +1,72 @@
+/*
+ * 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.sparksql.metrics;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.LongAdder;
+
+import org.apache.spark.sql.connector.metric.CustomTaskMetric;
+
+public class TaskTotalS3HeadObjectDuration implements CustomTaskMetric
+{
+ private final long value;
+
+ private TaskTotalS3HeadObjectDuration(long value)
+ {
+ this.value = value;
+ }
+
+ @Override
+ public String name()
+ {
+ return TotalS3HeadObjectDuration.NAME;
+ }
+
+ @Override
+ public long value()
+ {
+ return value;
+ }
+
+ /**
+ * Create a TaskTotalS3HeadObjectDuration metric from accumulated timing data.
+ * This should be called at the end of a task to capture the total S3 headObject time.
+ *
+ * @param totalS3HeadObjectNanos total time spent on S3 headObject operations in nanoseconds
+ * @return TaskTotalS3HeadObjectDuration instance with the time converted to milliseconds
+ */
+ public static TaskTotalS3HeadObjectDuration from(long totalS3HeadObjectNanos)
+ {
+ long millis = totalS3HeadObjectNanos > 0 ? TimeUnit.NANOSECONDS.toMillis(totalS3HeadObjectNanos) : 0;
+ return new TaskTotalS3HeadObjectDuration(millis);
+ }
+
+ /**
+ * Create a TaskTotalS3HeadObjectDuration metric from a LongAdder accumulator.
+ * Convenience method for working with thread-local accumulation counters.
+ *
+ * @param totalS3HeadObjectNanos LongAdder containing accumulated S3 headObject time in nanoseconds
+ * @return TaskTotalS3HeadObjectDuration instance with the time converted to milliseconds
+ */
+ public static TaskTotalS3HeadObjectDuration from(LongAdder totalS3HeadObjectNanos)
+ {
+ return from(totalS3HeadObjectNanos.sum());
+ }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalSkippedSSTableCount.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalSkippedSSTableCount.java
new file mode 100644
index 000000000..f37e3f131
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalSkippedSSTableCount.java
@@ -0,0 +1,70 @@
+/*
+ * 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.sparksql.metrics;
+
+import java.util.concurrent.atomic.LongAdder;
+
+import org.apache.spark.sql.connector.metric.CustomTaskMetric;
+
+public class TaskTotalSkippedSSTableCount implements CustomTaskMetric
+{
+ private final long value;
+
+ private TaskTotalSkippedSSTableCount(long value)
+ {
+ this.value = value;
+ }
+
+ @Override
+ public String name()
+ {
+ return TotalSkippedSSTableCount.NAME;
+ }
+
+ @Override
+ public long value()
+ {
+ return value;
+ }
+
+ /**
+ * Create a TaskTotalSkippedSSTableCount metric from accumulated count data.
+ * This should be called at the end of a task to capture the total skipped SSTable count.
+ *
+ * @param totalSkippedSSTableCount total count of SSTable files skipped
+ * @return TaskTotalSkippedSSTableCount instance with the count
+ */
+ public static TaskTotalSkippedSSTableCount from(long totalSkippedSSTableCount)
+ {
+ return new TaskTotalSkippedSSTableCount(totalSkippedSSTableCount);
+ }
+
+ /**
+ * Create a TaskTotalSkippedSSTableCount metric from a LongAdder accumulator.
+ * Convenience method for working with thread-local accumulation counters.
+ *
+ * @param totalSkippedSSTableCount LongAdder containing accumulated skipped SSTable count
+ * @return TaskTotalSkippedSSTableCount instance with the count
+ */
+ public static TaskTotalSkippedSSTableCount from(LongAdder totalSkippedSSTableCount)
+ {
+ return from(totalSkippedSSTableCount.sum());
+ }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalSummaryReadDuration.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalSummaryReadDuration.java
new file mode 100644
index 000000000..e034c26d1
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TaskTotalSummaryReadDuration.java
@@ -0,0 +1,72 @@
+/*
+ * 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.sparksql.metrics;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.LongAdder;
+
+import org.apache.spark.sql.connector.metric.CustomTaskMetric;
+
+public class TaskTotalSummaryReadDuration implements CustomTaskMetric
+{
+ private final long value;
+
+ private TaskTotalSummaryReadDuration(long value)
+ {
+ this.value = value;
+ }
+
+ @Override
+ public String name()
+ {
+ return TotalSummaryReadDuration.NAME;
+ }
+
+ @Override
+ public long value()
+ {
+ return value;
+ }
+
+ /**
+ * Create a TaskTotalSummaryReadDuration metric from accumulated timing data.
+ * This should be called at the end of a task to capture the total summary read time.
+ *
+ * @param totalSummaryReadNanos total time spent reading summary files in nanoseconds
+ * @return TaskTotalSummaryReadDuration instance with the time converted to milliseconds
+ */
+ public static TaskTotalSummaryReadDuration from(long totalSummaryReadNanos)
+ {
+ long millis = totalSummaryReadNanos > 0 ? TimeUnit.NANOSECONDS.toMillis(totalSummaryReadNanos) : 0;
+ return new TaskTotalSummaryReadDuration(millis);
+ }
+
+ /**
+ * Create a TaskTotalSummaryReadDuration metric from a LongAdder accumulator.
+ * Convenience method for working with thread-local accumulation counters.
+ *
+ * @param totalSummaryReadNanos LongAdder containing accumulated summary read time in nanoseconds
+ * @return TaskTotalSummaryReadDuration instance with the time converted to milliseconds
+ */
+ public static TaskTotalSummaryReadDuration from(LongAdder totalSummaryReadNanos)
+ {
+ return from(totalSummaryReadNanos.sum());
+ }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalCorruptSSTableCount.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalCorruptSSTableCount.java
new file mode 100644
index 000000000..1df284d88
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalCorruptSSTableCount.java
@@ -0,0 +1,40 @@
+/*
+ * 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.sparksql.metrics;
+
+import org.apache.spark.sql.connector.metric.CustomSumMetric;
+
+public class TotalCorruptSSTableCount extends CustomSumMetric
+{
+
+ static final String NAME = "TotalCorruptSSTableCount";
+
+ @Override
+ public String name()
+ {
+ return NAME;
+ }
+
+ @Override
+ public String description()
+ {
+ return "total count of corrupt SSTable files encountered";
+ }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalMutableMetadataDriftCount.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalMutableMetadataDriftCount.java
new file mode 100644
index 000000000..a5b2cb968
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalMutableMetadataDriftCount.java
@@ -0,0 +1,39 @@
+/*
+ * 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.sparksql.metrics;
+
+import org.apache.spark.sql.connector.metric.CustomSumMetric;
+
+public class TotalMutableMetadataDriftCount extends CustomSumMetric
+{
+ static final String NAME = "TotalMutableMetadataDriftCount";
+
+ @Override
+ public String name()
+ {
+ return NAME;
+ }
+
+ @Override
+ public String description()
+ {
+ return "number of mutable SSTable metadata component resolutions where actual S3 size differed from manifest";
+ }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalMutableMetadataHeadFallbackCount.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalMutableMetadataHeadFallbackCount.java
new file mode 100644
index 000000000..290544bcd
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalMutableMetadataHeadFallbackCount.java
@@ -0,0 +1,39 @@
+/*
+ * 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.sparksql.metrics;
+
+import org.apache.spark.sql.connector.metric.CustomSumMetric;
+
+public class TotalMutableMetadataHeadFallbackCount extends CustomSumMetric
+{
+ static final String NAME = "TotalMutableMetadataHeadFallbackCount";
+
+ @Override
+ public String name()
+ {
+ return NAME;
+ }
+
+ @Override
+ public String description()
+ {
+ return "number of mutable SSTable metadata component reads that attempted HEAD fallback";
+ }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalOpenedSSTableDuration.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalOpenedSSTableDuration.java
new file mode 100644
index 000000000..d7b6fd2a5
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalOpenedSSTableDuration.java
@@ -0,0 +1,40 @@
+/*
+ * 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.sparksql.metrics;
+
+import org.apache.spark.sql.connector.metric.CustomSumMetric;
+
+public class TotalOpenedSSTableDuration extends CustomSumMetric
+{
+
+ static final String NAME = "TotalOpenedSSTableDuration";
+
+ @Override
+ public String name()
+ {
+ return NAME;
+ }
+
+ @Override
+ public String description()
+ {
+ return "total time spent opening SSTable files (ms)";
+ }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalS3GetObjectDuration.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalS3GetObjectDuration.java
new file mode 100644
index 000000000..d972743b3
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalS3GetObjectDuration.java
@@ -0,0 +1,40 @@
+/*
+ * 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.sparksql.metrics;
+
+import org.apache.spark.sql.connector.metric.CustomSumMetric;
+
+public class TotalS3GetObjectDuration extends CustomSumMetric
+{
+
+ static final String NAME = "TotalS3GetObjectDuration";
+
+ @Override
+ public String name()
+ {
+ return NAME;
+ }
+
+ @Override
+ public String description()
+ {
+ return "total time spent on S3 getObject data retrieval calls (ms)";
+ }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalS3HeadObjectDuration.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalS3HeadObjectDuration.java
new file mode 100644
index 000000000..54f07ccf3
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalS3HeadObjectDuration.java
@@ -0,0 +1,40 @@
+/*
+ * 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.sparksql.metrics;
+
+import org.apache.spark.sql.connector.metric.CustomSumMetric;
+
+public class TotalS3HeadObjectDuration extends CustomSumMetric
+{
+
+ static final String NAME = "TotalS3HeadObjectDuration";
+
+ @Override
+ public String name()
+ {
+ return NAME;
+ }
+
+ @Override
+ public String description()
+ {
+ return "total time spent on S3 headObject existence checks (ms)";
+ }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalSkippedSSTableCount.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalSkippedSSTableCount.java
new file mode 100644
index 000000000..de4291558
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalSkippedSSTableCount.java
@@ -0,0 +1,40 @@
+/*
+ * 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.sparksql.metrics;
+
+import org.apache.spark.sql.connector.metric.CustomSumMetric;
+
+public class TotalSkippedSSTableCount extends CustomSumMetric
+{
+
+ static final String NAME = "TotalSkippedSSTableCount";
+
+ @Override
+ public String name()
+ {
+ return NAME;
+ }
+
+ @Override
+ public String description()
+ {
+ return "total count of SSTable files skipped";
+ }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalSummaryReadDuration.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalSummaryReadDuration.java
new file mode 100644
index 000000000..eda8b38ec
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/metrics/TotalSummaryReadDuration.java
@@ -0,0 +1,40 @@
+/*
+ * 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.sparksql.metrics;
+
+import org.apache.spark.sql.connector.metric.CustomSumMetric;
+
+public class TotalSummaryReadDuration extends CustomSumMetric
+{
+
+ static final String NAME = "TotalSummaryReadDuration";
+
+ @Override
+ public String name()
+ {
+ return NAME;
+ }
+
+ @Override
+ public String description()
+ {
+ return "total time spend reading Summary sstable file (ms)";
+ }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/KryoSerializationTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/KryoSerializationTests.java
index 62e76774b..4213f9a6d 100644
--- a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/KryoSerializationTests.java
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/KryoSerializationTests.java
@@ -46,6 +46,7 @@
import org.apache.cassandra.spark.data.ReplicationFactor;
import org.apache.cassandra.spark.data.partitioner.CassandraInstance;
import org.apache.cassandra.spark.data.partitioner.CassandraRing;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
import org.apache.cassandra.spark.data.partitioner.TokenPartitioner;
import org.apache.cassandra.spark.transports.storage.StorageAccessConfiguration;
import org.apache.cassandra.spark.transports.storage.StorageCredentialPair;
@@ -295,6 +296,36 @@ public void testCassandraRing(CassandraBridge bridge)
});
}
+ @ParameterizedTest
+ @MethodSource("org.apache.cassandra.bridge.VersionRunner#bridges")
+ public void testCassandraRingWithAuthoritativeReplicas(CassandraBridge bridge)
+ {
+ java.util.List instances = java.util.Arrays.asList(
+ new CassandraInstance("0", "dc1-a", "DC1"),
+ new CassandraInstance("100", "dc1-b", "DC1"),
+ new CassandraInstance("200", "dc1-c", "DC1"),
+ new CassandraInstance("1", "dc2-a", "DC2"),
+ new CassandraInstance("101", "dc2-b", "DC2"),
+ new CassandraInstance("201", "dc2-c", "DC2"));
+ ReplicationFactor rf = new ReplicationFactor(ImmutableMap.of(
+ "class", "org.apache.cassandra.locator.NetworkTopologyStrategy",
+ "DC1", "3", "DC2", "3"));
+ java.math.BigInteger minToken = Partitioner.Murmur3Partitioner.minToken();
+ java.math.BigInteger maxToken = Partitioner.Murmur3Partitioner.maxToken();
+ com.google.common.collect.RangeMap> auth =
+ com.google.common.collect.TreeRangeMap.create();
+ auth.put(com.google.common.collect.Range.openClosed(minToken, java.math.BigInteger.ZERO),
+ java.util.Arrays.asList(instances.get(0), instances.get(3)));
+ auth.put(com.google.common.collect.Range.openClosed(java.math.BigInteger.ZERO, maxToken),
+ java.util.Arrays.asList(instances.get(1), instances.get(2),
+ instances.get(4), instances.get(5)));
+ CassandraRing ring = new CassandraRing(Partitioner.Murmur3Partitioner, "ks", rf, instances, auth);
+ Output out = serialize(bridge.getVersion(), ring);
+ CassandraRing deserialized = deserialize(bridge.getVersion(), out, CassandraRing.class);
+ assertThat(deserialized).isEqualTo(ring);
+ assertThat(deserialized.rangeMap().asMapOfRanges()).isEqualTo(auth.asMapOfRanges());
+ }
+
@ParameterizedTest
@MethodSource("org.apache.cassandra.bridge.VersionRunner#bridges")
public void testLocalDataLayer(CassandraBridge bridge)
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/S3ClientCacheTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/S3ClientCacheTest.java
new file mode 100644
index 000000000..c4067f0ba
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/S3ClientCacheTest.java
@@ -0,0 +1,411 @@
+/*
+ * 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 java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.ImmutableMap;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+import software.amazon.awssdk.services.s3.S3AsyncClient;
+import software.amazon.awssdk.services.s3.S3Client;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Unit tests for {@link S3ClientCache}.
+ */
+class S3ClientCacheTest
+{
+ private static final Map BASE_OPTIONS = ImmutableMap.builder()
+ .put("s3-region", "us-west-2")
+ .put("s3-bucket", "test-bucket")
+ .build();
+
+ @BeforeEach
+ void setUp()
+ {
+ S3ClientCache.reset();
+ }
+
+ @AfterEach
+ void tearDown()
+ {
+ S3ClientCache.reset();
+ }
+
+ @Test
+ void testCacheKeyGeneration()
+ {
+ S3ClientConfig config = createConfig(BASE_OPTIONS);
+ String key = S3ClientCache.getCacheKey(config);
+
+ // Key format: region|endpoint|accessKeyId|secretHash
+ assertThat(key).startsWith("us-west-2|");
+ assertThat(key).contains("|default|");
+ }
+
+ @Test
+ void testCacheKeyWithCredentials()
+ {
+ Map options = new HashMap<>(BASE_OPTIONS);
+ options.put("s3-access-key-id", "AKIATEST123");
+ options.put("s3-secret-access-key", "secretKey123");
+
+ S3ClientConfig config = createConfig(options);
+ String key = S3ClientCache.getCacheKey(config);
+
+ assertThat(key).startsWith("us-west-2|");
+ assertThat(key).contains("|AKIATEST123|");
+ // Secret hash should be consistent
+ int expectedHash = "secretKey123".hashCode();
+ assertThat(key).endsWith("|" + expectedHash);
+ }
+
+ @Test
+ void testCacheKeyWithEndpoint()
+ {
+ Map options = new HashMap<>(BASE_OPTIONS);
+ options.put("s3-endpoint-override", "http://localhost:9000");
+
+ S3ClientConfig config = createConfig(options);
+ String key = S3ClientCache.getCacheKey(config);
+
+ assertThat(key).contains("http://localhost:9000");
+ }
+
+ @Test
+ void testCacheKeyDifferentRegions()
+ {
+ S3ClientConfig config1 = createConfig(BASE_OPTIONS);
+
+ Map options2 = new HashMap<>(BASE_OPTIONS);
+ options2.put("s3-region", "eu-west-1");
+ S3ClientConfig config2 = createConfig(options2);
+
+ String key1 = S3ClientCache.getCacheKey(config1);
+ String key2 = S3ClientCache.getCacheKey(config2);
+
+ assertThat(key1).isNotEqualTo(key2);
+ assertThat(key1).startsWith("us-west-2|");
+ assertThat(key2).startsWith("eu-west-1|");
+ }
+
+ @Test
+ void testSameConfigReturnsSameClient()
+ {
+ S3ClientConfig config = createConfig(BASE_OPTIONS);
+
+ // Request the same client type twice
+ S3Client client1 = S3ClientCache.getS3Client(config);
+ S3Client client2 = S3ClientCache.getS3Client(config);
+
+ // Should return the exact same instance
+ assertThat(client1).isSameAs(client2);
+ assertThat(S3ClientCache.cacheSize()).isEqualTo(1);
+ }
+
+ @Test
+ void testSameConfigReturnsSameAsyncClient()
+ {
+ S3ClientConfig config = createConfig(BASE_OPTIONS);
+
+ S3AsyncClient client1 = S3ClientCache.getS3AsyncClient(config);
+ S3AsyncClient client2 = S3ClientCache.getS3AsyncClient(config);
+
+ assertThat(client1).isSameAs(client2);
+ }
+
+ @Test
+ void testAsyncCacheKeyIncludesHttpSignature()
+ {
+ Map lowConcurrencyOptions = new HashMap<>(BASE_OPTIONS);
+ lowConcurrencyOptions.put("s3-http-max-concurrency", "128");
+ S3ClientConfig lowConcurrency = createConfig(lowConcurrencyOptions);
+
+ Map highConcurrencyOptions = new HashMap<>(BASE_OPTIONS);
+ highConcurrencyOptions.put("s3-http-max-concurrency", "256");
+ S3ClientConfig highConcurrency = createConfig(highConcurrencyOptions);
+
+ assertThat(S3ClientCache.getCacheKey(lowConcurrency)).isEqualTo(S3ClientCache.getCacheKey(highConcurrency));
+ assertThat(S3ClientCache.getAsyncCacheKey(lowConcurrency)).isNotEqualTo(S3ClientCache.getAsyncCacheKey(highConcurrency));
+ assertThat(S3ClientCache.getAsyncCacheKey(highConcurrency)).contains("maxConcurrency=256");
+ }
+
+ @Test
+ void testDifferentAsyncHttpConfigReturnsDifferentAsyncClient()
+ {
+ Map lowConcurrencyOptions = new HashMap<>(BASE_OPTIONS);
+ lowConcurrencyOptions.put("s3-http-max-concurrency", "128");
+ S3ClientConfig lowConcurrency = createConfig(lowConcurrencyOptions);
+
+ Map highConcurrencyOptions = new HashMap<>(BASE_OPTIONS);
+ highConcurrencyOptions.put("s3-http-max-concurrency", "256");
+ S3ClientConfig highConcurrency = createConfig(highConcurrencyOptions);
+
+ S3AsyncClient lowConcurrencyClient = S3ClientCache.getS3AsyncClient(lowConcurrency);
+ S3AsyncClient highConcurrencyClient = S3ClientCache.getS3AsyncClient(highConcurrency);
+
+ assertThat(lowConcurrencyClient).isNotSameAs(highConcurrencyClient);
+ assertThat(S3ClientCache.cacheSize()).isEqualTo(2);
+
+ S3ClientCache.close(lowConcurrency);
+
+ assertThat(S3ClientCache.isCached(lowConcurrency)).isFalse();
+ assertThat(S3ClientCache.isCached(highConcurrency)).isFalse();
+ }
+
+ @Test
+ void testDifferentConfigReturnsDifferentClient()
+ {
+ S3ClientConfig config1 = createConfig(BASE_OPTIONS);
+
+ Map options2 = new HashMap<>(BASE_OPTIONS);
+ options2.put("s3-region", "eu-west-1");
+ S3ClientConfig config2 = createConfig(options2);
+
+ S3Client client1 = S3ClientCache.getS3Client(config1);
+ S3Client client2 = S3ClientCache.getS3Client(config2);
+
+ assertThat(client1).isNotSameAs(client2);
+ assertThat(S3ClientCache.cacheSize()).isEqualTo(2);
+ }
+
+ @Test
+ void testResolveMaxConcurrencyUsesOverride()
+ {
+ Map options = new HashMap<>(BASE_OPTIONS);
+ options.put("s3-http-max-concurrency", "333");
+
+ assertThat(S3ClientCache.resolveMaxConcurrency(createConfig(options))).isEqualTo(333);
+ }
+
+ @Test
+ void testResolveMaxConcurrencyAutoClampsFromTaskSlots()
+ {
+ assertThat(S3ClientCache.resolveMaxConcurrency(1)).isEqualTo(128);
+ assertThat(S3ClientCache.resolveMaxConcurrency(16)).isEqualTo(256);
+ assertThat(S3ClientCache.resolveMaxConcurrency(256)).isEqualTo(1024);
+ }
+
+ @Test
+ void testResolveMaxPendingConnectionAcquires()
+ {
+ assertThat(S3ClientCache.resolveMaxPendingConnectionAcquires(50)).isEqualTo(256);
+ assertThat(S3ClientCache.resolveMaxPendingConnectionAcquires(256)).isEqualTo(512);
+ assertThat(S3ClientCache.resolveMaxPendingConnectionAcquires(2000)).isEqualTo(2048);
+ }
+
+ @Test
+ void testNonBlockingDnsResolverAvailabilityMatchesClasspath()
+ {
+ assertThat(S3ClientCache.isNonBlockingDnsResolverAvailable())
+ .isEqualTo(isClassAvailable(S3ClientCache.NETTY_DNS_RESOLVER_CLASS));
+ }
+
+ @Test
+ void testResolveTaskSlotsPrioritizesSparkConfThenEnvThenRuntime()
+ {
+ SparkConf conf = new SparkConf(false)
+ .set("spark.executor.cores", "16")
+ .set("spark.task.cpus", "2");
+
+ S3ClientCache.ResolvedTaskSlots fromSparkConf =
+ S3ClientCache.resolveTaskSlotsForAutoSize(conf, "4", 128);
+ assertThat(fromSparkConf.taskSlots).isEqualTo(8);
+ assertThat(fromSparkConf.source).isEqualTo("SparkEnv.spark.executor.cores/spark.task.cpus");
+
+ S3ClientCache.ResolvedTaskSlots fromEnv =
+ S3ClientCache.resolveTaskSlotsForAutoSize(null, "4", 128);
+ assertThat(fromEnv.taskSlots).isEqualTo(4);
+ assertThat(fromEnv.source).isEqualTo("SPARK_EXECUTOR_CORES env");
+
+ S3ClientCache.ResolvedTaskSlots fromRuntime =
+ S3ClientCache.resolveTaskSlotsForAutoSize(null, "not-an-int", 7);
+ assertThat(fromRuntime.taskSlots).isEqualTo(7);
+ assertThat(fromRuntime.source).isEqualTo("Runtime.availableProcessors fallback");
+ }
+
+ @Test
+ void testCloseAll()
+ {
+ S3ClientConfig config1 = createConfig(BASE_OPTIONS);
+
+ Map options2 = new HashMap<>(BASE_OPTIONS);
+ options2.put("s3-region", "eu-west-1");
+ S3ClientConfig config2 = createConfig(options2);
+
+ // Create some clients
+ S3ClientCache.getS3Client(config1);
+ S3ClientCache.getS3AsyncClient(config2);
+
+ assertThat(S3ClientCache.cacheSize()).isGreaterThan(0);
+
+ // Close all
+ S3ClientCache.closeAll();
+
+ assertThat(S3ClientCache.cacheSize()).isEqualTo(0);
+ assertThat(S3ClientCache.isCached(config1)).isFalse();
+ assertThat(S3ClientCache.isCached(config2)).isFalse();
+ }
+
+ @Test
+ void testCloseSpecificConfig()
+ {
+ S3ClientConfig config1 = createConfig(BASE_OPTIONS);
+
+ Map options2 = new HashMap<>(BASE_OPTIONS);
+ options2.put("s3-region", "eu-west-1");
+ S3ClientConfig config2 = createConfig(options2);
+
+ // Create clients for both configs
+ S3ClientCache.getS3Client(config1);
+ S3ClientCache.getS3Client(config2);
+
+ assertThat(S3ClientCache.cacheSize()).isEqualTo(2);
+
+ // Close only config1
+ S3ClientCache.close(config1);
+
+ assertThat(S3ClientCache.isCached(config1)).isFalse();
+ assertThat(S3ClientCache.isCached(config2)).isTrue();
+ assertThat(S3ClientCache.cacheSize()).isEqualTo(1);
+ }
+
+ @Test
+ void testThreadSafety() throws InterruptedException
+ {
+ int numThreads = 10;
+ int iterationsPerThread = 100;
+ ExecutorService executor = Executors.newFixedThreadPool(numThreads);
+ CountDownLatch startLatch = new CountDownLatch(1);
+ CountDownLatch doneLatch = new CountDownLatch(numThreads);
+ AtomicInteger errors = new AtomicInteger(0);
+
+ S3ClientConfig config = createConfig(BASE_OPTIONS);
+
+ for (int t = 0; t < numThreads; t++)
+ {
+ executor.submit(() -> {
+ try
+ {
+ startLatch.await();
+ for (int i = 0; i < iterationsPerThread; i++)
+ {
+ S3Client client = S3ClientCache.getS3Client(config);
+ if (client == null)
+ {
+ errors.incrementAndGet();
+ }
+ }
+ }
+ catch (Exception e)
+ {
+ errors.incrementAndGet();
+ }
+ finally
+ {
+ doneLatch.countDown();
+ }
+ });
+ }
+
+ // Start all threads simultaneously
+ startLatch.countDown();
+
+ // Wait for completion
+ boolean completed = doneLatch.await(30, TimeUnit.SECONDS);
+ executor.shutdown();
+
+ assertThat(completed).isTrue();
+ assertThat(errors.get()).isEqualTo(0);
+ // All threads should get the same cached instance
+ assertThat(S3ClientCache.cacheSize()).isEqualTo(1);
+ }
+
+ @Test
+ void testNullCredentialsUsesDefaultProvider()
+ {
+ // Config without explicit credentials
+ S3ClientConfig config = createConfig(BASE_OPTIONS);
+
+ // Should not throw - will use DefaultCredentialsProvider
+ S3Client client = S3ClientCache.getS3Client(config);
+ assertThat(client).isNotNull();
+ }
+
+ @Test
+ void testCacheKeyDeterministic()
+ {
+ // Same config created multiple times should produce same key
+ S3ClientConfig config1 = createConfig(BASE_OPTIONS);
+ S3ClientConfig config2 = createConfig(BASE_OPTIONS);
+
+ String key1 = S3ClientCache.getCacheKey(config1);
+ String key2 = S3ClientCache.getCacheKey(config2);
+
+ assertThat(key1).isEqualTo(key2);
+ }
+
+ @Test
+ void testSecretHashDeterministic()
+ {
+ Map options = new HashMap<>(BASE_OPTIONS);
+ options.put("s3-access-key-id", "AKIATEST");
+ options.put("s3-secret-access-key", "mySecretKey");
+
+ S3ClientConfig config1 = createConfig(options);
+ S3ClientConfig config2 = createConfig(options);
+
+ String key1 = S3ClientCache.getCacheKey(config1);
+ String key2 = S3ClientCache.getCacheKey(config2);
+
+ // Same secret should produce same hash
+ assertThat(key1).isEqualTo(key2);
+ }
+
+ private S3ClientConfig createConfig(Map options)
+ {
+ return S3ClientConfig.create(new CaseInsensitiveStringMap(options));
+ }
+
+ private static boolean isClassAvailable(String className)
+ {
+ try
+ {
+ Class.forName(className, false, S3ClientCacheTest.class.getClassLoader());
+ return true;
+ }
+ catch (ClassNotFoundException exception)
+ {
+ return false;
+ }
+ }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/S3ClientConfigTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/S3ClientConfigTest.java
new file mode 100644
index 000000000..d4a39ca5b
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/S3ClientConfigTest.java
@@ -0,0 +1,127 @@
+/*
+ * 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 org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/**
+ * Unit tests for {@link S3ClientConfig}.
+ */
+class S3ClientConfigTest
+{
+ private static final Map REQUIRED_OPTIONS = ImmutableMap.builder()
+ .put("s3-region", "us-west-2")
+ .put("s3-bucket", "test-bucket")
+ .build();
+
+ @Test
+ void testCreateWithRequiredOptions()
+ {
+ S3ClientConfig config = S3ClientConfig.create(new CaseInsensitiveStringMap(REQUIRED_OPTIONS));
+
+ assertThat(config.s3Region()).isEqualTo("us-west-2");
+ assertThat(config.s3Bucket()).isEqualTo("test-bucket");
+ assertThat(config.s3EndpointOverride()).isNull();
+ assertThat(config.s3AccessKeyId()).isNull();
+ assertThat(config.s3SecretAccessKey()).isNull();
+ }
+
+ @Test
+ void testCreateWithAllOptions()
+ {
+ Map options = new HashMap<>(REQUIRED_OPTIONS);
+ options.put("s3-endpoint-override", "http://localhost:9000");
+ options.put("s3-access-key-id", "test-access-key");
+ options.put("s3-secret-access-key", "test-secret-key");
+
+ S3ClientConfig config = S3ClientConfig.create(new CaseInsensitiveStringMap(options));
+
+ assertThat(config.s3Region()).isEqualTo("us-west-2");
+ assertThat(config.s3Bucket()).isEqualTo("test-bucket");
+ assertThat(config.s3EndpointOverride()).isEqualTo("http://localhost:9000");
+ assertThat(config.s3AccessKeyId()).isEqualTo("test-access-key");
+ assertThat(config.s3SecretAccessKey()).isEqualTo("test-secret-key");
+ }
+
+ @Test
+ void testMissingRegion()
+ {
+ Map options = new HashMap<>();
+ options.put("s3-bucket", "test-bucket");
+
+ assertThatThrownBy(() -> S3ClientConfig.create(new CaseInsensitiveStringMap(options)))
+ .isInstanceOf(RuntimeException.class);
+ }
+
+ @Test
+ void testMissingBucket()
+ {
+ Map options = new HashMap<>();
+ options.put("s3-region", "us-west-2");
+
+ assertThatThrownBy(() -> S3ClientConfig.create(new CaseInsensitiveStringMap(options)))
+ .isInstanceOf(RuntimeException.class);
+ }
+
+ @Test
+ void testCreateWithExplicitValues()
+ {
+ S3ClientConfig config = S3ClientConfig.create(
+ "eu-west-1",
+ "my-bucket",
+ "http://minio:9000",
+ "access123",
+ "secret456"
+ );
+
+ assertThat(config.s3Region()).isEqualTo("eu-west-1");
+ assertThat(config.s3Bucket()).isEqualTo("my-bucket");
+ assertThat(config.s3EndpointOverride()).isEqualTo("http://minio:9000");
+ assertThat(config.s3AccessKeyId()).isEqualTo("access123");
+ assertThat(config.s3SecretAccessKey()).isEqualTo("secret456");
+ }
+
+ @Test
+ void testCreateWithExplicitValuesNullOptional()
+ {
+ S3ClientConfig config = S3ClientConfig.create(
+ "us-east-1",
+ "bucket",
+ null,
+ null,
+ null
+ );
+
+ assertThat(config.s3Region()).isEqualTo("us-east-1");
+ assertThat(config.s3Bucket()).isEqualTo("bucket");
+ assertThat(config.s3EndpointOverride()).isNull();
+ assertThat(config.s3AccessKeyId()).isNull();
+ assertThat(config.s3SecretAccessKey()).isNull();
+ }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/S3DataSourceClientConfigBufferTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/S3DataSourceClientConfigBufferTest.java
new file mode 100644
index 000000000..98a88f38f
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/S3DataSourceClientConfigBufferTest.java
@@ -0,0 +1,254 @@
+/*
+ * 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 java.util.UUID;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+
+import org.apache.cassandra.spark.data.backup.BackupReaderRegistry;
+import org.apache.cassandra.spark.data.backup.FakeBackupReader;
+import org.apache.cassandra.spark.utils.Properties;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/**
+ * Unit tests for the new DataSource options on {@link S3DataSourceClientConfig}:
+ * Data.db buffer sizing and SSTable metadata cache sizing. These are the user-visible knobs that
+ * determine how many S3 GETs (and KMS decrypts) a given Spark job will issue, so regressions that
+ * silently fall back to the tiny legacy defaults would immediately erode the gains from the
+ * optimization. Serialization/Kryo coverage for the buffer-size round trip lives in
+ * {@link S3CassandraDataLayerTests}.
+ */
+class S3DataSourceClientConfigBufferTest
+{
+ private static final String TEST_BACKUP_READER_TYPE = "fake";
+
+ // Sysprops we may touch inside applySSTableCacheSystemProperties() — remembered here so each
+ // test cleans up after itself and does not leak into sibling tests running in the same JVM.
+ private static final String[] CACHE_SYSPROPS = {
+ "sbr.cache.summary.maxEntries",
+ "sbr.cache.index.maxEntries",
+ "sbr.cache.stats.maxEntries",
+ "sbr.cache.filter.maxEntries",
+ "sbr.cache.compressionInfo.maxEntries"
+ };
+
+ @BeforeAll
+ static void registerBackupReader()
+ {
+ BackupReaderRegistry.register(TEST_BACKUP_READER_TYPE, config -> new FakeBackupReader(config.s3Config(), config.s3Config().s3Bucket()));
+ }
+
+ @AfterEach
+ void clearCacheSysprops()
+ {
+ for (String name : CACHE_SYSPROPS)
+ {
+ System.clearProperty(name);
+ }
+ }
+
+ private static Map minimalOptions()
+ {
+ // Case-sensitivity: MapUtils.lowerCaseKey() lower-cases the keys, so we store lower-cased
+ // forms here to match the production option path. Real Spark DataSource options are already
+ // case-insensitive from the caller's perspective.
+ Map options = new HashMap<>();
+ // The cluster identifier is opaque to the buffer-size path we are testing.
+ options.put("clusterid", UUID.randomUUID().toString());
+ options.put("keyspace", "ks");
+ options.put("table", "tbl");
+ options.put("tablecreatestmt", "CREATE TABLE ks.tbl (k int PRIMARY KEY)");
+ options.put("s3-region", "us-west-2");
+ options.put("s3-bucket", "bucket");
+ options.put("backupreadertype", TEST_BACKUP_READER_TYPE);
+ return options;
+ }
+
+ @Test
+ void defaultsMatchPropertiesConstants()
+ {
+ S3DataSourceClientConfig config = S3DataSourceClientConfig.create(minimalOptions());
+
+ assertThat(config.s3DataChunkBufferSize()).isEqualTo(Properties.DEFAULT_S3_DATA_CHUNK_BUFFER_SIZE);
+ assertThat(config.s3DataMaxBufferSize()).isEqualTo(Properties.DEFAULT_S3_DATA_MAX_BUFFER_SIZE);
+ assertThat(config.s3DataMaxBufferSize())
+ .as("max buffer must be >= chunk buffer so BufferingInputStream can enqueue a full chunk")
+ .isGreaterThanOrEqualTo(config.s3DataChunkBufferSize());
+
+ // Data.db publisher-read path defaults to enabled.
+ assertThat(config.sstableDataPublisherReadEnabled())
+ .as("sstableDataPublisherReadEnabled must default true to use the streaming path")
+ .isTrue();
+
+ // Cache defaults — aligned with SSTableCache bump.
+ assertThat(config.sstableCacheSummaryMaxEntries()).isEqualTo(32768);
+ assertThat(config.sstableCacheIndexMaxEntries()).isEqualTo(16384);
+ assertThat(config.sstableCacheStatsMaxEntries()).isEqualTo(16384);
+ assertThat(config.sstableCacheFilterMaxEntries()).isEqualTo(16384);
+ assertThat(config.sstableCacheCompressionInfoMaxEntries()).isEqualTo(16384);
+ }
+
+ @Test
+ void sstableDataPublisherReadEnabledExplicitTrueIsHonored()
+ {
+ Map options = minimalOptions();
+ options.put("sstabledatapublisherreadenabled", "true");
+
+ S3DataSourceClientConfig config = S3DataSourceClientConfig.create(options);
+
+ assertThat(config.sstableDataPublisherReadEnabled())
+ .as("explicit true must enable the AsyncResponseTransformer.toPublisher() experiment path")
+ .isTrue();
+ }
+
+ @Test
+ void sstableDataPublisherReadEnabledExplicitFalseIsHonored()
+ {
+ Map options = minimalOptions();
+ options.put("sstabledatapublisherreadenabled", "false");
+
+ S3DataSourceClientConfig config = S3DataSourceClientConfig.create(options);
+
+ assertThat(config.sstableDataPublisherReadEnabled())
+ .as("explicit false must keep the toBytes() baseline path for Data.db")
+ .isFalse();
+ }
+
+ @Test
+ void overridesAreHonored()
+ {
+ // Use values distinct from the 8/32 MiB defaults to ensure the override path actually flows
+ // through, not just a coincidental match with the constants.
+ Map options = minimalOptions();
+ options.put("sstabledatachunkbuffersize", String.valueOf(16L * 1024 * 1024));
+ options.put("sstabledatamaxbuffersize", String.valueOf(48L * 1024 * 1024));
+ options.put("sstablecachesummarymaxentries", "4096");
+ options.put("sstablecachecompressioninfomaxentries", "2048");
+
+ S3DataSourceClientConfig config = S3DataSourceClientConfig.create(options);
+
+ assertThat(config.s3DataChunkBufferSize()).isEqualTo(16L * 1024 * 1024);
+ assertThat(config.s3DataMaxBufferSize()).isEqualTo(48L * 1024 * 1024);
+ assertThat(config.sstableCacheSummaryMaxEntries()).isEqualTo(4096);
+ assertThat(config.sstableCacheCompressionInfoMaxEntries()).isEqualTo(2048);
+ }
+
+ @Test
+ void maxBufferIsClampedToAtLeastChunkSize()
+ {
+ // Users who supply a maxBuffer smaller than the chunk size would break the BufferingInputStream
+ // invariant (one chunk must fit without immediately tripping isBufferFull()). Verify the
+ // defensive clamp we apply in the constructor.
+ Map options = minimalOptions();
+ options.put("sstabledatachunkbuffersize", String.valueOf(64L * 1024 * 1024));
+ options.put("sstabledatamaxbuffersize", String.valueOf(1L * 1024 * 1024));
+
+ S3DataSourceClientConfig config = S3DataSourceClientConfig.create(options);
+
+ assertThat(config.s3DataChunkBufferSize()).isEqualTo(64L * 1024 * 1024);
+ assertThat(config.s3DataMaxBufferSize())
+ .as("maxBuffer must be clamped up to at least chunk size")
+ .isEqualTo(64L * 1024 * 1024);
+ }
+
+ @Test
+ void chunkSizeHasMinimumOneMebibyte()
+ {
+ // Defense-in-depth: an accidentally tiny override would nerf throughput severely. 1 MiB is a
+ // conservative floor — still smaller than any observed Data.db component — so it is safe to
+ // clamp rather than throw.
+ Map options = minimalOptions();
+ options.put("sstabledatachunkbuffersize", "0");
+
+ S3DataSourceClientConfig config = S3DataSourceClientConfig.create(options);
+
+ assertThat(config.s3DataChunkBufferSize()).isEqualTo(1024L * 1024L);
+ }
+
+ @Test
+ void applySSTableCacheSystemPropertiesSetsUnsetSysprops()
+ {
+ for (String name : CACHE_SYSPROPS)
+ {
+ assertThat(System.getProperty(name))
+ .as("precondition: cache sysprop %s should not be set prior to the test", name)
+ .isNull();
+ }
+
+ S3DataSourceClientConfig config = S3DataSourceClientConfig.create(minimalOptions());
+ config.applySSTableCacheSystemProperties();
+
+ assertThat(System.getProperty("sbr.cache.summary.maxEntries")).isEqualTo("32768");
+ assertThat(System.getProperty("sbr.cache.index.maxEntries")).isEqualTo("16384");
+ assertThat(System.getProperty("sbr.cache.stats.maxEntries")).isEqualTo("16384");
+ assertThat(System.getProperty("sbr.cache.filter.maxEntries")).isEqualTo("16384");
+ assertThat(System.getProperty("sbr.cache.compressionInfo.maxEntries")).isEqualTo("16384");
+ }
+
+ @Test
+ void negativeCacheSizeFailsFastWithOptionNameInError()
+ {
+ // Negative max-entries values would otherwise reach Guava CacheBuilder.maximumSize() and surface
+ // as a generic IllegalArgumentException ("size cannot be negative") with no mention of which
+ // option the operator misconfigured. Catching this at config-parse time with the option key in
+ // the message is dramatically more debuggable.
+ Map options = minimalOptions();
+ options.put("sstablecacheindexmaxentries", "-1");
+
+ assertThatThrownBy(() -> S3DataSourceClientConfig.create(options))
+ .isInstanceOf(IllegalArgumentException.class)
+ .hasMessageContaining("sstableCacheIndexMaxEntries")
+ .hasMessageContaining("-1");
+ }
+
+ @Test
+ void zeroCacheSizeIsAccepted()
+ {
+ // 0 is a legitimate way to disable the cache entirely (CacheBuilder.maximumSize(0) is valid).
+ // Make sure the fast-fail validation does not swing too hard and reject this case.
+ Map options = minimalOptions();
+ options.put("sstablecachesummarymaxentries", "0");
+
+ S3DataSourceClientConfig config = S3DataSourceClientConfig.create(options);
+ assertThat(config.sstableCacheSummaryMaxEntries()).isZero();
+ }
+
+ @Test
+ void applySSTableCacheSystemPropertiesDoesNotOverwriteExistingSysprops()
+ {
+ // Operators that ship cluster-wide -D flags should see their values win over the defaults —
+ // otherwise a fleet-wide deploy could silently revert their tuning.
+ System.setProperty("sbr.cache.summary.maxEntries", "65536");
+
+ S3DataSourceClientConfig config = S3DataSourceClientConfig.create(minimalOptions());
+ config.applySSTableCacheSystemProperties();
+
+ assertThat(System.getProperty("sbr.cache.summary.maxEntries"))
+ .as("operator-set sysprops must win over config defaults")
+ .isEqualTo("65536");
+ }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/S3DataSourceClientConfigTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/S3DataSourceClientConfigTest.java
new file mode 100644
index 000000000..24f22dbc1
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/S3DataSourceClientConfigTest.java
@@ -0,0 +1,548 @@
+/*
+ * 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 java.util.Set;
+
+import com.google.common.collect.ImmutableMap;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.CsvSource;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.spark.data.backup.BackupReaderRegistry;
+import org.apache.cassandra.spark.data.backup.FakeBackupReader;
+import org.apache.cassandra.spark.data.partitioner.ConsistencyLevel;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+class S3DataSourceClientConfigTest
+{
+ private static final String TEST_CLUSTER_UUID = "aaaaaaaa-bbbb-cccc-dddd-eeeeeeeeeeee";
+ private static final String TEST_BACKUP_READER_TYPE = "fake";
+ public static final Map REQUIRED_CONFIG_OPTIONS = ImmutableMap.builder()
+ .put("clusterName", TEST_CLUSTER_UUID)
+ .put("keyspace", "test_keyspace")
+ .put("table", "test_table")
+ .put("tableCreateStmt",
+ "CREATE TABLE test_keyspace.test_table (id uuid PRIMARY KEY, name text)")
+ .put("s3-region", "us-west-2")
+ .put("s3-bucket", "test-bucket")
+ .put("backupReaderType", TEST_BACKUP_READER_TYPE)
+ .build();
+
+ @BeforeAll
+ static void registerBackupReader()
+ {
+ BackupReaderRegistry.register(TEST_BACKUP_READER_TYPE, config -> new FakeBackupReader(config.s3Config(), config.s3Config().s3Bucket()));
+ }
+
+ @Test
+ void testValidConfigurationWithRequiredOptionsOnly()
+ {
+ S3DataSourceClientConfig config = S3DataSourceClientConfig.create(new CaseInsensitiveStringMap(REQUIRED_CONFIG_OPTIONS));
+
+ // Test Cassandra-related fields
+ assertThat(config.clusterName()).isEqualTo(TEST_CLUSTER_UUID);
+ assertThat(config.keyspace()).isEqualTo("test_keyspace");
+ assertThat(config.table()).isEqualTo("test_table");
+ assertThat(config.tableCreateStmt()).isEqualTo("CREATE TABLE test_keyspace.test_table (id uuid PRIMARY KEY, name text)");
+
+ // Test S3-related fields
+ assertThat(config.s3Region()).isEqualTo("us-west-2");
+ assertThat(config.s3Bucket()).isEqualTo("test-bucket");
+ assertThat(config.s3EndpointOverride()).isNull();
+ assertThat(config.s3AccessKeyId()).isNull();
+ assertThat(config.s3SecretAccessKey()).isNull();
+ assertThat(config.s3Config().s3HttpMaxConcurrency()).isEqualTo(0);
+
+ // Test defaults
+ assertThat(config.datacenter()).isNull();
+ assertThat(config.defaultParallelism()).isEqualTo(1);
+ assertThat(config.numCores()).isEqualTo(1);
+ assertThat(config.consistencyLevel()).isNull();
+ assertThat(config.enableStats()).isTrue();
+ assertThat(config.readIndexOffset()).isTrue();
+ assertThat(config.sizing()).isEqualTo("default");
+ assertThat(config.maxPartitionSize()).isEqualTo(1);
+ assertThat(config.numberSplits()).isEqualTo(-1);
+ assertThat(config.sstableTokenIndexEnabled()).isFalse();
+ assertThat(config.sstableTokenIndexPrebuildPartitions()).isEqualTo(0);
+ assertThat(config.sstableTokenIndexPrebuildPerTaskConcurrency()).isEqualTo(4);
+ assertThat(config.resolveSSTableTokenIndexPrebuildPartitions(1_400_000, 16_000)).isEqualTo(140);
+ assertThat(config.lastModifiedTimestampField()).isNull();
+ assertThat(config.udts()).isEmpty();
+ assertThat(config.cassandraVersion()).isEqualTo(CassandraVersion.FOURZERO);
+ }
+
+ @Test
+ void testValidConfigurationWithAllOptions()
+ {
+ Map allOptions = new HashMap<>(REQUIRED_CONFIG_OPTIONS);
+ allOptions.put("dc", "dc1");
+ allOptions.put("defaultParallelism", "4");
+ allOptions.put("numCores", "8");
+ allOptions.put("consistencyLevel", "LOCAL_QUORUM");
+ allOptions.put("enableStats", "false");
+ allOptions.put("readIndexOffset", "false");
+ allOptions.put("sizing", "dynamic");
+ allOptions.put("maxPartitionSize", "5");
+ allOptions.put("number_splits", "10");
+ allOptions.put("sstableTokenIndexEnabled", "true");
+ allOptions.put("sstableTokenIndexPrebuildPartitions", "80");
+ allOptions.put("sstableTokenIndexPrebuildPerTaskConcurrency", "0");
+ allOptions.put("lastModifiedColumnName", "last_updated");
+ allOptions.put("udts", "type1\ntype2\ntype3");
+ allOptions.put("cassandraVersion", "THREEZERO");
+ allOptions.put("s3-endpoint-override", "http://localhost:9000");
+ allOptions.put("s3-access-key-id", "test-access-key");
+ allOptions.put("s3-secret-access-key", "test-secret-key");
+ allOptions.put("s3-http-max-concurrency", "333");
+ allOptions.put("replicationStrategy", "SimpleStrategy");
+ allOptions.put("replicationFactor", "2");
+ Map caseInsensitiveOptions = new CaseInsensitiveStringMap(allOptions);
+
+ S3DataSourceClientConfig config = S3DataSourceClientConfig.create(caseInsensitiveOptions);
+
+ // Test Cassandra-related fields
+ assertThat(config.datacenter()).isEqualTo("dc1");
+ assertThat(config.defaultParallelism()).isEqualTo(4);
+ assertThat(config.numCores()).isEqualTo(8);
+ assertThat(config.consistencyLevel()).isEqualTo(ConsistencyLevel.LOCAL_QUORUM);
+ assertThat(config.enableStats()).isFalse();
+ assertThat(config.readIndexOffset()).isFalse();
+ assertThat(config.sizing()).isEqualTo("dynamic");
+ assertThat(config.maxPartitionSize()).isEqualTo(5);
+ assertThat(config.numberSplits()).isEqualTo(10);
+ assertThat(config.sstableTokenIndexEnabled()).isTrue();
+ assertThat(config.sstableTokenIndexPrebuildPartitions()).isEqualTo(80);
+ assertThat(config.sstableTokenIndexPrebuildPerTaskConcurrency()).isEqualTo(1);
+ assertThat(config.resolveSSTableTokenIndexPrebuildPartitions(1_400_000, 16_000)).isEqualTo(80);
+ assertThat(config.lastModifiedTimestampField()).isEqualTo("last_updated");
+ assertThat(config.udts()).isEqualTo("type1\ntype2\ntype3");
+ assertThat(config.cassandraVersion()).isEqualTo(CassandraVersion.THREEZERO);
+
+ // Test S3-related fields
+ assertThat(config.s3EndpointOverride()).isEqualTo("http://localhost:9000");
+ assertThat(config.s3AccessKeyId()).isEqualTo("test-access-key");
+ assertThat(config.s3SecretAccessKey()).isEqualTo("test-secret-key");
+ assertThat(config.s3Config().s3HttpMaxConcurrency()).isEqualTo(333);
+ }
+
+ @Test
+ void testRejectsNegativeS3HttpMaxConcurrency()
+ {
+ Map options = new HashMap<>(REQUIRED_CONFIG_OPTIONS);
+ options.put("s3-http-max-concurrency", "-1");
+ Map caseInsensitiveOptions = new CaseInsensitiveStringMap(options);
+
+ assertThatThrownBy(() -> S3DataSourceClientConfig.create(caseInsensitiveOptions))
+ .isInstanceOf(IllegalArgumentException.class)
+ .hasMessageContaining("s3-http-max-concurrency");
+ }
+
+ @Test
+ void testParsedUdts()
+ {
+ Map options = new HashMap<>(REQUIRED_CONFIG_OPTIONS);
+ options.put("udts", "type1\ntype2\n\ntype3\n");
+ Map caseInsensitiveOptions = new CaseInsensitiveStringMap(options);
+
+ S3DataSourceClientConfig config = S3DataSourceClientConfig.create(caseInsensitiveOptions);
+ Set parsedUdts = config.parsedUdts();
+
+ assertThat(parsedUdts).containsExactlyInAnyOrder("type1", "type2", "type3");
+ }
+
+ @Test
+ void testParsedUdtsEmpty()
+ {
+ S3DataSourceClientConfig config = S3DataSourceClientConfig.create(new CaseInsensitiveStringMap(REQUIRED_CONFIG_OPTIONS));
+ Set parsedUdts = config.parsedUdts();
+ assertThat(parsedUdts).isEmpty();
+ }
+
+ @Test
+ void testRequestedFeaturesWithLastModifiedColumn()
+ {
+ Map options = new HashMap<>(REQUIRED_CONFIG_OPTIONS);
+ options.put("lastModifiedColumnName", "updated_at");
+ Map caseInsensitiveOptions = new CaseInsensitiveStringMap(options);
+
+ S3DataSourceClientConfig config = S3DataSourceClientConfig.create(caseInsensitiveOptions);
+ assertThat(config.requestedFeatures()).isNotNull();
+ assertThat(config.requestedFeatures()).isNotEmpty();
+ }
+
+ @ParameterizedTest
+ @ValueSource(strings = { "clusterName", "keyspace", "table", "tableCreateStmt", "s3-region", "s3-bucket" })
+ void testMissingRequiredFields(String requiredField)
+ {
+ Map incompleteOptions = new HashMap<>(REQUIRED_CONFIG_OPTIONS);
+ incompleteOptions.remove(requiredField);
+ Map caseInsensitiveOptions = new CaseInsensitiveStringMap(incompleteOptions);
+
+ assertThatThrownBy(() -> S3DataSourceClientConfig.create(caseInsensitiveOptions))
+ .isInstanceOf(RuntimeException.class);
+ }
+
+ @Test
+ void testClusterIdOnlyWithoutClusterName()
+ {
+ Map