diff --git a/analytics-sidecar-client/src/main/java/org/apache/cassandra/sidecar/client/SidecarClient.java b/analytics-sidecar-client/src/main/java/org/apache/cassandra/sidecar/client/SidecarClient.java index 56b3b31c0..3b2eecf7c 100644 --- a/analytics-sidecar-client/src/main/java/org/apache/cassandra/sidecar/client/SidecarClient.java +++ b/analytics-sidecar-client/src/main/java/org/apache/cassandra/sidecar/client/SidecarClient.java @@ -244,6 +244,19 @@ public CompletableFuture gossipInfo() return executor.executeRequestAsync(requestBuilder().gossipInfoRequest().build()); } + /** + * Executes the gossip info request using the default retry policy and configured selection policy + * + * @param instance the instance where the request will be executed + * @return a completable future of the gossip info + */ + public CompletableFuture gossipInfo(SidecarInstance instance) + { + return executor.executeRequestAsync(requestBuilder().singleInstanceSelectionPolicy(instance) + .gossipInfoRequest() + .build()); + } + /** * Executes the GET gossip health request using the default retry policy and configured selection policy diff --git a/cassandra-analytics-common/src/main/java/org/apache/cassandra/bridge/BridgeInitializationParameters.java b/cassandra-analytics-common/src/main/java/org/apache/cassandra/bridge/BridgeInitializationParameters.java index d7857a1be..3cc7714d3 100644 --- a/cassandra-analytics-common/src/main/java/org/apache/cassandra/bridge/BridgeInitializationParameters.java +++ b/cassandra-analytics-common/src/main/java/org/apache/cassandra/bridge/BridgeInitializationParameters.java @@ -24,21 +24,20 @@ */ public class BridgeInitializationParameters { - private final String sstableFormat; + private final String configuredSSTableFormat; - public BridgeInitializationParameters(String sstableFormat) + public BridgeInitializationParameters(String configuredSSTableFormat) { - this.sstableFormat = sstableFormat; + this.configuredSSTableFormat = configuredSSTableFormat; } public static BridgeInitializationParameters fromEnvironment() { - String sstableFormat = CassandraVersion.sstableFormat(); - return new BridgeInitializationParameters(sstableFormat); + return new BridgeInitializationParameters(CassandraVersion.configuredSSTableFormat()); } - public String getSstableFormat() + public String getConfiguredSSTableFormat() { - return sstableFormat; + return configuredSSTableFormat; } } diff --git a/cassandra-analytics-common/src/main/java/org/apache/cassandra/bridge/CassandraVersion.java b/cassandra-analytics-common/src/main/java/org/apache/cassandra/bridge/CassandraVersion.java index 167039efd..4181b3ecd 100644 --- a/cassandra-analytics-common/src/main/java/org/apache/cassandra/bridge/CassandraVersion.java +++ b/cassandra-analytics-common/src/main/java/org/apache/cassandra/bridge/CassandraVersion.java @@ -20,7 +20,9 @@ package org.apache.cassandra.bridge; import java.util.Arrays; +import java.util.Collections; import java.util.HashSet; +import java.util.List; import java.util.Optional; import java.util.Set; @@ -35,22 +37,49 @@ */ public enum CassandraVersion { - THREEZERO(30, "3.0", "three-zero", "big"), - FOURZERO(40, "4.0", "four-zero", "big"), - FOURONE(41, "4.1", "four-zero", "big"), - FIVEZERO(50, "5.0", "five-zero", "big", "bti"); + THREEZERO(30, "3.0", "three-zero", new String[]{"big"}, + new String[]{ + // Cassandra 3.x native sstable versions + // order is important, used to determine the latest version + "big-ma", + "big-mb", + "big-mc", + "big-md", + "big-me", + "big-mf" + }), + FOURZERO(40, "4.0", "four-zero", new String[]{"big"}, + new String[]{ + // Cassandra 4.0 native sstable versions + // order is important, used to determine the latest version + "big-na", + "big-nb", + }), + FOURONE(41, "4.1", "four-zero", new String[]{"big"}, + new String[]{ + // Cassandra 4.1 did not introduce new native SSTable versions + }), + FIVEZERO(50, "5.0", "five-zero", new String[]{"big", "bti"}, + new String[]{ + // Cassandra 5.0 native sstable versions + "big-oa", + "bti-da", + }); private final int number; private final String name; private final String jarBaseName; // Must match shadowJar.archiveFileName from Gradle configuration (without extension) private final Set sstableFormats; + private final List nativeSStableVersions; // Preserves array order for version comparison - CassandraVersion(int number, String name, String jarBaseName, String... sstableFormats) + + CassandraVersion(int number, String name, String jarBaseName, String[] sstableFormats, String[] nativeSStableVersions) { this.number = number; this.name = name; this.jarBaseName = jarBaseName; this.sstableFormats = new HashSet<>(Arrays.asList(sstableFormats)); + this.nativeSStableVersions = Collections.unmodifiableList(Arrays.asList(nativeSStableVersions)); } public int versionNumber() @@ -68,26 +97,116 @@ public String jarBaseName() return jarBaseName; } - private static final String sstableFormat; + /** + * Get the set of SSTable formats supported by this Cassandra version. + * + * @return Set of supported SSTable format strings + */ + public Set sstableFormats() + { + return sstableFormats; + } + + /** + * Get the ordered list of native SSTable version strings for this Cassandra version. + * The order matches the definition in the enum and represents version progression. + * For example, in FOURZERO: ["big-na", "big-nb"], big-nb is considered newer/higher. + * + * @return Ordered list of native SSTable version strings + */ + public List getNativeSStableVersions() + { + return nativeSStableVersions; + } + + /** + * Get the index/position of an SSTable version within this Cassandra version's native versions. + * This can be used for version comparison while sorting sstable versions from oldest to latest order. + * + * @param sstableVersion The SSTable version string to find + * @return Index of the version (0-based), or -1 if not found + */ + public int getSSTableVersionIndex(String sstableVersion) + { + return nativeSStableVersions.indexOf(sstableVersion); + } + + /** + * Get the set of SSTable version strings that this Cassandra version can read. + * This includes: + * - Native versions for this Cassandra version + * - All SSTable versions from the previous major version (including all minor versions) + * For example, Cassandra 5.0 can read: + * - 5.0 native versions (big-oa, bti-da) + * - 4.0 versions (big-na, big-nb) + * - 4.1 versions (if any) + * But NOT 3.0 versions + * + * @return Set of full SSTable version strings that can be read + */ + public Set getSupportedSStableVersionsForRead() + { + Set readableVersions = new HashSet<>(this.nativeSStableVersions); + + int previousMajor = getPreviousMajorVersion(); + + // Add all SSTable versions from the previous major version and its minors + // E.g., C* 5.0 (version 50) can read C* 4.0 (40) and C* 4.1 (41) SSTables, but not C* 3.x (30) + for (CassandraVersion version : CassandraVersion.values()) + { + // Include versions from the previous major version family (e.g., 40-49 for C* 5.0) + if (version.versionNumber() >= previousMajor && version.versionNumber() < this.number) + { + readableVersions.addAll(version.nativeSStableVersions); + } + } + + return Collections.unmodifiableSet(readableVersions); + } + + /** + * Get the previous major version number for this Cassandra version. + * Calculates dynamically using: (majorVersion - 1) * 10 + * For example: + * - C5.0 (50) returns 40 (C4.x) + * - C4.1 (41) returns 30 (C3.x) + * - C4.0 (40) returns 30 (C3.x) + * - C3.0 (30) returns 20 (C2.x - which doesn't exist) + * - C10.0 (100) returns 90 (C9.x) + * + * @return previous major version number + */ + @VisibleForTesting + int getPreviousMajorVersion() + { + // Get major version: 50 -> 5, 41 -> 4, 40 -> 4, 30 -> 3 + int majorVersion = this.number / 10; + + // Calculate previous major version: (majorVersion - 1) * 10 + // E.g., 5 -> 40, 4 -> 30, 3 -> 20 + return (majorVersion - 1) * 10; + } + + private static final String configuredSSTableFormat; private static final CassandraVersion[] implementedVersions; private static final String[] supportedVersions; static { - sstableFormat = System.getProperty("cassandra.analytics.bridges.sstable_format", "big"); + configuredSSTableFormat = System.getProperty("cassandra.analytics.bridges.sstable_format", "big"); String providedVersionsOrDefault = System.getProperty("cassandra.analytics.bridges.implemented_versions", String.join(",", FOURZERO.name(), FIVEZERO.name())); implementedVersions = Arrays.stream(providedVersionsOrDefault.split(",")) .map(CassandraVersion::valueOf) - .filter(v -> v.sstableFormats.contains(sstableFormat)) + .filter(v -> v.sstableFormats().contains(configuredSSTableFormat)) .toArray(CassandraVersion[]::new); String providedSupportedVersionsOrDefault = System.getProperty("cassandra.analytics.bridges.supported_versions", "cassandra-4.0.17,cassandra-5.0.5"); supportedVersions = Arrays.stream(providedSupportedVersionsOrDefault.split(",")) .filter(version -> CassandraVersion.fromVersion(version) - .filter(v -> v.sstableFormats.contains(sstableFormat)) + .filter(v -> v.sstableFormats().contains(configuredSSTableFormat)) .isPresent()) .toArray(String[]::new); @@ -95,9 +214,9 @@ public String jarBaseName() "No versions available"); } - public static String sstableFormat() + public static String configuredSSTableFormat() { - return sstableFormat; + return configuredSSTableFormat; } public static Optional fromVersion(String cassandraVersion) @@ -108,6 +227,37 @@ public static Optional fromVersion(String cassandraVersion) .findAny(); } + /** + * Find the Cassandra version that originally writes SSTables with this version string. + * Returns the native Cassandra version that introduced this SSTable version. + * + * @param sstableVersion full version string including format (e.g., "big-na", "bti-da") + * @return Optional containing the CassandraVersion that natively writes this format, + * or Optional.empty() if: + *
    + *
  • The version string is null
  • + *
  • The version string is unrecognized (not in any enum's nativeSStableVersions)
  • + *
  • The version format is invalid or doesn't match expected pattern
  • + *
+ */ + public static Optional fromSSTableVersion(String sstableVersion) + { + if (sstableVersion == null) + { + return Optional.empty(); + } + + for (CassandraVersion version : CassandraVersion.values()) + { + if (version.nativeSStableVersions.contains(sstableVersion)) + { + return Optional.of(version); + } + } + + return Optional.empty(); + } + public static CassandraVersion[] implementedVersions() { return implementedVersions; diff --git a/cassandra-analytics-common/src/main/java/org/apache/cassandra/bridge/SSTableVersionAnalyzer.java b/cassandra-analytics-common/src/main/java/org/apache/cassandra/bridge/SSTableVersionAnalyzer.java new file mode 100644 index 000000000..55acbd2cc --- /dev/null +++ b/cassandra-analytics-common/src/main/java/org/apache/cassandra/bridge/SSTableVersionAnalyzer.java @@ -0,0 +1,233 @@ +/* + * 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.bridge; + +import java.util.Comparator; +import java.util.Optional; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Analyzes SSTable versions on a cluster to determine the appropriate + * Cassandra bridge to load for bulk write operations. + * + *

This class provides logic to select Cassandra bridge based on the highest SSTable + * version detected on the cluster and the user's requested format preference.

+ */ +public final class SSTableVersionAnalyzer +{ + private static final Logger LOGGER = LoggerFactory.getLogger(SSTableVersionAnalyzer.class); + + private SSTableVersionAnalyzer() + { + // Utility class + } + + /** + * Determines which CassandraVersion bridge to load based on: + * - Highest SSTable version detected on cluster + * - User's format preference + * + * @param sstableVersionsOnCluster Set of SSTable versions found on cluster nodes + * @param requestedFormat User's requested format, example: "big" or "bti" + * @param cassandraVersion Cassandra version string for fallback + * @param isSSTableVersionBasedBridgeDisabled flag to disable sstable version based bridge determination + * @return CassandraVersion enum indicating which bridge to load + * @throws IllegalArgumentException if format is invalid, or cluster doesn't support requested format, + * or SSTable versions are empty/unknown + */ + public static CassandraVersion determineBridgeVersionForWrite(Set sstableVersionsOnCluster, + String requestedFormat, + String cassandraVersion, + boolean isSSTableVersionBasedBridgeDisabled) + { + // Check for fallback mode + if (isSSTableVersionBasedBridgeDisabled) + { + LOGGER.info("SSTable version-based bridge selection is disabled via configuration. " + + "Using cassandra.version for bridge selection: {}", cassandraVersion); + return CassandraVersion.fromVersion(cassandraVersion) + .orElseThrow(() -> new UnsupportedOperationException( + String.format("Unsupported Cassandra version: %s", cassandraVersion))); + } + + // Validate SSTable versions are present + ensureSSTableVersionsNotEmpty(sstableVersionsOnCluster); + + // Find highest Cassandra version based on SSTable versions + CassandraVersion highestCassandraVersion = findHighestCassandraVersion(sstableVersionsOnCluster); + + // Check if highestCassandraVersion supports the requested format + boolean supportsRequestedFormat = highestCassandraVersion.getNativeSStableVersions() + .stream() + .anyMatch(v -> v.startsWith(requestedFormat + "-")); + + if (supportsRequestedFormat) + { + return highestCassandraVersion; + } + else + { + throw new IllegalArgumentException( + String.format("Cluster does not support requested SSTable format '%s'. " + + "Bridge version determined is %s, which only supports formats: %s", + requestedFormat, highestCassandraVersion.versionName(), + highestCassandraVersion.sstableFormats())); + } + } + + /** + * Determines which CassandraVersion bridge to load for read operations based on: + * - Highest SSTable version detected on cluster + * + * @param sstableVersionsOnCluster Set of SSTable versions found on cluster nodes + * @param cassandraVersion Cassandra version string for fallback + * @param isSSTableVersionBasedBridgeDisabled flag to disable sstable version based bridge determination + * @return CassandraVersion enum indicating which bridge to load + * @throws IllegalArgumentException if SSTable versions are empty/unknown + */ + public static CassandraVersion determineBridgeVersionForRead(Set sstableVersionsOnCluster, + String cassandraVersion, + boolean isSSTableVersionBasedBridgeDisabled) + { + // Check for fallback mode + if (isSSTableVersionBasedBridgeDisabled) + { + LOGGER.info("SSTable version-based bridge selection is disabled via configuration. " + + "Using cassandra.version for bridge selection: {}", cassandraVersion); + return CassandraVersion.fromVersion(cassandraVersion) + .orElseThrow(() -> new UnsupportedOperationException( + String.format("Unsupported Cassandra version: %s", cassandraVersion))); + } + + // Validate SSTable versions are present + ensureSSTableVersionsNotEmpty(sstableVersionsOnCluster); + + // Find highest Cassandra version based on SSTable versions + CassandraVersion bridgeVersion = findHighestCassandraVersion(sstableVersionsOnCluster); + + LOGGER.debug("Determined bridge version {} for read based on SSTable versions on cluster: {}", + bridgeVersion.versionName(), sstableVersionsOnCluster); + + return bridgeVersion; + } + + /** + * Ensures that SSTable versions from cluster are not null or empty. + * + * @param sstableVersionsOnCluster Set of SSTable versions to validate + * @throws IllegalStateException if versions are null or empty + */ + private static void ensureSSTableVersionsNotEmpty(Set sstableVersionsOnCluster) + { + if (sstableVersionsOnCluster == null || sstableVersionsOnCluster.isEmpty()) + { + throw new IllegalStateException(String.format( + "Unable to retrieve SSTable versions from cluster. " + + "This is required for SSTable version-based bridge selection. " + + "If you want to bypass this check and use cassandra.version for bridge selection, " + + "set %s=true", "spark.cassandra_analytics.bridge.disable_sstable_version_based")); + } + } + + /** + * Finds the highest Cassandra version based on SSTable versions found on cluster. + * + * @param sstableVersionsOnCluster Set of SSTable versions found on cluster + * @return CassandraVersion corresponding to the highest SSTable version + * @throws IllegalArgumentException if highest version is unknown + */ + private static CassandraVersion findHighestCassandraVersion(Set sstableVersionsOnCluster) + { + String highestSSTableVersion = findHighestSSTableVersion(sstableVersionsOnCluster); + return CassandraVersion.fromSSTableVersion(highestSSTableVersion) + .orElseThrow(() -> new IllegalArgumentException( + String.format("Unknown SSTable version: %s. Cannot determine bridge version. " + + "SSTable versions on cluster: %s", + highestSSTableVersion, sstableVersionsOnCluster))); + } + + /** + * Finds the highest SSTable version from the set using CassandraVersion mappings. + * Ordering is based on: + * 1. CassandraVersion number + * 2. Within same CassandraVersion, the order defined in nativeSStableVersions array + * + * @param versions Set of SSTable version strings + * @return Highest SSTable version string + * @throws IllegalArgumentException if versions is empty, contains null values, or contains unknown versions + */ + public static String findHighestSSTableVersion(Set versions) + { + if (versions == null || versions.isEmpty()) + { + throw new IllegalArgumentException("SSTable versions set cannot be empty"); + } + + Comparator sstableVersionComparator = (v1, v2) -> { + // Find which CassandraVersion each SSTable version belongs to + Optional v1Opt = CassandraVersion.fromSSTableVersion(v1); + Optional v2Opt = CassandraVersion.fromSSTableVersion(v2); + + if (!v1Opt.isPresent()) + { + throw new IllegalArgumentException( + String.format("Unknown SSTable version: %s. Cannot determine Cassandra version.", v1)); + } + + if (!v2Opt.isPresent()) + { + throw new IllegalArgumentException( + String.format("Unknown SSTable version: %s. Cannot determine Cassandra version.", v2)); + } + + CassandraVersion cv1 = v1Opt.get(); + CassandraVersion cv2 = v2Opt.get(); + + // First, compare by CassandraVersion number + // FIVEZERO (50) > FOURONE (41) > FOURZERO (40) > THREEZERO (30) + int versionComparison = Integer.compare(cv1.versionNumber(), cv2.versionNumber()); + if (versionComparison != 0) + { + return versionComparison; + } + + // Same CassandraVersion - use the index position in nativeSStableVersions + // The order in the enum definition determines precedence within a Cassandra version + int index1 = cv1.getSSTableVersionIndex(v1); + int index2 = cv2.getSSTableVersionIndex(v2); + + if (index1 == -1 || index2 == -1) + { + throw new IllegalArgumentException( + String.format("Version index not found for %s or %s in %s", + v1, v2, cv1.versionName())); + } + + return Integer.compare(index1, index2); + }; + + return versions.stream() + .max(sstableVersionComparator) + .orElseThrow(() -> new IllegalArgumentException("Unable to find highest SSTable version")); + } +} diff --git a/cassandra-analytics-common/src/test/java/org/apache/cassandra/bridge/CassandraVersionTest.java b/cassandra-analytics-common/src/test/java/org/apache/cassandra/bridge/CassandraVersionTest.java new file mode 100644 index 000000000..af5f05db0 --- /dev/null +++ b/cassandra-analytics-common/src/test/java/org/apache/cassandra/bridge/CassandraVersionTest.java @@ -0,0 +1,231 @@ +/* + * 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.bridge; + +import java.util.List; +import java.util.Optional; +import java.util.Set; + +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Tests for CassandraVersion SSTable version methods + */ +public class CassandraVersionTest +{ + @Test + void testFromSSTableVersionFourZeroNativeVersions() + { + Optional result = CassandraVersion.fromSSTableVersion("big-na"); + assertThat(result).isPresent(); + assertThat(result.get()).isEqualTo(CassandraVersion.FOURZERO); + + result = CassandraVersion.fromSSTableVersion("big-nb"); + assertThat(result).isPresent(); + assertThat(result.get()).isEqualTo(CassandraVersion.FOURZERO); + } + + @Test + void testFromSSTableVersionFiveZeroNativeVersions() + { + Optional result = CassandraVersion.fromSSTableVersion("big-oa"); + assertThat(result).isPresent(); + assertThat(result.get()).isEqualTo(CassandraVersion.FIVEZERO); + + result = CassandraVersion.fromSSTableVersion("bti-da"); + assertThat(result).isPresent(); + assertThat(result.get()).isEqualTo(CassandraVersion.FIVEZERO); + } + + @Test + void testFromSSTableVersionThreeZeroNativeVersions() + { + Optional result = CassandraVersion.fromSSTableVersion("big-ma"); + assertThat(result).isPresent(); + assertThat(result.get()).isEqualTo(CassandraVersion.THREEZERO); + } + + @Test + void testFromSSTableVersionUnknownVersion() + { + Optional result = CassandraVersion.fromSSTableVersion("unknown-xx"); + assertThat(result).isEmpty(); + } + + @Test + void testFromSSTableVersionNullVersion() + { + Optional result = CassandraVersion.fromSSTableVersion(null); + assertThat(result).isEmpty(); + } + + @Test + void testGetSupportedSStableVersionsForReadFourZero() + { + Set supported = CassandraVersion.FOURZERO.getSupportedSStableVersionsForRead(); + + // C* 4.0 can read its own versions + assertThat(supported).contains("big-na", "big-nb"); + + // C* 4.0 can read C* 3.0 versions (previous major version family) + assertThat(supported).contains("big-ma", "big-mb", "big-mc", "big-md", "big-me", "big-mf"); + + // C* 4.0 cannot read C* 5.0 versions + assertThat(supported).doesNotContain("big-oa", "bti-da"); + } + + @Test + void testGetSupportedSStableVersionsForReadFiveZero() + { + Set supported = CassandraVersion.FIVEZERO.getSupportedSStableVersionsForRead(); + + // C* 5.0 can read its own versions + assertThat(supported).contains("big-oa", "bti-da"); + + // C* 5.0 can read C* 4.0 and 4.1 versions (previous major version family) + assertThat(supported).contains("big-na", "big-nb"); + + // C* 5.0 cannot read C* 3.0 versions (not in previous major version family) + assertThat(supported).doesNotContain("big-ma", "big-mb", "big-mc", "big-md", "big-me", "big-mf"); + } + + @Test + void testGetSupportedSStableVersionsForReadThreeZero() + { + Set supported = CassandraVersion.THREEZERO.getSupportedSStableVersionsForRead(); + + // C* 3.0 can read its own versions + assertThat(supported).contains("big-ma", "big-mb", "big-mc", "big-md", "big-me", "big-mf"); + + // C* 3.0 cannot read C* 4.0+ versions + assertThat(supported).doesNotContain("big-na", "big-nb", "big-oa", "bti-da"); + + // C* 3.0's previous major version (2.x) is not defined, so only its own versions are readable + assertThat(supported).hasSize(6); // Only the 6 native 3.0 versions + } + + @Test + void testGetSupportedSStableVersionsForReadFourOne() + { + Set supported = CassandraVersion.FOURONE.getSupportedSStableVersionsForRead(); + + // C* 4.1 has no native SSTable versions of its own, but can read C* 4.0 and C* 3.0 versions + // C* 4.1 can read C* 4.0 versions (same major version family) + assertThat(supported).contains("big-na", "big-nb"); + + // C* 4.1 can read C* 3.0 versions (previous major version family) + assertThat(supported).contains("big-ma", "big-mb", "big-mc", "big-md", "big-me", "big-mf"); + + // C* 4.1 cannot read C* 5.0 versions + assertThat(supported).doesNotContain("big-oa", "bti-da"); + } + + @Test + void testGetNativeSStableVersionsFourZero() + { + List nativeVersions = CassandraVersion.FOURZERO.getNativeSStableVersions(); + assertThat(nativeVersions).containsExactlyInAnyOrder("big-na", "big-nb"); + } + + @Test + void testGetNativeSStableVersionsFiveZero() + { + List nativeVersions = CassandraVersion.FIVEZERO.getNativeSStableVersions(); + assertThat(nativeVersions).containsExactlyInAnyOrder("big-oa", "bti-da"); + } + + @Test + void testGetNativeSStableVersionsFourOneEmpty() + { + // C* 4.1 did not introduce new native SSTable versions + List nativeVersions = CassandraVersion.FOURONE.getNativeSStableVersions(); + assertThat(nativeVersions).isEmpty(); + } + + @Test + void testGetSSTableVersionIndexValidVersion() + { + int index = CassandraVersion.FOURZERO.getSSTableVersionIndex("big-na"); + assertThat(index).isEqualTo(0); + + index = CassandraVersion.FOURZERO.getSSTableVersionIndex("big-nb"); + assertThat(index).isEqualTo(1); + } + + @Test + void testGetSSTableVersionIndexInvalidVersion() + { + int index = CassandraVersion.FOURZERO.getSSTableVersionIndex("big-oa"); + assertThat(index).isEqualTo(-1); + } + + @Test + void testGetPreviousMajorVersionFiveZero() + { + int previous = CassandraVersion.FIVEZERO.getPreviousMajorVersion(); + assertThat(previous).isEqualTo(40); // C* 4.x + } + + @Test + void testGetPreviousMajorVersionFourOne() + { + int previous = CassandraVersion.FOURONE.getPreviousMajorVersion(); + assertThat(previous).isEqualTo(30); // C* 3.x + } + + @Test + void testGetPreviousMajorVersionFourZero() + { + int previous = CassandraVersion.FOURZERO.getPreviousMajorVersion(); + assertThat(previous).isEqualTo(30); // C* 3.x + } + + @Test + void testGetPreviousMajorVersionThreeZeroEmpty() + { + int previous = CassandraVersion.THREEZERO.getPreviousMajorVersion(); + assertThat(previous).isEqualTo(20); + } + + @Test + void testConfiguredSSTableFormatDefault() + { + // Assuming no system property set, should return "big" + String format = CassandraVersion.configuredSSTableFormat(); + assertThat(format).isNotNull(); + assertThat(format).isIn("big", "bti"); // Could be either depending on env + } + + @Test + void testSStableFormatsFourZero() + { + Set formats = CassandraVersion.FOURZERO.sstableFormats(); + assertThat(formats).containsExactly("big"); + } + + @Test + void testSStableFormatsFiveZero() + { + Set formats = CassandraVersion.FIVEZERO.sstableFormats(); + assertThat(formats).containsExactly("big", "bti"); + } +} diff --git a/cassandra-analytics-common/src/test/java/org/apache/cassandra/bridge/SSTableVersionAnalyzerTest.java b/cassandra-analytics-common/src/test/java/org/apache/cassandra/bridge/SSTableVersionAnalyzerTest.java new file mode 100644 index 000000000..a502c1f21 --- /dev/null +++ b/cassandra-analytics-common/src/test/java/org/apache/cassandra/bridge/SSTableVersionAnalyzerTest.java @@ -0,0 +1,219 @@ +/* + * 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.bridge; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** + * Unit tests for SSTableVersionAnalyzer + */ +public class SSTableVersionAnalyzerTest +{ + @Test + void testDetermineBridgeVersionForWriteFallbackDisabledSingleVersion() + { + Set sstableVersions = Collections.singleton("big-oa"); + CassandraVersion result = SSTableVersionAnalyzer.determineBridgeVersionForWrite( + sstableVersions, + "big", + "5.0.0", + false + ); + assertThat(result).isEqualTo(CassandraVersion.FIVEZERO); + } + + @Test + void testDetermineBridgeVersionForWriteFallbackDisabledMultipleVersions() + { + Set sstableVersions = new HashSet<>(Arrays.asList("big-na", "big-nb")); + CassandraVersion result = SSTableVersionAnalyzer.determineBridgeVersionForWrite( + sstableVersions, + "big", + "4.0.0", + false + ); + assertThat(result).isEqualTo(CassandraVersion.FOURZERO); + } + + @Test + void testDetermineBridgeVersionForWriteFallbackDisabledMixedVersions() + { + // Mix of 4.0 and 5.0 versions - should pick 5.0 (highest) + Set sstableVersions = new HashSet<>(Arrays.asList("big-na", "big-oa")); + CassandraVersion result = SSTableVersionAnalyzer.determineBridgeVersionForWrite( + sstableVersions, + "big", + "5.0.0", + false + ); + assertThat(result).isEqualTo(CassandraVersion.FIVEZERO); + } + + @Test + void testDetermineBridgeVersionForWriteFallbackEnabled() + { + CassandraVersion result = SSTableVersionAnalyzer.determineBridgeVersionForWrite( + null, + "big", + "5.0.0", + true + ); + assertThat(result).isEqualTo(CassandraVersion.FIVEZERO); + } + + @Test + void testDetermineBridgeVersionForWriteEmptyVersionsThrowsException() + { + assertThatThrownBy(() -> SSTableVersionAnalyzer.determineBridgeVersionForWrite( + Collections.emptySet(), + "big", + "5.0.0", + false + )).isInstanceOf(IllegalStateException.class) + .hasMessageContaining("Unable to retrieve SSTable versions from cluster"); + } + + @Test + void testDetermineBridgeVersionForWriteNullVersionsThrowsException() + { + assertThatThrownBy(() -> SSTableVersionAnalyzer.determineBridgeVersionForWrite( + null, + "big", + "5.0.0", + false + )).isInstanceOf(IllegalStateException.class) + .hasMessageContaining("Unable to retrieve SSTable versions from cluster"); + } + + @Test + void testDetermineBridgeVersionForWriteUnsupportedFormat() + { + // C* 4.0 only supports "big" format, not "bti" + Set sstableVersions = Collections.singleton("big-na"); + assertThatThrownBy(() -> SSTableVersionAnalyzer.determineBridgeVersionForWrite( + sstableVersions, + "bti", // C* 4.0 doesn't support bti format + "4.0.0", + false + )).isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cluster does not support requested SSTable format 'bti'"); + } + + @Test + void testDetermineBridgeVersionForReadFallbackDisabled() + { + Set sstableVersions = Collections.singleton("big-oa"); + CassandraVersion result = SSTableVersionAnalyzer.determineBridgeVersionForRead( + sstableVersions, + "5.0.0", + false + ); + assertThat(result).isEqualTo(CassandraVersion.FIVEZERO); + } + + @Test + void testDetermineBridgeVersionForReadFallbackEnabled() + { + CassandraVersion result = SSTableVersionAnalyzer.determineBridgeVersionForRead( + null, + "4.0.0", + true + ); + assertThat(result).isEqualTo(CassandraVersion.FOURZERO); + } + + @Test + void testDetermineBridgeVersionForReadEmptyVersionsThrowsException() + { + assertThatThrownBy(() -> SSTableVersionAnalyzer.determineBridgeVersionForRead( + Collections.emptySet(), + "5.0.0", + false + )).isInstanceOf(IllegalStateException.class) + .hasMessageContaining("Unable to retrieve SSTable versions from cluster"); + } + + @Test + void testFindHighestSSTableVersionSingleVersion() + { + Set versions = Collections.singleton("big-na"); + String result = SSTableVersionAnalyzer.findHighestSSTableVersion(versions); + assertThat(result).isEqualTo("big-na"); + } + + @Test + void testFindHighestSSTableVersionMultipleVersionsSameCassandra() + { + Set versions = new HashSet<>(Arrays.asList("big-na", "big-nb")); + String result = SSTableVersionAnalyzer.findHighestSSTableVersion(versions); + assertThat(result).isEqualTo("big-nb"); // nb is later than na + } + + @Test + void testFindHighestSSTableVersionDifferentCassandraVersions() + { + Set versions = new HashSet<>(Arrays.asList("big-na", "big-oa")); + String result = SSTableVersionAnalyzer.findHighestSSTableVersion(versions); + assertThat(result).isEqualTo("big-oa"); // C* 5.0 > C* 4.0 + } + + @Test + void testFindHighestSSTableVersionDifferentFormats() + { + Set versions = new HashSet<>(Arrays.asList("big-oa", "bti-da")); + String result = SSTableVersionAnalyzer.findHighestSSTableVersion(versions); + // Both from C* 5.0, bti-da comes first in array so big-oa wins (index 0 vs 1) + assertThat(result).isEqualTo("bti-da"); + } + + @Test + void testFindHighestSSTableVersionEmptySetThrowsException() + { + assertThatThrownBy(() -> SSTableVersionAnalyzer.findHighestSSTableVersion(Collections.emptySet())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("SSTable versions set cannot be empty"); + } + + @Test + void testFindHighestSSTableVersionNullSetThrowsException() + { + assertThatThrownBy(() -> SSTableVersionAnalyzer.findHighestSSTableVersion(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("SSTable versions set cannot be empty"); + } + + @Test + void testFindHighestSSTableVersionUnknownVersionThrowsException() + { + // Use two unknown versions to trigger the comparator + Set versions = new HashSet<>(Arrays.asList("unknown-xx", "unknown-yy")); + assertThatThrownBy(() -> SSTableVersionAnalyzer.findHighestSSTableVersion(versions)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Unknown SSTable version:"); + } +} 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..054de0c7d 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 @@ -83,6 +83,44 @@ public static void addSerializer(@NotNull Class type, @NotNull Serializer KRYO_SERIALIZERS.put(type, serializer); } + /** + * Validates that a Kryo registrator exists for the given Cassandra version. + * This should be called after bridge version determination to ensure that + * Spark can properly serialize objects for the selected bridge. + * + * @param bridgeVersion the CassandraVersion to validate + * @param clusterCassandraVersion optional Cassandra version string for additional context in error messages + * @throws IllegalStateException if no Kryo registrator is configured for this version + */ + public static void validateKryoRegistratorExists(@NotNull CassandraVersion bridgeVersion, + String clusterCassandraVersion) + { + Class registratorClass = KRYO_REGISTRATORS.get(bridgeVersion); + if (registratorClass == null) + { + // Build available versions list for suggestion + String availableKryoVersions = KRYO_REGISTRATORS.keySet().stream() + .map(v -> v.name() + " (" + v.versionName() + ")") + .collect(Collectors.joining(", ")); + + throw new IllegalStateException( + String.format("No Kryo registrator configured for bridge version %s (%s). " + + "Cluster Cassandra version: %s. " + + "Available Kryo registrators: %s. " + + "Cannot proceed with job as Spark serialization will fail. " + + "To resolve this issue, update the '%s' configuration property to match the bridge version: %s", + bridgeVersion.name(), + bridgeVersion.versionName(), + clusterCassandraVersion, + availableKryoVersions, + CASSANDRA_VERSION, + bridgeVersion.versionName())); + } + + LOGGER.debug("Validated Kryo registrator exists for bridge version {}: {}", + bridgeVersion.versionName(), registratorClass.getName()); + } + private final CassandraVersion cassandraVersion; protected KryoRegister(CassandraVersion cassandraVersion) diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/AbstractBulkWriterContext.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/AbstractBulkWriterContext.java index 6acbb2fcc..367ee0e95 100644 --- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/AbstractBulkWriterContext.java +++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/AbstractBulkWriterContext.java @@ -32,6 +32,9 @@ import com.esotericsoftware.kryo.io.Output; import org.apache.cassandra.bridge.CassandraBridge; import org.apache.cassandra.bridge.CassandraBridgeFactory; +import org.apache.cassandra.bridge.CassandraVersion; +import org.apache.cassandra.bridge.SSTableVersionAnalyzer; +import org.apache.cassandra.spark.KryoRegister; import org.apache.cassandra.spark.bulkwriter.cloudstorage.coordinated.CassandraClusterInfoGroup; import org.apache.cassandra.spark.bulkwriter.cloudstorage.coordinated.MultiClusterContainer; import org.apache.cassandra.spark.bulkwriter.token.TokenRangeMapping; @@ -76,7 +79,7 @@ public abstract class AbstractBulkWriterContext implements BulkWriterContext, Kr private final JobInfo jobInfo; private final ClusterInfo clusterInfo; private final SchemaInfo schemaInfo; - private final String lowestCassandraVersion; + private final CassandraVersion bridgeVersion; // Note: do not declare transient fields as final; but they need to be volatile as there could be contention when recreating them after deserialization // For the transient field, they are assigned null once deserialized, remember to use getOrRebuildAfterDeserialization for their getters private transient volatile CassandraBridge bridge; @@ -93,15 +96,42 @@ public abstract class AbstractBulkWriterContext implements BulkWriterContext, Kr */ protected AbstractBulkWriterContext(@NotNull BulkSparkConf conf, @NotNull StructType structType, - @NotNull int sparkDefaultParallelism) + int sparkDefaultParallelism) { this.conf = conf; this.sparkDefaultParallelism = sparkDefaultParallelism; - // Build everything fresh on driver - this.clusterInfo = buildClusterInfo(); + // Retrieve lowest Cassandra version without building full ClusterInfo + String lowestCassandraVersion = getLowestCassandraVersion(conf); + Set sstableVersionsOnCluster = null; + + // Get SSTable versions from cluster only if SSTable version-based selection is enabled + // If disabled, skip retrieval to allow job to proceed even when SSTable version detection fails + if (!conf.isSSTableVersionBasedBridgeDisabled()) + { + sstableVersionsOnCluster = getSSTableVersionsOnCluster(conf); + } + + // Determine bridge version + this.bridgeVersion = SSTableVersionAnalyzer.determineBridgeVersionForWrite( + sstableVersionsOnCluster, + CassandraVersion.configuredSSTableFormat(), + lowestCassandraVersion, + conf.isSSTableVersionBasedBridgeDisabled() + ); + + logger.info("Selected bridge version: {}, lowest Cassandra version: {}, SSTable versions: {}", + this.bridgeVersion.versionName(), + lowestCassandraVersion, + sstableVersionsOnCluster); + + // Validate that Kryo registrator exists for this bridge version + KryoRegister.validateKryoRegistratorExists(this.bridgeVersion, lowestCassandraVersion); + + // Build cluster info with determined bridge version + this.clusterInfo = buildClusterInfo(this.bridgeVersion); this.clusterInfo.startupValidate(); - this.lowestCassandraVersion = findLowestCassandraVersion(); + this.bridge = buildCassandraBridge(); this.jobInfo = buildJobInfo(); this.schemaInfo = buildSchemaInfo(structType); @@ -121,9 +151,12 @@ protected AbstractBulkWriterContext(@NotNull BulkWriterConfig config) this.conf = config.getConf(); this.sparkDefaultParallelism = config.getSparkDefaultParallelism(); - // Reconstruct from broadcast data on executor - this.clusterInfo = reconstructClusterInfoOnExecutor(config.getBroadcastableClusterInfo()); - this.lowestCassandraVersion = config.getLowestCassandraVersion(); + // Get bridge version from broadcast config + this.bridgeVersion = config.getBridgeVersion(); + + // Reconstruct from broadcast data on executor with bridge version + this.clusterInfo = reconstructClusterInfoOnExecutor(config.getBroadcastableClusterInfo(), this.bridgeVersion); + this.bridge = buildCassandraBridge(); this.jobInfo = reconstructJobInfoOnExecutor(config.getBroadcastableJobInfo()); this.schemaInfo = reconstructSchemaInfoOnExecutor(config.getBroadcastableSchemaInfo()); @@ -141,14 +174,44 @@ protected final int sparkDefaultParallelism() return sparkDefaultParallelism; } - protected String lowestCassandraVersion() + @Override + public CassandraVersion bridgeVersion() { - return lowestCassandraVersion; + if (bridgeVersion == null) + { + throw new IllegalStateException( + "Bridge version must be determined before accessing it. " + + "Ensure SSTable versions are retrieved from cluster and bridge version is set."); + } + + return bridgeVersion; } /*--- Methods to build required fields ---*/ - protected abstract ClusterInfo buildClusterInfo(); + /** + * Retrieves the lowest Cassandra version from the cluster(s). + * + * @param conf Bulk Spark configuration + * @return lowest Cassandra version string + */ + protected abstract String getLowestCassandraVersion(@NotNull BulkSparkConf conf); + + /** + * Retrieves SSTable versions from the cluster(s). + * + * @param conf Bulk Spark configuration + * @return set of SSTable version strings present on the cluster(s) + */ + protected abstract Set getSSTableVersionsOnCluster(@NotNull BulkSparkConf conf); + + /** + * Builds the ClusterInfo with the determined bridge version. + * + * @param bridgeVersion the determined Cassandra bridge version + * @return ClusterInfo instance with bridge version set + */ + protected abstract ClusterInfo buildClusterInfo(CassandraVersion bridgeVersion); /** * Reconstructs ClusterInfo on executors from broadcastable versions. @@ -157,11 +220,13 @@ protected String lowestCassandraVersion() * into the appropriate full ClusterInfo implementation. * * @param clusterInfo the BroadcastableClusterInfo from broadcast + * @param bridgeVersion the bridge version from broadcast * @return reconstructed ClusterInfo (CassandraClusterInfo or CassandraClusterInfoGroup) */ - protected ClusterInfo reconstructClusterInfoOnExecutor(IBroadcastableClusterInfo clusterInfo) + protected ClusterInfo reconstructClusterInfoOnExecutor(IBroadcastableClusterInfo clusterInfo, + CassandraVersion bridgeVersion) { - return clusterInfo.reconstruct(); + return clusterInfo.reconstruct(bridgeVersion); } /** @@ -216,7 +281,7 @@ protected JobInfo buildJobInfo() protected CassandraBridge buildCassandraBridge() { - return CassandraBridgeFactory.get(lowestCassandraVersion()); + return CassandraBridgeFactory.get(bridgeVersion()); } protected TransportContext buildTransportContext(boolean isOnDriver) @@ -229,11 +294,6 @@ protected JobStatsPublisher buildJobStatsPublisher() return new LogStatsPublisher(); } - protected String findLowestCassandraVersion() - { - return cluster().getLowestCassandraVersion(); - } - protected SchemaInfo buildSchemaInfo(StructType structType) { QualifiedTableName tableName = job().qualifiedTableName(); @@ -248,7 +308,7 @@ protected SchemaInfo buildSchemaInfo(StructType structType) CqlTable cqlTable = bridge().buildSchema(createTableSchema, keyspace, replicationFactor, partitioner, udts, null, indexCount, false); TableInfoProvider tableInfoProvider = new CqlTableInfoProvider(createTableSchema, cqlTable); - TableSchema tableSchema = initializeTableSchema(bulkSparkConf(), structType, tableInfoProvider, lowestCassandraVersion()); + TableSchema tableSchema = initializeTableSchema(bulkSparkConf(), structType, tableInfoProvider, bridgeVersion()); return new CassandraSchemaInfo(tableSchema, udts); } @@ -313,14 +373,14 @@ public void shutdown() protected TableSchema initializeTableSchema(@NotNull BulkSparkConf conf, @NotNull StructType dfSchema, TableInfoProvider tableInfoProvider, - String lowestCassandraVersion) + CassandraVersion bridgeVersion) { return new TableSchema(dfSchema, tableInfoProvider, conf.writeMode, conf.getTTLOptions(), conf.getTimestampOptions(), - lowestCassandraVersion, + bridgeVersion, job().qualifiedTableName().quoteIdentifiers()); } diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BroadcastableClusterInfo.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BroadcastableClusterInfo.java index de06151ef..0b52a1927 100644 --- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BroadcastableClusterInfo.java +++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BroadcastableClusterInfo.java @@ -19,6 +19,7 @@ package org.apache.cassandra.spark.bulkwriter; +import org.apache.cassandra.bridge.CassandraVersion; import org.apache.cassandra.spark.data.partitioner.Partitioner; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -45,7 +46,6 @@ public final class BroadcastableClusterInfo implements IBroadcastableClusterInfo // Essential fields broadcast to executors private final Partitioner partitioner; - private final String cassandraVersion; private final String clusterId; private final BulkSparkConf conf; @@ -58,31 +58,26 @@ public final class BroadcastableClusterInfo implements IBroadcastableClusterInfo */ public static BroadcastableClusterInfo from(@NotNull ClusterInfo source, @NotNull BulkSparkConf conf) { - return new BroadcastableClusterInfo(source.getPartitioner(), source.getLowestCassandraVersion(), source.clusterId(), conf); + return new BroadcastableClusterInfo(source.getPartitioner(), + source.clusterId(), + conf); } private BroadcastableClusterInfo(@NotNull Partitioner partitioner, - @NotNull String cassandraVersion, @Nullable String clusterId, @NotNull BulkSparkConf conf) { this.partitioner = partitioner; - this.cassandraVersion = cassandraVersion; this.clusterId = clusterId; this.conf = conf; } + @NotNull public BulkSparkConf getConf() { return conf; } - @Override - public String getLowestCassandraVersion() - { - return cassandraVersion; - } - @Override public Partitioner getPartitioner() { @@ -97,8 +92,8 @@ public String clusterId() } @Override - public ClusterInfo reconstruct() + public ClusterInfo reconstruct(CassandraVersion bridgeVersion) { - return new CassandraClusterInfo(this); + return new CassandraClusterInfo(this, bridgeVersion); } } diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BroadcastableClusterInfoGroup.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BroadcastableClusterInfoGroup.java index d5ad1e13d..9cba76fca 100644 --- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BroadcastableClusterInfoGroup.java +++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BroadcastableClusterInfoGroup.java @@ -24,6 +24,7 @@ import java.util.List; import java.util.function.BiConsumer; +import org.apache.cassandra.bridge.CassandraVersion; import org.apache.cassandra.spark.bulkwriter.cloudstorage.coordinated.CassandraClusterInfoGroup; import org.apache.cassandra.spark.bulkwriter.cloudstorage.coordinated.MultiClusterSupport; import org.apache.cassandra.spark.data.partitioner.Partitioner; @@ -34,7 +35,7 @@ * Broadcastable wrapper for coordinated writes with ZERO transient fields to optimize Spark broadcasting. *

* This class wraps multiple BroadcastableCluster instances for multi-cluster scenarios. - * Pre-computed values (partitioner, lowestCassandraVersion) are extracted from CassandraClusterInfoGroup on the driver + * Pre-computed values (partitioner) are extracted from CassandraClusterInfoGroup on the driver * to avoid duplicating aggregation/validation logic on executors. *

* Why ZERO transient fields matters:
@@ -56,12 +57,10 @@ public final class BroadcastableClusterInfoGroup implements IBroadcastableCluste private final String clusterId; private final BulkSparkConf conf; private final Partitioner partitioner; - private final String lowestCassandraVersion; /** * Creates a BroadcastableClusterInfoGroup from a source ClusterInfo group. - * Extracts pre-computed values (partitioner, lowestCassandraVersion) from the source - * to avoid duplicating aggregation/validation logic on executors. + * Extracts pre-computed partitioner from the source to avoid duplicating validation logic on executors. * * @param source the source CassandraClusterInfoGroup * @param conf the BulkSparkConf needed to connect to Sidecar on executors @@ -72,25 +71,22 @@ public static BroadcastableClusterInfoGroup from(@NotNull CassandraClusterInfoGr List broadcastableInfos = new ArrayList<>(); source.forEach((clusterId, clusterInfo) -> broadcastableInfos.add(BroadcastableClusterInfo.from(clusterInfo, conf))); - // Extract pre-computed values from CassandraClusterInfoGroup + // Extract pre-computed value from CassandraClusterInfoGroup // These have already been validated/computed on the driver Partitioner partitioner = source.getPartitioner(); - String lowestVersion = source.getLowestCassandraVersion(); - return new BroadcastableClusterInfoGroup(broadcastableInfos, source.clusterId(), conf, partitioner, lowestVersion); + return new BroadcastableClusterInfoGroup(broadcastableInfos, source.clusterId(), conf, partitioner); } private BroadcastableClusterInfoGroup(List clusterInfos, String clusterId, BulkSparkConf conf, - Partitioner partitioner, - String lowestCassandraVersion) + Partitioner partitioner) { this.clusterInfos = Collections.unmodifiableList(clusterInfos); this.conf = conf; this.clusterId = clusterId; this.partitioner = partitioner; - this.lowestCassandraVersion = lowestCassandraVersion; } @Override @@ -100,14 +96,6 @@ public BulkSparkConf getConf() return conf; } - @Override - public String getLowestCassandraVersion() - { - // Return pre-computed value from CassandraClusterInfoGroup - // No need to duplicate aggregation/validation logic - return lowestCassandraVersion; - } - @Override public Partitioner getPartitioner() { @@ -143,8 +131,8 @@ public IBroadcastableClusterInfo getValueOrNull(@NotNull String clusterId) } @Override - public ClusterInfo reconstruct() + public ClusterInfo reconstruct(CassandraVersion bridgeVersion) { - return CassandraClusterInfoGroup.from(this); + return CassandraClusterInfoGroup.from(this, bridgeVersion); } } diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BroadcastableTableSchema.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BroadcastableTableSchema.java index 322e815dd..bdcd91d43 100644 --- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BroadcastableTableSchema.java +++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BroadcastableTableSchema.java @@ -24,6 +24,7 @@ import com.google.common.base.Preconditions; +import org.apache.cassandra.bridge.CassandraVersion; import org.apache.cassandra.spark.common.schema.ColumnType; import org.jetbrains.annotations.NotNull; @@ -58,7 +59,7 @@ public final class BroadcastableTableSchema implements Serializable private final WriteMode writeMode; private final TTLOption ttlOption; private final TimestampOption timestampOption; - private final String lowestCassandraVersion; + private final CassandraVersion bridgeVersion; private final boolean quoteIdentifiers; /** @@ -80,7 +81,7 @@ public static BroadcastableTableSchema from(@NotNull TableSchema source) source.writeMode, source.ttlOption, source.timestampOption, - source.lowestCassandraVersion, + source.bridgeVersion, source.quoteIdentifiers ); } @@ -94,7 +95,7 @@ private BroadcastableTableSchema(String createStatement, WriteMode writeMode, TTLOption ttlOption, TimestampOption timestampOption, - String lowestCassandraVersion, + CassandraVersion bridgeVersion, boolean quoteIdentifiers) { this.createStatement = createStatement; @@ -106,7 +107,7 @@ private BroadcastableTableSchema(String createStatement, this.writeMode = writeMode; this.ttlOption = ttlOption; this.timestampOption = timestampOption; - this.lowestCassandraVersion = lowestCassandraVersion; + this.bridgeVersion = bridgeVersion; this.quoteIdentifiers = quoteIdentifiers; } @@ -155,9 +156,9 @@ public TimestampOption getTimestampOption() return timestampOption; } - public String getLowestCassandraVersion() + public CassandraVersion getBridgeVersion() { - return lowestCassandraVersion; + return bridgeVersion; } public boolean isQuoteIdentifiers() diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BulkSparkConf.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BulkSparkConf.java index 3f52cd74d..5cea6c379 100644 --- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BulkSparkConf.java +++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BulkSparkConf.java @@ -100,6 +100,9 @@ public class BulkSparkConf implements Serializable // Cassandra version of target cluster. Configuration parameter is exposed to be able to correctly initialize static // components, before cluster version is discovered via Sidecar. public static final String CASSANDRA_VERSION = SETTING_PREFIX + "cassandra.version"; + // Disable SSTable version-based bridge determination. When true, falls back to using cassandra.version for bridge selection. + // This provides a safety fallback mechanism if SSTable version detection fails or encounters issues. + public static final String DISABLE_SSTABLE_VERSION_BASED_BRIDGE = SETTING_PREFIX + "bridge.disable_sstable_version_based"; public static final String HTTP_MAX_CONNECTIONS = SETTING_PREFIX + "request.max_connections"; public static final String HTTP_RESPONSE_TIMEOUT = SETTING_PREFIX + "request.response_timeout"; public static final String HTTP_CONNECTION_TIMEOUT = SETTING_PREFIX + "request.connection_timeout"; @@ -151,6 +154,7 @@ public class BulkSparkConf implements Serializable protected final String configuredJobId; protected boolean useOpenSsl; protected int ringRetryCount; + protected final boolean disableSSTableVersionBasedBridge; // create sidecarInstances from sidecarContactPointsValue and effectiveSidecarPort private final String sidecarContactPointsValue; // It takes comma separated values private transient Set sidecarContactPoints; // not serialized @@ -203,6 +207,7 @@ public BulkSparkConf(SparkConf conf, Map options, @Nullable Logg // else fall back to props, and then default if neither specified this.useOpenSsl = getBoolean(USE_OPENSSL, true); this.ringRetryCount = getInt(RING_RETRY_COUNT, DEFAULT_RING_RETRY_COUNT); + this.disableSSTableVersionBasedBridge = getBoolean(DISABLE_SSTABLE_VERSION_BASED_BRIDGE, false); this.importCoordinatorTimeoutMultiplier = getDouble(IMPORT_COORDINATOR_TIMEOUT_MULTIPLIER, 0.5); this.ttl = MapUtils.getOrDefault(options, WriterOptions.TTL.name(), null); this.timestamp = MapUtils.getOrDefault(options, WriterOptions.TIMESTAMP.name(), null); @@ -694,6 +699,25 @@ public int getRingRetryCount() return ringRetryCount; } + public boolean isSSTableVersionBasedBridgeDisabled() + { + return disableSSTableVersionBasedBridge; + } + + /** + * Utility method to retrieve the disable SSTable version-based bridge flag + * from Spark configuration. This can be called from contexts where a BulkSparkConf + * instance is not available. + * + * @return true if SSTable version-based bridge selection should be disabled + */ + public static boolean getDisableSSTableVersionBasedBridge() + { + return org.apache.spark.SparkContext.getOrCreate() + .getConf() + .getBoolean(DISABLE_SSTABLE_VERSION_BASED_BRIDGE, false); + } + public StorageClientConfig getStorageClientConfig() { return storageClientConfig; diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BulkWriterConfig.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BulkWriterConfig.java index e2a3283aa..d67f370a5 100644 --- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BulkWriterConfig.java +++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BulkWriterConfig.java @@ -21,6 +21,7 @@ import java.io.Serializable; +import org.apache.cassandra.bridge.CassandraVersion; import org.jetbrains.annotations.NotNull; /** @@ -55,31 +56,32 @@ public final class BulkWriterConfig implements Serializable // BroadcastableClusterInfo can be either BroadcastableCluster or BroadcastableClusterInfoGroup private final IBroadcastableClusterInfo clusterInfo; private final BroadcastableSchemaInfo schemaInfo; - private final String lowestCassandraVersion; + // SSTable version-based bridge loading fields + private final CassandraVersion bridgeVersion; /** * Creates a new immutable BulkWriterConfig with pre-computed values * - * @param conf Bulk writer Spark configuration - * @param sparkDefaultParallelism Spark default parallelism setting - * @param jobInfo Broadcastable job information - * @param clusterInfo Broadcastable cluster information (BroadcastableCluster or BroadcastableClusterInfoGroup) - * @param schemaInfo Broadcastable schema information - * @param lowestCassandraVersion Lowest Cassandra version in the cluster + * @param conf Bulk writer Spark configuration + * @param sparkDefaultParallelism Spark default parallelism setting + * @param jobInfo Broadcastable job information + * @param clusterInfo Broadcastable cluster information (BroadcastableCluster or BroadcastableClusterInfoGroup) + * @param schemaInfo Broadcastable schema information + * @param bridgeVersion Cassandra bridge version to use */ public BulkWriterConfig(@NotNull BulkSparkConf conf, int sparkDefaultParallelism, @NotNull BroadcastableJobInfo jobInfo, @NotNull IBroadcastableClusterInfo clusterInfo, @NotNull BroadcastableSchemaInfo schemaInfo, - @NotNull String lowestCassandraVersion) + @NotNull CassandraVersion bridgeVersion) { this.conf = conf; this.sparkDefaultParallelism = sparkDefaultParallelism; this.jobInfo = jobInfo; this.clusterInfo = clusterInfo; this.schemaInfo = schemaInfo; - this.lowestCassandraVersion = lowestCassandraVersion; + this.bridgeVersion = bridgeVersion; } public BulkSparkConf getConf() @@ -107,8 +109,8 @@ public BroadcastableSchemaInfo getBroadcastableSchemaInfo() return schemaInfo; } - public String getLowestCassandraVersion() + public CassandraVersion getBridgeVersion() { - return lowestCassandraVersion; + return bridgeVersion; } } diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BulkWriterContext.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BulkWriterContext.java index a76b7c60f..e3554eb95 100644 --- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BulkWriterContext.java +++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BulkWriterContext.java @@ -22,6 +22,7 @@ import org.apache.cassandra.spark.bulkwriter.cloudstorage.coordinated.CassandraCoordinatedBulkWriterContext; import org.apache.cassandra.spark.common.stats.JobStatsPublisher; import org.apache.cassandra.bridge.CassandraBridge; +import org.apache.cassandra.bridge.CassandraVersion; /** * Context for bulk write operations, providing access to cluster, job, schema, and transport information. @@ -46,6 +47,8 @@ public interface BulkWriterContext CassandraBridge bridge(); + CassandraVersion bridgeVersion(); + // NOTE: This interface intentionally does *not* implement AutoClosable as Spark can close Broadcast variables // that implement AutoClosable while they are still in use, causing the underlying object to become unusable void shutdown(); diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraBulkSourceRelation.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraBulkSourceRelation.java index f6323af36..132944eb5 100644 --- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraBulkSourceRelation.java +++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraBulkSourceRelation.java @@ -147,7 +147,7 @@ private static BulkWriterConfig extractConfig(BulkWriterContext context, int spa broadcastableJobInfo, broadcastableClusterInfo, broadcastableSchemaInfo, - abstractContext.lowestCassandraVersion() + abstractContext.bridgeVersion() ); } throw new IllegalArgumentException("Cannot extract config from context type: " + context.getClass().getName()); diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraBulkWriterContext.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraBulkWriterContext.java index 5c5168374..f2bd7b3c7 100644 --- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraBulkWriterContext.java +++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraBulkWriterContext.java @@ -19,11 +19,13 @@ package org.apache.cassandra.spark.bulkwriter; +import java.util.Set; import java.util.UUID; import com.google.common.base.Preconditions; import org.apache.commons.lang3.StringUtils; +import org.apache.cassandra.bridge.CassandraVersion; import org.apache.cassandra.spark.bulkwriter.cloudstorage.coordinated.MultiClusterContainer; import org.apache.spark.sql.types.StructType; import org.jetbrains.annotations.NotNull; @@ -56,9 +58,21 @@ protected CassandraBulkWriterContext(@NotNull BulkWriterConfig config) } @Override - protected ClusterInfo buildClusterInfo() + protected String getLowestCassandraVersion(@NotNull BulkSparkConf conf) { - return new CassandraClusterInfo(bulkSparkConf()); + return CassandraClusterInfo.getLowestCassandraVersion(conf, null); + } + + @Override + protected Set getSSTableVersionsOnCluster(@NotNull BulkSparkConf conf) + { + return CassandraClusterInfo.getSSTableVersionsOnCluster(conf, null); + } + + @Override + protected ClusterInfo buildClusterInfo(CassandraVersion bridgeVersion) + { + return new CassandraClusterInfo(bulkSparkConf(), bridgeVersion); } @Override diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraClusterInfo.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraClusterInfo.java index ad143fb0c..f9cd41b88 100644 --- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraClusterInfo.java +++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraClusterInfo.java @@ -47,6 +47,7 @@ import o.a.c.sidecar.client.shaded.common.response.TokenRangeReplicasResponse; import org.apache.cassandra.bridge.CassandraBridge; import org.apache.cassandra.bridge.CassandraBridgeFactory; +import org.apache.cassandra.bridge.CassandraVersion; import org.apache.cassandra.bridge.CassandraVersionFeatures; import org.apache.cassandra.clients.Sidecar; import o.a.c.sidecar.client.shaded.client.SidecarInstance; @@ -80,26 +81,33 @@ public class CassandraClusterInfo implements ClusterInfo, Closeable protected final BulkSparkConf conf; protected final String clusterId; - protected String cassandraVersion; protected Partitioner partitioner; protected volatile TokenRangeMapping tokenRangeReplicas; protected volatile String keyspaceSchema; protected volatile ReplicationFactor replicationFactor; protected volatile CassandraContext cassandraContext; + protected volatile CassandraVersion bridgeVersion; protected final AtomicReference nodeSettings; protected final List> allNodeSettingFutures; - public CassandraClusterInfo(BulkSparkConf conf) + public CassandraClusterInfo(BulkSparkConf conf, CassandraVersion bridgeVersion) { - this(conf, null); + this(conf, null, bridgeVersion); } - // Used by CassandraClusterInfoGroup - public CassandraClusterInfo(BulkSparkConf conf, String clusterId) + /** + * Constructor with bridge version for driver-side usage. + * + * @param conf Bulk Spark configuration + * @param clusterId Optional cluster identifier + * @param bridgeVersion Determined bridge version (nullable for preliminary construction) + */ + public CassandraClusterInfo(BulkSparkConf conf, String clusterId, CassandraVersion bridgeVersion) { this.conf = conf; this.clusterId = clusterId; + this.bridgeVersion = bridgeVersion; this.cassandraContext = buildCassandraContext(); LOGGER.info("Getting Cassandra versions from all nodes"); this.nodeSettings = new AtomicReference<>(null); @@ -109,19 +117,21 @@ public CassandraClusterInfo(BulkSparkConf conf, String clusterId) /** * Reconstruct from BroadcastableCluster on executor. - * Reuses cassandraVersion and partitioner from broadcast, + * Reuses partitioner and bridge version from broadcast, * fetches other data (tokenRangeMapping, replicationFactor, keyspaceSchema, writeAvailability) fresh from Sidecar. * * @param broadcastable the broadcastable cluster info from broadcast + * @param bridgeVersion the bridge version from broadcast */ - public CassandraClusterInfo(BroadcastableClusterInfo broadcastable) + public CassandraClusterInfo(BroadcastableClusterInfo broadcastable, CassandraVersion bridgeVersion) { this.conf = broadcastable.getConf(); this.clusterId = broadcastable.clusterId(); - this.cassandraVersion = broadcastable.getLowestCassandraVersion(); this.partitioner = broadcastable.getPartitioner(); + this.bridgeVersion = bridgeVersion; this.cassandraContext = buildCassandraContext(); - LOGGER.info("Reconstructing CassandraClusterInfo on executor from BroadcastableCluster. clusterId={}", clusterId); + LOGGER.info("Reconstructing CassandraClusterInfo on executor from BroadcastableCluster. clusterId={}, bridgeVersion={}", + clusterId, bridgeVersion != null ? bridgeVersion.versionName() : "null"); this.nodeSettings = new AtomicReference<>(null); // Executors do not need to query all node settings since cassandraVersion is already set from broadcast this.allNodeSettingFutures = null; @@ -382,33 +392,6 @@ public TokenRangeMapping getTokenRangeMapping(boolean cached) } } - @Override - public String getLowestCassandraVersion() - { - String currentCassandraVersion = cassandraVersion; - if (currentCassandraVersion != null) - { - return currentCassandraVersion; - } - - synchronized (this) - { - if (cassandraVersion == null) - { - String versionFromFeature = getVersionFromFeature(); - if (versionFromFeature != null) - { - // Forcing writer to use a particular version - cassandraVersion = versionFromFeature; - } - else - { - cassandraVersion = getVersionFromSidecar(); - } - } - } - return cassandraVersion; - } @Override public Map clusterWriteAvailability() @@ -439,77 +422,85 @@ private TokenRangeMapping getTokenRangeReplicasFromSidecar() metadata -> new RingInstance(metadata, clusterId())); } - public String getVersionFromFeature() - { - return null; - } - - protected List getAllNodeSettings() + /** + * Retrieves the lowest Cassandra version from a cluster. + * + * @param conf Bulk Spark configuration + * @param clusterId Optional cluster identifier + * @return lowest Cassandra version string + */ + public static String getLowestCassandraVersion(BulkSparkConf conf, @Nullable String clusterId) { - if (allNodeSettingFutures == null) + try (CassandraContext context = buildCassandraContext(conf, clusterId)) { - throw new IllegalStateException("getAllNodeSettings should not be called on executor. " - + "Cassandra version is pre-computed on driver and broadcast to executors."); - } + List> allNodeSettingFutures = Sidecar.allNodeSettings( + context.getSidecarClient(), + context.getCluster() + ); + + // Worst-case timeout calculation + final long totalTimeout = conf.getSidecarRequestMaxRetryDelayMillis() * + conf.getSidecarRequestRetries() * + allNodeSettingFutures.size(); + List allNodeSettings = FutureUtils.bestEffortGet(allNodeSettingFutures, + totalTimeout, + TimeUnit.MILLISECONDS); + + if (allNodeSettings.isEmpty()) + { + throw new RuntimeException(String.format("Unable to determine the node settings. 0/%d instances available.", + allNodeSettingFutures.size())); + } + else if (allNodeSettings.size() < allNodeSettingFutures.size()) + { + LOGGER.warn("{}/{} instances were used to determine the node settings", + allNodeSettings.size(), allNodeSettingFutures.size()); + } - // Worst-case, the http client is configured for 1 worker pool. - // In that case, each future can take the full retry delay * number of retries, - // and each instance will be processed serially. - final long totalTimeout = conf.getSidecarRequestMaxRetryDelayMillis() * - conf.getSidecarRequestRetries() * - allNodeSettingFutures.size(); - List allNodeSettings = FutureUtils.bestEffortGet(allNodeSettingFutures, - totalTimeout, - TimeUnit.MILLISECONDS); - - if (allNodeSettings.isEmpty()) - { - throw new RuntimeException(String.format("Unable to determine the node settings. 0/%d instances available.", - allNodeSettingFutures.size())); - } - else if (allNodeSettings.size() < allNodeSettingFutures.size()) - { - LOGGER.warn("{}/{} instances were used to determine the node settings", - allNodeSettings.size(), allNodeSettingFutures.size()); - } + // Find the lowest version + NodeSettings ns = allNodeSettings + .stream() + .filter(settings -> !settings.releaseVersion().equalsIgnoreCase("unknown")) + .min(Comparator.comparing(settings -> + CassandraVersionFeatures.cassandraVersionFeaturesFromCassandraVersion(settings.releaseVersion()))) + .orElseThrow(() -> new RuntimeException("No valid Cassandra Versions were returned from Cassandra Sidecar")); - return allNodeSettings; + return ns.releaseVersion(); + } } - public String getVersionFromSidecar() + /** + * Retrieves SSTable versions from a cluster without building full ClusterInfo. + * Creates a temporary CassandraContext to query Sidecar, retrieves the versions, and cleans up. + * + * @param conf Bulk Spark configuration + * @param clusterId Optional cluster identifier + * @return set of SSTable version strings present on the cluster + */ + public static Set getSSTableVersionsOnCluster(BulkSparkConf conf, @Nullable String clusterId) { - NodeSettings nodeSettings = this.nodeSettings.get(); - if (nodeSettings != null) + try (CassandraContext context = buildCassandraContext(conf, clusterId)) { - return nodeSettings.releaseVersion(); + return Sidecar.getSSTableVersionsFromCluster( + context.getSidecarClient(), + context.getCluster(), + conf.getSidecarRequestMaxRetryDelayMillis(), + conf.getSidecarRequestRetries() + ); } - - return getLowestVersion(getAllNodeSettings()); } - @VisibleForTesting - public String getLowestVersion(List allNodeSettings) + protected CassandraBridge bridge() { - NodeSettings ns = this.nodeSettings.get(); - if (ns != null) + // Use the pre-determined bridgeVersion if available + if (bridgeVersion != null) { - return ns.releaseVersion(); + return CassandraBridgeFactory.get(bridgeVersion); } - // It is possible to run the below computation multiple times. Since the computation is local-only, it is OK. - ns = allNodeSettings - .stream() - .filter(settings -> !settings.releaseVersion().equalsIgnoreCase("unknown")) - .min(Comparator.comparing(settings -> - CassandraVersionFeatures.cassandraVersionFeaturesFromCassandraVersion(settings.releaseVersion()))) - .orElseThrow(() -> new RuntimeException("No valid Cassandra Versions were returned from Cassandra Sidecar")); - nodeSettings.compareAndSet(null, ns); - return ns.releaseVersion(); - } - - protected CassandraBridge bridge() - { - return CassandraBridgeFactory.get(getLowestCassandraVersion()); + // Bridge version must be set before accessing bridge + throw new IllegalStateException( + "Bridge version must be set during construction before using bridge()."); } // Startup Validation diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraDirectDataTransportContext.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraDirectDataTransportContext.java index bed36ac4b..7f2db54ca 100644 --- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraDirectDataTransportContext.java +++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraDirectDataTransportContext.java @@ -25,7 +25,6 @@ import com.google.common.collect.Range; import org.apache.cassandra.bridge.CassandraBridge; -import org.apache.cassandra.bridge.CassandraBridgeFactory; import org.apache.cassandra.spark.bulkwriter.token.ReplicaAwareFailureHandler; import org.jetbrains.annotations.NotNull; @@ -36,12 +35,15 @@ public class CassandraDirectDataTransportContext implements TransportContext.Dir @NotNull private final ClusterInfo clusterInfo; @NotNull + private final CassandraBridge bridge; + @NotNull private final DirectDataTransferApi dataTransferApi; public CassandraDirectDataTransportContext(@NotNull BulkWriterContext bulkWriterContext) { this.jobInfo = bulkWriterContext.job(); this.clusterInfo = bulkWriterContext.cluster(); + this.bridge = bulkWriterContext.bridge(); // Use bridge from context (SSTable version-based) this.dataTransferApi = createDirectDataTransferApi(); } @@ -71,7 +73,6 @@ public DirectDataTransferApi dataTransferApi() // only invoke in constructor protected DirectDataTransferApi createDirectDataTransferApi() { - CassandraBridge bridge = CassandraBridgeFactory.get(clusterInfo.getLowestCassandraVersion()); return new SidecarDataTransferApi(clusterInfo.getCassandraContext(), bridge, jobInfo); } } diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/ClusterInfo.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/ClusterInfo.java index 64b654449..a5d0f66d1 100644 --- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/ClusterInfo.java +++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/ClusterInfo.java @@ -24,6 +24,7 @@ import com.google.common.collect.Range; +import org.apache.cassandra.bridge.CassandraVersion; import org.apache.cassandra.spark.bulkwriter.token.TokenRangeMapping; import org.apache.cassandra.spark.data.ReplicationFactor; import org.apache.cassandra.spark.data.partitioner.Partitioner; @@ -43,7 +44,7 @@ * for broadcasting to executors via {@link BulkWriterConfig}. *

* On executors, ClusterInfo instances are reconstructed from the broadcastable wrappers using - * {@link AbstractBulkWriterContext#reconstructClusterInfoOnExecutor(IBroadcastableClusterInfo)}. + * {@link AbstractBulkWriterContext#reconstructClusterInfoOnExecutor(IBroadcastableClusterInfo, CassandraVersion)}. */ public interface ClusterInfo extends StartupValidatable { @@ -51,8 +52,6 @@ public interface ClusterInfo extends StartupValidatable TokenRangeMapping getTokenRangeMapping(boolean cached); - String getLowestCassandraVersion(); - /** * @return WriteAvailability per RingInstance in the cluster */ diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/IBroadcastableClusterInfo.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/IBroadcastableClusterInfo.java index fb4eb2e29..a1c0af404 100644 --- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/IBroadcastableClusterInfo.java +++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/IBroadcastableClusterInfo.java @@ -21,6 +21,7 @@ import java.io.Serializable; +import org.apache.cassandra.bridge.CassandraVersion; import org.apache.cassandra.spark.data.partitioner.Partitioner; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -37,10 +38,9 @@ * Methods in this interface: *

    *
  • {@link #getPartitioner()} - static cluster partitioner configuration
  • - *
  • {@link #getLowestCassandraVersion()} - pre-computed version string
  • *
  • {@link #clusterId()} - cluster identifier (optional)
  • *
  • {@link #getConf()} - BulkSparkConf needed for reconstruction on executors
  • - *
  • {@link #reconstruct()} - reconstructs full ClusterInfo instance on executors
  • + *
  • {@link #reconstruct(CassandraVersion)} - reconstructs full ClusterInfo instance on executors with bridge version
  • *
*/ public interface IBroadcastableClusterInfo extends Serializable @@ -50,11 +50,6 @@ public interface IBroadcastableClusterInfo extends Serializable */ Partitioner getPartitioner(); - /** - * @return the lowest Cassandra version in the cluster - */ - String getLowestCassandraVersion(); - /** * ID string that can uniquely identify a cluster. * When writing to a single cluster, this may be null. @@ -77,7 +72,8 @@ public interface IBroadcastableClusterInfo extends Serializable * This allows adding new broadcastable types without modifying the reconstruction logic * in {@link AbstractBulkWriterContext}. * + * @param bridgeVersion the bridge version from broadcast * @return reconstructed ClusterInfo (CassandraClusterInfo or CassandraClusterInfoGroup) */ - ClusterInfo reconstruct(); + ClusterInfo reconstruct(CassandraVersion bridgeVersion); } diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SSTableWriterFactory.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SSTableWriterFactory.java index 77b8f5f42..f2350c5d4 100644 --- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SSTableWriterFactory.java +++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SSTableWriterFactory.java @@ -22,10 +22,14 @@ import java.util.Set; import org.apache.cassandra.bridge.CassandraBridge; -import org.apache.cassandra.bridge.CassandraBridgeFactory; -import org.apache.cassandra.bridge.CassandraVersionFeatures; import org.apache.cassandra.bridge.SSTableWriter; +/** + * Factory for creating SSTableWriter instances. + * + * @deprecated This class is deprecated. Use {@code BulkWriterContext.bridge().getSSTableWriter()} directly instead. + */ +@Deprecated public final class SSTableWriterFactory { private SSTableWriterFactory() @@ -33,7 +37,13 @@ private SSTableWriterFactory() throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated"); } - public static SSTableWriter getSSTableWriter(CassandraVersionFeatures serverVersion, + /** + * Creates an SSTableWriter using the provided bridge. + * + * @deprecated Use {@code cassandraBridge.getSSTableWriter()} directly instead. + */ + @Deprecated + public static SSTableWriter getSSTableWriter(CassandraBridge cassandraBridge, String inDirectory, String partitioner, String createStatement, @@ -41,7 +51,6 @@ public static SSTableWriter getSSTableWriter(CassandraVersionFeatures serverVers Set userDefinedTypeStatements, int bufferSizeMB) { - CassandraBridge cassandraBridge = CassandraBridgeFactory.get(serverVersion); return cassandraBridge.getSSTableWriter(inDirectory, partitioner, createStatement, diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SortedSSTableWriter.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SortedSSTableWriter.java index 3d09be476..9365acb60 100644 --- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SortedSSTableWriter.java +++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SortedSSTableWriter.java @@ -37,9 +37,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.bridge.CassandraBridgeFactory; import org.apache.cassandra.bridge.CassandraVersion; -import org.apache.cassandra.bridge.CassandraVersionFeatures; import org.apache.cassandra.bridge.SSTableDescriptor; import org.apache.cassandra.spark.common.Digest; import org.apache.cassandra.spark.common.SSTables; @@ -118,26 +116,17 @@ public SortedSSTableWriter(BulkWriterContext writerContext, Path outDir, DigestA this.digestAlgorithm = digestAlgorithm; this.partitionId = partitionId; - String lowestCassandraVersion = writerContext.cluster().getLowestCassandraVersion(); - String packageVersion = getPackageVersion(lowestCassandraVersion); - LOGGER.info("Running with version {}", packageVersion); - SchemaInfo schema = writerContext.schema(); TableSchema tableSchema = schema.getTableSchema(); - this.cqlSSTableWriter = SSTableWriterFactory.getSSTableWriter( - CassandraVersionFeatures.cassandraVersionFeaturesFromCassandraVersion(packageVersion), - this.outDir.toString(), - writerContext.cluster().getPartitioner().toString(), - tableSchema.createStatement, - tableSchema.modificationStatement, - schema.getUserDefinedTypeStatements(), - writerContext.job().sstableDataSizeInMiB()); - } - @NotNull - public String getPackageVersion(String lowestCassandraVersion) - { - return CASSANDRA_VERSION_PREFIX + lowestCassandraVersion; + // Use the bridge from context which is already loaded based on SSTable version analysis + this.cqlSSTableWriter = writerContext.bridge().getSSTableWriter( + this.outDir.toString(), + writerContext.cluster().getPartitioner().toString(), + tableSchema.createStatement, + tableSchema.modificationStatement, + schema.getUserDefinedTypeStatements(), + writerContext.job().sstableDataSizeInMiB()); } /** @@ -319,12 +308,12 @@ public void validateSSTables(@NotNull BulkWriterContext writerContext, @NotNull // and then validate all of them in parallel threads try { - CassandraVersion version = CassandraBridgeFactory.getCassandraVersion(writerContext.cluster().getLowestCassandraVersion()); + CassandraVersion bridgeVersion = writerContext.bridgeVersion(); String keyspace = writerContext.job().qualifiedTableName().keyspace(); String schema = writerContext.schema().getTableSchema().createStatement; Partitioner partitioner = writerContext.cluster().getPartitioner(); Set udtStatements = writerContext.schema().getUserDefinedTypeStatements(); - LocalDataLayer layer = new LocalDataLayer(version, + LocalDataLayer layer = new LocalDataLayer(bridgeVersion, partitioner, keyspace, schema, diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/TableSchema.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/TableSchema.java index 93f02c14d..c99a3a67d 100644 --- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/TableSchema.java +++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/TableSchema.java @@ -33,6 +33,7 @@ import org.apache.cassandra.bridge.CassandraBridge; import org.apache.cassandra.bridge.CassandraBridgeFactory; +import org.apache.cassandra.bridge.CassandraVersion; import org.apache.cassandra.spark.common.schema.ColumnType; import org.apache.cassandra.spark.data.CqlField; import org.apache.cassandra.spark.exception.UnsupportedAnalyticsOperationException; @@ -60,7 +61,7 @@ public class TableSchema final WriteMode writeMode; final TTLOption ttlOption; final TimestampOption timestampOption; - final String lowestCassandraVersion; + final CassandraVersion bridgeVersion; final boolean quoteIdentifiers; public TableSchema(StructType dfSchema, @@ -68,13 +69,13 @@ public TableSchema(StructType dfSchema, WriteMode writeMode, TTLOption ttlOption, TimestampOption timestampOption, - String lowestCassandraVersion, + CassandraVersion bridgeVersion, boolean quoteIdentifiers) { this.writeMode = writeMode; this.ttlOption = ttlOption; this.timestampOption = timestampOption; - this.lowestCassandraVersion = lowestCassandraVersion; + this.bridgeVersion = bridgeVersion; this.quoteIdentifiers = quoteIdentifiers; validateDataFrameCompatibility(dfSchema, tableInfo); @@ -106,7 +107,7 @@ public TableSchema(BroadcastableTableSchema broadcastable) this.writeMode = broadcastable.getWriteMode(); this.ttlOption = broadcastable.getTtlOption(); this.timestampOption = broadcastable.getTimestampOption(); - this.lowestCassandraVersion = broadcastable.getLowestCassandraVersion(); + this.bridgeVersion = broadcastable.getBridgeVersion(); this.quoteIdentifiers = broadcastable.isQuoteIdentifiers(); } @@ -181,7 +182,7 @@ private String getInsertStatement(StructType dfSchema, TTLOption ttlOption, TimestampOption timestampOption) { - CassandraBridge bridge = CassandraBridgeFactory.get(lowestCassandraVersion); + CassandraBridge bridge = CassandraBridgeFactory.get(bridgeVersion); List columnNames = Arrays.stream(dfSchema.fieldNames()) .filter(fieldName -> !fieldName.equals(ttlOption.columnName())) .filter(fieldName -> !fieldName.equals(timestampOption.columnName())) @@ -224,7 +225,7 @@ else if (ttlOption.withTTl()) private String getDeleteStatement(StructType dfSchema, TableInfoProvider tableInfo) { - CassandraBridge bridge = CassandraBridgeFactory.get(lowestCassandraVersion); + CassandraBridge bridge = CassandraBridgeFactory.get(bridgeVersion); Stream fieldEqualityStatements = Arrays.stream(dfSchema.fieldNames()).map(key -> maybeQuotedIdentifier(bridge, quoteIdentifiers, key) + "=?"); String deleteStatement = String.format("DELETE FROM %s.%s where %s;", maybeQuotedIdentifier(bridge, quoteIdentifiers, tableInfo.getKeyspaceName()), diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/cloudstorage/CloudStorageStreamSession.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/cloudstorage/CloudStorageStreamSession.java index cd92da277..b3d86ba56 100644 --- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/cloudstorage/CloudStorageStreamSession.java +++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/cloudstorage/CloudStorageStreamSession.java @@ -36,7 +36,6 @@ import o.a.c.sidecar.client.shaded.common.request.data.CreateSliceRequestPayload; import o.a.c.sidecar.client.shaded.common.response.data.RestoreJobSummaryResponsePayload; import org.apache.cassandra.bridge.CassandraBridge; -import org.apache.cassandra.bridge.CassandraBridgeFactory; import org.apache.cassandra.bridge.SSTableDescriptor; import org.apache.cassandra.clients.Sidecar; import o.a.c.sidecar.client.shaded.client.SidecarInstance; @@ -80,7 +79,7 @@ public CloudStorageStreamSession(BulkWriterContext bulkWriterContext, SortedSSTa ExecutorService executorService) { this(bulkWriterContext, sstableWriter, transportContext, sessionID, tokenRange, - CassandraBridgeFactory.get(bulkWriterContext.cluster().getLowestCassandraVersion()), + bulkWriterContext.bridge(), // Use bridge from context (SSTable version-based) failureHandler, executorService); } diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/cloudstorage/coordinated/CassandraClusterInfoGroup.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/cloudstorage/coordinated/CassandraClusterInfoGroup.java index 9fd870cdc..a3862f390 100644 --- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/cloudstorage/coordinated/CassandraClusterInfoGroup.java +++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/cloudstorage/coordinated/CassandraClusterInfoGroup.java @@ -24,6 +24,7 @@ import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -40,6 +41,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.bridge.CassandraVersion; import org.apache.cassandra.bridge.CassandraVersionFeatures; import org.apache.cassandra.spark.bulkwriter.BulkSparkConf; import org.apache.cassandra.spark.bulkwriter.CassandraClusterInfo; @@ -83,35 +85,37 @@ public class CassandraClusterInfoGroup implements ClusterInfo, MultiClusterSuppo private volatile TokenRangeMapping consolidatedTokenRangeMapping; // Pre-computed values from BroadcastableClusterInfoGroup (only set when reconstructed on executors) private Partitioner cachedPartitioner; - private String cachedLowestCassandraVersion; /** * Creates {@link CassandraClusterInfoGroup} with the list of {@link ClusterInfo} from {@link BulkSparkConf} and validation * The validation ensures non-empty list of {@link ClusterInfo}, where all objects have non-empty and unique clusterId * @param conf bulk write conf + * @param bridgeVersion bridge version (nullable for preliminary construction) * @return new {@link CassandraClusterInfoGroup} instance */ - public static CassandraClusterInfoGroup fromBulkSparkConf(BulkSparkConf conf) + public static CassandraClusterInfoGroup fromBulkSparkConf(BulkSparkConf conf, CassandraVersion bridgeVersion) { - return fromBulkSparkConf(conf, clusterId -> new CassandraClusterInfo(conf, clusterId)); + return fromBulkSparkConf(conf, clusterId -> new CassandraClusterInfo(conf, clusterId, bridgeVersion)); } + /** * Reconstruct from BroadcastableClusterInfoGroup on executor. * Creates CassandraClusterInfo instances for each cluster that will fetch data from Sidecar. - * Leverages pre-computed values (partitioner, lowestCassandraVersion) from the broadcastable + * Leverages pre-computed values (partitioner, bridgeVersion) from the broadcastable * to avoid re-validation and re-computation on executors. * * @param broadcastable the broadcastable cluster info group from broadcast * @return new {@link CassandraClusterInfoGroup} instance */ - public static CassandraClusterInfoGroup from(BroadcastableClusterInfoGroup broadcastable) + public static CassandraClusterInfoGroup from(BroadcastableClusterInfoGroup broadcastable, + CassandraVersion bridgeVersion) { - return new CassandraClusterInfoGroup(broadcastable); + return new CassandraClusterInfoGroup(broadcastable, bridgeVersion); } /** - * Similar to {@link #fromBulkSparkConf(BulkSparkConf)} but takes additional function to create {@link ClusterInfo} + * Similar to {@link #fromBulkSparkConf(BulkSparkConf, CassandraVersion)} (BulkSparkConf)} but takes additional function to create {@link ClusterInfo} */ public static CassandraClusterInfoGroup fromBulkSparkConf(BulkSparkConf conf, Function clusterInfoFactory) { @@ -155,19 +159,18 @@ private CassandraClusterInfoGroup(List clusterInfos) * re-validation and re-computation on executors. * * @param broadcastable the broadcastable cluster info group from broadcast + * @param bridgeVersion the bridge version from broadcast */ - private CassandraClusterInfoGroup(BroadcastableClusterInfoGroup broadcastable) + private CassandraClusterInfoGroup(BroadcastableClusterInfoGroup broadcastable, CassandraVersion bridgeVersion) { // Build list of ClusterInfo from broadcastable data List clusterInfosList = new ArrayList<>(); - broadcastable.forEach((clusterId, broadcastableInfo) -> { - clusterInfosList.add(new CassandraClusterInfo((BroadcastableClusterInfo) broadcastableInfo)); - }); + broadcastable.forEach((clusterId, broadcastableInfo) -> clusterInfosList.add( + new CassandraClusterInfo((BroadcastableClusterInfo) broadcastableInfo, bridgeVersion))); this.clusterInfos = Collections.unmodifiableList(clusterInfosList); // Extract pre-computed values from driver to avoid re-validation on executors this.cachedPartitioner = broadcastable.getPartitioner(); - this.cachedLowestCassandraVersion = broadcastable.getLowestCassandraVersion(); this.clusterId = broadcastable.clusterId(); clusterInfoById(); } @@ -203,38 +206,6 @@ public TokenRangeMapping getTokenRangeMapping(boolean cached) return consolidatedTokenRangeMapping; } - /** - * @return the lowest cassandra version among all clusters - */ - @Override - public String getLowestCassandraVersion() - { - // Return cached value if available (executor-side reconstruction) - if (cachedLowestCassandraVersion != null) - { - return cachedLowestCassandraVersion; - } - - if (clusterInfos.size() == 1) - { - return clusterInfos.get(0).getLowestCassandraVersion(); - } - - Map aggregated = applyOnEach(ClusterInfo::getLowestCassandraVersion); - List versions = aggregated.values() - .stream() - .map(CassandraVersionFeatures::cassandraVersionFeaturesFromCassandraVersion) - .sorted() - .collect(Collectors.toList()); - CassandraVersionFeatures first = versions.get(0); - CassandraVersionFeatures last = versions.get(versions.size() - 1); - Preconditions.checkState(first.getMajorVersion() == last.getMajorVersion(), - "Cluster versions are not compatible. lowest=%s and highest=%s", - first.getRawVersionString(), last.getRawVersionString()); - - return first.getRawVersionString(); - } - @Override public Map clusterWriteAvailability() { @@ -395,4 +366,61 @@ Map clusterInfoByIdUnsafe() { return clusterInfoById; } + + /** + * Retrieves the lowest Cassandra version from all clusters. + * + * @param conf Bulk Spark configuration + * @return lowest Cassandra version string across all clusters + */ + public static String getLowestCassandraVersion(BulkSparkConf conf) + { + CoordinatedWriteConf coordinatedWriteConf = conf.coordinatedWriteConf(); + Preconditions.checkArgument(coordinatedWriteConf != null, + "CoordinatedWriteConf is required for multi-cluster operations"); + + Map clusterVersions = new HashMap<>(); + for (String clusterId : coordinatedWriteConf.clusters().keySet()) + { + String version = CassandraClusterInfo.getLowestCassandraVersion(conf, clusterId); + clusterVersions.put(clusterId, version); + } + + // Find the lowest version across all clusters + List versions = clusterVersions.values() + .stream() + .map(CassandraVersionFeatures::cassandraVersionFeaturesFromCassandraVersion) + .sorted() + .collect(Collectors.toList()); + + CassandraVersionFeatures first = versions.get(0); + CassandraVersionFeatures last = versions.get(versions.size() - 1); + Preconditions.checkState(first.getMajorVersion() == last.getMajorVersion(), + "Cluster versions are not compatible. lowest=%s and highest=%s", + first.getRawVersionString(), last.getRawVersionString()); + + return first.getRawVersionString(); + } + + /** + * Retrieves aggregated SSTable versions from all clusters. + * + * @param conf Bulk Spark configuration + * @return set of SSTable versions present across all clusters + */ + public static Set getSSTableVersionsOnCluster(BulkSparkConf conf) + { + CoordinatedWriteConf coordinatedWriteConf = conf.coordinatedWriteConf(); + Preconditions.checkArgument(coordinatedWriteConf != null, + "CoordinatedWriteConf is required for multi-cluster operations"); + + Set aggregatedSSTableVersions = new HashSet<>(); + for (String clusterId : coordinatedWriteConf.clusters().keySet()) + { + Set sstableVersions = CassandraClusterInfo.getSSTableVersionsOnCluster(conf, clusterId); + aggregatedSSTableVersions.addAll(sstableVersions); + } + + return aggregatedSSTableVersions; + } } diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/cloudstorage/coordinated/CassandraCoordinatedBulkWriterContext.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/cloudstorage/coordinated/CassandraCoordinatedBulkWriterContext.java index 5329fc213..02cabd227 100644 --- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/cloudstorage/coordinated/CassandraCoordinatedBulkWriterContext.java +++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/cloudstorage/coordinated/CassandraCoordinatedBulkWriterContext.java @@ -19,11 +19,13 @@ package org.apache.cassandra.spark.bulkwriter.cloudstorage.coordinated; +import java.util.Set; import java.util.UUID; import com.google.common.base.Preconditions; import org.apache.commons.lang3.StringUtils; +import org.apache.cassandra.bridge.CassandraVersion; import org.apache.cassandra.spark.bulkwriter.AbstractBulkWriterContext; import org.apache.cassandra.spark.bulkwriter.BulkSparkConf; import org.apache.cassandra.spark.bulkwriter.BulkWriterConfig; @@ -72,9 +74,21 @@ private void validateConfiguration(BulkSparkConf conf) } @Override - protected ClusterInfo buildClusterInfo() + protected String getLowestCassandraVersion(@NotNull BulkSparkConf conf) { - CassandraClusterInfoGroup clusterInfoGroup = CassandraClusterInfoGroup.fromBulkSparkConf(bulkSparkConf()); + return CassandraClusterInfoGroup.getLowestCassandraVersion(conf); + } + + @Override + protected Set getSSTableVersionsOnCluster(@NotNull BulkSparkConf conf) + { + return CassandraClusterInfoGroup.getSSTableVersionsOnCluster(conf); + } + + @Override + protected ClusterInfo buildClusterInfo(CassandraVersion bridgeVersion) + { + CassandraClusterInfoGroup clusterInfoGroup = CassandraClusterInfoGroup.fromBulkSparkConf(bulkSparkConf(), bridgeVersion); clusterInfoGroup.startupValidate(); return clusterInfoGroup; } 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 b6e84c6f5..fa16de8fe 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 @@ -68,6 +68,7 @@ import org.apache.cassandra.bridge.CassandraBridge; import org.apache.cassandra.bridge.CassandraBridgeFactory; import org.apache.cassandra.bridge.CassandraVersion; +import org.apache.cassandra.bridge.SSTableVersionAnalyzer; import org.apache.cassandra.clients.ExecutorHolder; import org.apache.cassandra.clients.Sidecar; import org.apache.cassandra.secrets.SslConfig; @@ -77,6 +78,8 @@ import o.a.c.sidecar.client.shaded.client.SidecarInstanceImpl; import o.a.c.sidecar.client.shaded.client.SimpleSidecarInstancesProvider; import o.a.c.sidecar.client.shaded.client.exception.RetriesExhaustedException; +import org.apache.cassandra.spark.KryoRegister; +import org.apache.cassandra.spark.bulkwriter.BulkSparkConf; import org.apache.cassandra.spark.common.SidecarInstanceFactory; import org.apache.cassandra.spark.common.SizingFactory; import org.apache.cassandra.spark.config.SchemaFeature; @@ -140,6 +143,7 @@ public class CassandraDataLayer extends PartitionedDataLayer implements StartupV protected Map rfMap; @Nullable protected String lastModifiedTimestampField; + protected Set sstableVersionsOnCluster; // volatile in order to publish the reference for visibility protected volatile CqlTable cqlTable; protected transient TimeProvider timeProvider; @@ -185,7 +189,7 @@ protected CassandraDataLayer(@Nullable String keyspace, @Nullable SslConfig sslConfig, @NotNull CqlTable cqlTable, @NotNull TokenPartitioner tokenPartitioner, - @NotNull CassandraVersion version, + @NotNull CassandraVersion bridgeVersion, @NotNull ConsistencyLevel consistencyLevel, @NotNull String sidecarInstances, @NotNull int sidecarPort, @@ -198,11 +202,12 @@ protected CassandraDataLayer(@Nullable String keyspace, List requestedFeatures, @NotNull Map rfMap, TimeProvider timeProvider, - SSTableTimeRangeFilter sstableTimeRangeFilter) + SSTableTimeRangeFilter sstableTimeRangeFilter, + Set sstableVersionsOnCluster) { super(consistencyLevel, datacenter); this.snapshotName = snapshotName; - this.bridge = CassandraBridgeFactory.get(version); + this.bridge = CassandraBridgeFactory.get(bridgeVersion); this.keyspace = keyspace; this.table = table; this.quoteIdentifiers = quoteIdentifiers; @@ -225,6 +230,7 @@ protected CassandraDataLayer(@Nullable String keyspace, this.rfMap = rfMap; this.timeProvider = timeProvider; this.sstableTimeRangeFilter = sstableTimeRangeFilter; + this.sstableVersionsOnCluster = sstableVersionsOnCluster; this.maybeQuoteKeyspaceAndTable(); this.initSidecarClient(); this.initInstanceMap(); @@ -272,7 +278,11 @@ private int initBulkReader(@NotNull ClientConfig options) throws ExecutionExcept NodeSettings nodeSettings = sidecar.nodeSettings().get(); String cassandraVersion = getEffectiveCassandraVersionForRead(clusterConfig, nodeSettings); Partitioner partitioner = Partitioner.from(nodeSettings.partitioner()); - bridge = CassandraBridgeFactory.get(cassandraVersion); + + // Initialize SSTable versions and bridge version + CassandraVersion bridgeVersion = initializeSSTableVersionsAndBridgeVersion(cassandraVersion); + bridge = CassandraBridgeFactory.get(bridgeVersion); + // optionally quote identifiers if the option has been set, we need an instance for the bridge maybeQuoteKeyspaceAndTable(); @@ -324,6 +334,73 @@ private int initBulkReader(@NotNull ClientConfig options) throws ExecutionExcept return effectiveNumberOfCores; } + /** + * Checks if SSTable version-based bridge selection is disabled. + * Protected to allow test overrides without SparkContext. + * + * @return true if disabled, false if enabled + */ + @VisibleForTesting + protected boolean isSSTableVersionBasedBridgeDisabled() + { + return BulkSparkConf.getDisableSSTableVersionBasedBridge(); + } + + /** + * Initializes SSTable versions from cluster gossip and determines bridge version. + * + * @param cassandraVersion the Cassandra version + * @return the determined bridge version + */ + @VisibleForTesting + protected CassandraVersion initializeSSTableVersionsAndBridgeVersion(String cassandraVersion) + { + // Check if user has explicitly disabled SSTable version-based selection via Spark configuration + boolean isSSTableVersionBasedBridgeDisabled = isSSTableVersionBasedBridgeDisabled(); + + // Get SSTable versions from cluster only if SSTable version-based selection is enabled + // If disabled, skip retrieval to allow job to proceed even when SSTable version detection fails + if (isSSTableVersionBasedBridgeDisabled) + { + this.sstableVersionsOnCluster = null; + } + else + { + this.sstableVersionsOnCluster = retrieveSSTableVersionsFromCluster(); + } + + // Determine bridge version + CassandraVersion bridgeVersion = SSTableVersionAnalyzer.determineBridgeVersionForRead( + sstableVersionsOnCluster, + cassandraVersion, + isSSTableVersionBasedBridgeDisabled); + + // Validate SSTable versions + validateSStableVersions(this.sstableVersionsOnCluster, bridgeVersion, isSSTableVersionBasedBridgeDisabled); + + // Validate that Kryo registrator exists for this bridge version + KryoRegister.validateKryoRegistratorExists(bridgeVersion, cassandraVersion); + + return bridgeVersion; + } + + /** + * Retrieves SSTable versions from cluster via Sidecar gossip. + * Protected to allow test overrides. + * + * @return set of SSTable versions from cluster + */ + @VisibleForTesting + protected Set retrieveSSTableVersionsFromCluster() + { + return Sidecar.getSSTableVersionsFromCluster( + sidecar, + clusterConfig, + sidecarClientConfig.maxMillisToSleep(), + sidecarClientConfig.maxRetries() + ); + } + protected void shutdownHook(ClientConfig options) { ClientConfig.ClearSnapshotStrategy clearSnapshotStrategy = options.clearSnapshotStrategy(); @@ -625,7 +702,7 @@ private List collectSSTableList(SidecarInstance sidecarInstance, } // Map to SSTable - return result.values().stream() + List sstables = result.values().stream() .map(components -> new SidecarProvisionedSSTable(sidecar, sidecarClientConfig, sidecarInstance, @@ -636,6 +713,11 @@ private List collectSSTableList(SidecarInstance sidecarInstance, partitionId, stats())) .collect(Collectors.toList()); + + // Validate SSTable versions against expected versions from gossip + validateSStableVersions(sstables); + + return sstables; } @Override @@ -773,6 +855,7 @@ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundE this.rfMap = (Map) in.readObject(); this.timeProvider = new ReaderTimeProvider(in.readInt()); this.sstableTimeRangeFilter = (SSTableTimeRangeFilter) in.readObject(); + this.sstableVersionsOnCluster = (Set) in.readObject(); this.maybeQuoteKeyspaceAndTable(); this.initSidecarClient(); this.initInstanceMap(); @@ -819,6 +902,7 @@ private void writeObject(ObjectOutputStream out) throws IOException, ClassNotFou out.writeObject(this.rfMap); out.writeInt(timeProvider.referenceEpochInSeconds()); out.writeObject(this.sstableTimeRangeFilter); + out.writeObject(this.sstableVersionsOnCluster); } private static void writeNullable(ObjectOutputStream out, @Nullable String string) throws IOException @@ -844,6 +928,124 @@ private static String readNullable(ObjectInputStream in) throws IOException return null; } + /** + * Validates that all SSTable versions are supported for the given Cassandra version. + * This validation runs on the Spark driver. + * + * @param sstableVersionsOnCluster set of SSTable versions across all nodes in the cluster + * @param cassandraVersion the Cassandra version + * @param isSSTableVersionBasedBridgeDisabled flag to disable sstable version based bridge determination + * @throws UnsupportedOperationException if any unsupported SSTable version is detected + */ + @VisibleForTesting + void validateSStableVersions(Set sstableVersionsOnCluster, + CassandraVersion cassandraVersion, + boolean isSSTableVersionBasedBridgeDisabled) + { + // Skip validation when fallback mode is enabled + if (isSSTableVersionBasedBridgeDisabled) + { + LOGGER.debug("Skipping SSTable version validation on driver - fallback mode enabled", + BulkSparkConf.DISABLE_SSTABLE_VERSION_BASED_BRIDGE); + return; + } + + if (sstableVersionsOnCluster == null || sstableVersionsOnCluster.isEmpty()) + { + // Fail fast with helpful error message + throw new IllegalStateException(String.format( + "Unable to retrieve SSTable versions from cluster. " + + "This is required for SSTable version-based bridge selection. " + + "If you want to bypass this check and use cassandra.version for bridge selection, " + + "set %s=true", BulkSparkConf.DISABLE_SSTABLE_VERSION_BASED_BRIDGE)); + } + + // Get SSTable versions that can be read by this Cassandra version + Set supportedVersions = cassandraVersion.getSupportedSStableVersionsForRead(); + + // Find any unsupported versions + Set unsupportedVersions = sstableVersionsOnCluster.stream() + .filter(version -> !supportedVersions.contains(version)) + .collect(Collectors.toSet()); + + if (!unsupportedVersions.isEmpty()) + { + String errorMessage = String.format( + "Detected unsupported SSTable version(s) %s for Cassandra version %s. " + + "Supported versions: %s. " + + "Observed SSTable versions in the cluster: %s. ", + unsupportedVersions, + cassandraVersion, + supportedVersions, + sstableVersionsOnCluster); + LOGGER.error(errorMessage); + throw new UnsupportedOperationException(errorMessage); + } + + LOGGER.debug("SSTable version validation successful. All observed versions {} are supported for Cassandra version {}", + sstableVersionsOnCluster, cassandraVersion); + } + + /** + * Validates that all SSTables being read have versions that were observed in gossip info. + * This catches cases where SSTables have unexpected versions that weren't seen during driver initialization. + * This validation runs on executors. + * + * @param sstables list of SSTables to validate + * @throws UnsupportedOperationException if any SSTable has a version not in expected sstable versions + */ + @VisibleForTesting + void validateSStableVersions(List sstables) + { + // Check if user has explicitly disabled SSTable version-based selection + boolean isSSTableVersionBasedBridgeDisabled = isSSTableVersionBasedBridgeDisabled(); + + // Skip validation when fallback mode is enabled + if (isSSTableVersionBasedBridgeDisabled) + { + LOGGER.debug("Skipping SSTable version validation on executor - " + + "fallback mode enabled via {}=true", + BulkSparkConf.DISABLE_SSTABLE_VERSION_BASED_BRIDGE); + return; + } + + Set expectedVersions = this.sstableVersionsOnCluster; + if (expectedVersions == null || expectedVersions.isEmpty()) + { + // Fail fast with helpful error message + throw new IllegalStateException( + "Unable to validate SSTable versions - no expected versions available from cluster. " + + "This is required for SSTable version-based bridge selection. " + + "If you want to bypass this check and use cassandra.version for bridge selection, " + + "set " + BulkSparkConf.DISABLE_SSTABLE_VERSION_BASED_BRIDGE + "=true"); + } + + for (SSTable ssTable : sstables) + { + String ssTableFileName = ssTable.getDataFileName(); + // Extract full version string (e.g., "big-nb" from the filename) + String ssTableVersion = ssTable.getFormat() + "-" + ssTable.getVersion(); + + if (!expectedVersions.contains(ssTableVersion)) + { + String errorMessage = String.format( + "SSTable '%s' has version '%s' which was not observed in cluster gossip info. " + + "Expected versions from gossip: %s. ", + ssTableFileName, + ssTableVersion, + expectedVersions); + LOGGER.error(errorMessage); + throw new UnsupportedOperationException(errorMessage); + } + } + + if (!sstables.isEmpty()) + { + LOGGER.debug("Validated {} SSTable(s) against expected versions from gossip: {}", + sstables.size(), expectedVersions); + } + } + // Kryo Serialization public static class Serializer extends com.esotericsoftware.kryo.Serializer @@ -895,6 +1097,7 @@ public void write(Kryo kryo, Output out, CassandraDataLayer dataLayer) kryo.writeObject(out, dataLayer.rfMap); out.writeInt(dataLayer.timeProvider.referenceEpochInSeconds()); kryo.writeObject(out, dataLayer.sstableTimeRangeFilter); + kryo.writeObject(out, dataLayer.sstableVersionsOnCluster); } @SuppressWarnings("unchecked") @@ -937,7 +1140,8 @@ public CassandraDataLayer read(Kryo kryo, Input in, Class ty kryo.readObject(in, SchemaFeaturesListWrapper.class).toList(), kryo.readObject(in, HashMap.class), new ReaderTimeProvider(in.readInt()), - kryo.readObject(in, SSTableTimeRangeFilter.class)); + kryo.readObject(in, SSTableTimeRangeFilter.class), + kryo.readObject(in, HashSet.class)); } // Wrapper only used internally for Kryo serialization/deserialization diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/LocalDataLayer.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/LocalDataLayer.java index a6d44904d..251bb6965 100644 --- a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/LocalDataLayer.java +++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/LocalDataLayer.java @@ -188,12 +188,12 @@ public static LocalDataLayer from(Map options) getOrThrow(options, lowerCaseKey("dirs")).split(",")); } - public LocalDataLayer(@NotNull CassandraVersion version, + public LocalDataLayer(@NotNull CassandraVersion bridgeVersion, @NotNull String keyspace, @NotNull String createStatement, String... paths) { - this(version, + this(bridgeVersion, Partitioner.Murmur3Partitioner, keyspace, createStatement, @@ -205,13 +205,13 @@ public LocalDataLayer(@NotNull CassandraVersion version, paths); } - public LocalDataLayer(@NotNull CassandraVersion version, + public LocalDataLayer(@NotNull CassandraVersion bridgeVersion, @NotNull String keyspace, @NotNull String createStatement, @NotNull Set udtStatements, String... paths) { - this(version, + this(bridgeVersion, Partitioner.Murmur3Partitioner, keyspace, createStatement, @@ -224,7 +224,7 @@ public LocalDataLayer(@NotNull CassandraVersion version, } // CHECKSTYLE IGNORE: Constructor with many parameters - public LocalDataLayer(@NotNull CassandraVersion version, + public LocalDataLayer(@NotNull CassandraVersion bridgeVersion, @NotNull Partitioner partitioner, @NotNull String keyspace, @NotNull String createStatement, @@ -235,7 +235,7 @@ public LocalDataLayer(@NotNull CassandraVersion version, @NotNull SSTableTimeRangeFilter sstableTimeRangeFilter, String... paths) { - this.bridge = CassandraBridgeFactory.get(version); + this.bridge = CassandraBridgeFactory.get(bridgeVersion); this.partitioner = partitioner; this.cqlTable = bridge().buildSchema(createStatement, keyspace, @@ -260,7 +260,7 @@ public LocalDataLayer(@NotNull CassandraVersion version, } // For serialization - private LocalDataLayer(@NotNull CassandraVersion version, + private LocalDataLayer(@NotNull CassandraVersion bridgeVersion, @NotNull Partitioner partitioner, @NotNull CqlTable cqlTable, @NotNull String jobId, @@ -270,7 +270,7 @@ private LocalDataLayer(@NotNull CassandraVersion version, @NotNull SSTableTimeRangeFilter sstableTimeRangeFilter, String... paths) { - this.bridge = CassandraBridgeFactory.get(version); + this.bridge = CassandraBridgeFactory.get(bridgeVersion); this.partitioner = partitioner; this.cqlTable = cqlTable; this.jobId = jobId; diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/KryoRegisterTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/KryoRegisterTest.java new file mode 100644 index 000000000..a4e55bf3e --- /dev/null +++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/KryoRegisterTest.java @@ -0,0 +1,95 @@ +/* + * 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; + +import org.junit.jupiter.api.Test; + +import org.apache.cassandra.bridge.CassandraVersion; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatNoException; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** + * Unit tests for KryoRegister + */ +public class KryoRegisterTest +{ + @Test + void testValidateKryoRegistratorExistsForFourZero() + { + assertThatNoException() + .describedAs("FOURZERO should have a Kryo registrator") + .isThrownBy(() -> KryoRegister.validateKryoRegistratorExists(CassandraVersion.FOURZERO, "4.0.0")); + } + + @Test + void testValidateKryoRegistratorExistsForFourOne() + { + assertThatNoException() + .describedAs("FOURONE should have a Kryo registrator") + .isThrownBy(() -> KryoRegister.validateKryoRegistratorExists(CassandraVersion.FOURONE, "4.1.0")); + } + + @Test + void testValidateKryoRegistratorExistsForFiveZero() + { + assertThatNoException() + .describedAs("FIVEZERO should have a Kryo registrator") + .isThrownBy(() -> KryoRegister.validateKryoRegistratorExists(CassandraVersion.FIVEZERO, "5.0.0")); + } + + @Test + void testValidateKryoRegistratorMissingForThreeZero() + { + assertThatThrownBy(() -> KryoRegister.validateKryoRegistratorExists(CassandraVersion.THREEZERO, "3.0.0")) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("No Kryo registrator configured for bridge version THREEZERO") + .hasMessageContaining("Cluster Cassandra version: 3.0.0") + .hasMessageContaining("Available Kryo registrators:") + .hasMessageContaining("FOURZERO") + .hasMessageContaining("FOURONE") + .hasMessageContaining("FIVEZERO") + // should mention config param to update + .hasMessageContaining("spark.cassandra_analytics.cassandra.version"); + } + + @Test + void testKryoRegistratorClassesAreCorrect() + { + assertThat(KryoRegister.KRYO_REGISTRATORS.get(CassandraVersion.FOURZERO)) + .isEqualTo(KryoRegister.V40.class); + + assertThat(KryoRegister.KRYO_REGISTRATORS.get(CassandraVersion.FOURONE)) + .isEqualTo(KryoRegister.V41.class); + + assertThat(KryoRegister.KRYO_REGISTRATORS.get(CassandraVersion.FIVEZERO)) + .isEqualTo(KryoRegister.V50.class); + } + + @Test + void testValidateWithNullCassandraVersionString() + { + // Should not throw - clusterCassandraVersion is optional for error message context + assertThatNoException() + .describedAs("Validation should work with null cassandraVersion string") + .isThrownBy(() -> KryoRegister.validateKryoRegistratorExists(CassandraVersion.FOURZERO, null)); + } +} diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/AbstractBulkWriterContextTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/AbstractBulkWriterContextTest.java new file mode 100644 index 000000000..91bdd67cb --- /dev/null +++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/AbstractBulkWriterContextTest.java @@ -0,0 +1,168 @@ +/* + * 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.bulkwriter; + +import java.util.Collections; +import java.util.Set; +import java.util.UUID; + +import org.junit.jupiter.api.Test; + +import org.apache.cassandra.bridge.CassandraVersion; +import org.apache.cassandra.spark.bulkwriter.cloudstorage.coordinated.MultiClusterContainer; +import org.apache.spark.sql.types.StructType; +import org.jetbrains.annotations.NotNull; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +class AbstractBulkWriterContextTest +{ + @Test + void testKryoRegistrationWarningMessage() + { + // Test that the KRYO_REGISTRATION_WARNING constant exists and has expected content + assertThat(AbstractBulkWriterContext.KRYO_REGISTRATION_WARNING) + .isNotNull() + .contains("Spark Bulk Writer Kryo Registrator") + .contains("SbwKryoRegistrator") + .contains("was not registered with Spark"); + } + + @Test + void testSSTableVersionBasedBridgeDisabled() + { + BulkSparkConf conf = mock(BulkSparkConf.class); + when(conf.isSSTableVersionBasedBridgeDisabled()).thenReturn(true); + + StructType schema = mock(StructType.class); + TestBulkWriterContext context = TestBulkWriterContext.create(conf, schema, 1, "4.0.0", null); + + // Verify bridge version was still determined + assertThat(context.bridgeVersion()).isEqualTo(CassandraVersion.FOURZERO); + + // Verify that SSTable versions retrieval was not called (disabled) + assertThat(TestBulkWriterContext.sstableVersionRetrievalCount).isEqualTo(0); + assertThat(TestBulkWriterContext.versionRetrievalCount).isEqualTo(1); + } + + @Test + void testSSTableVersionBasedBridgeEnabled() + { + BulkSparkConf conf = mock(BulkSparkConf.class); + when(conf.isSSTableVersionBasedBridgeDisabled()).thenReturn(false); + + Set sstableVersions = Collections.singleton("big-oa"); + StructType schema = mock(StructType.class); + TestBulkWriterContext context = TestBulkWriterContext.create(conf, schema, 1, "5.0.0", sstableVersions); + + // Verify bridge version was determined + assertThat(context.bridgeVersion()).isEqualTo(CassandraVersion.FIVEZERO); + + // Verify that both version and SSTable versions retrieval were called + assertThat(TestBulkWriterContext.versionRetrievalCount).isEqualTo(1); + assertThat(TestBulkWriterContext.sstableVersionRetrievalCount).isEqualTo(1); + } + + /** + * Concrete test implementation of AbstractBulkWriterContext for testing + */ + static class TestBulkWriterContext extends AbstractBulkWriterContext + { + private static String staticLowestVersion; + private static Set staticSSTableVersions; + static int versionRetrievalCount = 0; + static int sstableVersionRetrievalCount = 0; + + private TestBulkWriterContext(@NotNull BulkSparkConf conf, + @NotNull StructType structType, + int sparkDefaultParallelism) + { + super(conf, structType, sparkDefaultParallelism); + } + + static TestBulkWriterContext create(@NotNull BulkSparkConf conf, + @NotNull StructType structType, + int sparkDefaultParallelism, + @NotNull String lowestVersion, + Set sstableVersions) + { + staticLowestVersion = lowestVersion; + staticSSTableVersions = sstableVersions; + versionRetrievalCount = 0; + sstableVersionRetrievalCount = 0; + return new TestBulkWriterContext(conf, structType, sparkDefaultParallelism); + } + + @Override + protected String getLowestCassandraVersion(@NotNull BulkSparkConf conf) + { + versionRetrievalCount++; + return staticLowestVersion; + } + + @Override + protected Set getSSTableVersionsOnCluster(@NotNull BulkSparkConf conf) + { + sstableVersionRetrievalCount++; + return staticSSTableVersions; + } + + @Override + protected ClusterInfo buildClusterInfo(CassandraVersion bridgeVersion) + { + return mock(ClusterInfo.class); + } + + @Override + protected JobInfo buildJobInfo() + { + // Return a mock JobInfo to avoid complex dependencies + return mock(JobInfo.class); + } + + @Override + protected SchemaInfo buildSchemaInfo(StructType structType) + { + // Return a mock SchemaInfo to avoid complex dependencies + return mock(SchemaInfo.class); + } + + @Override + protected TransportContext buildTransportContext(boolean isOnDriver) + { + // Return a mock TransportContext to avoid complex dependencies + return mock(TransportContext.class); + } + + @Override + protected void validateKeyspaceReplication() + { + // No-op for testing + } + + @Override + protected MultiClusterContainer generateRestoreJobIds() + { + return null; + } + } +} diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/BulkSparkConfTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/BulkSparkConfTest.java index ab0e663b1..3da5da4c9 100644 --- a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/BulkSparkConfTest.java +++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/BulkSparkConfTest.java @@ -33,6 +33,7 @@ import org.apache.cassandra.spark.bulkwriter.util.SbwKryoRegistrator; import org.apache.cassandra.spark.utils.BuildInfo; import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; import org.jetbrains.annotations.NotNull; import static org.apache.cassandra.spark.bulkwriter.BulkSparkConf.DEFAULT_SIDECAR_PORT; @@ -368,6 +369,103 @@ void testCassandraRoleRead() assertThat(conf.cassandraRole).isEqualTo("custom_role"); } + @Test + void testDefaultDisableSSTableVersionBasedBridge() + { + Map options = copyDefaultOptions(); + BulkSparkConf conf = new BulkSparkConf(sparkConf, options); + assertThat(conf.isSSTableVersionBasedBridgeDisabled()) + .describedAs("By default, SSTable version-based bridge should be enabled (false)") + .isFalse(); + } + + @Test + void testSSTableVersionBasedBridgeDisabledViaSparkConf() + { + SparkConf conf = new SparkConf() + .set(BulkSparkConf.DISABLE_SSTABLE_VERSION_BASED_BRIDGE, "true"); + BulkSparkConf bulkConf = new BulkSparkConf(conf, defaultOptions); + assertThat(bulkConf.isSSTableVersionBasedBridgeDisabled()) + .describedAs("SSTable version-based bridge should be disabled when configured") + .isTrue(); + } + + @Test + void testSSTableVersionBasedBridgeEnabledExplicitly() + { + SparkConf conf = new SparkConf() + .set(BulkSparkConf.DISABLE_SSTABLE_VERSION_BASED_BRIDGE, "false"); + BulkSparkConf bulkConf = new BulkSparkConf(conf, defaultOptions); + assertThat(bulkConf.isSSTableVersionBasedBridgeDisabled()) + .describedAs("SSTable version-based bridge should be enabled") + .isFalse(); + } + + @Test + void testGetDisableSSTableVersionBasedBridgeDefaultValue() + { + // Create a SparkContext without setting the DISABLE_SSTABLE_VERSION_BASED_BRIDGE flag + SparkConf conf = new SparkConf() + .setMaster("local[1]") + .setAppName("test-static-get-default"); + SparkContext sc = new SparkContext(conf); + try + { + boolean result = BulkSparkConf.getDisableSSTableVersionBasedBridge(); + assertThat(result) + .describedAs("getDisableSSTableVersionBasedBridge should return false (enabled) when flag is not set") + .isFalse(); + } + finally + { + sc.stop(); + } + } + + @Test + void testGetDisableSSTableVersionBasedBridgeWhenDisabled() + { + // Create a SparkContext with DISABLE_SSTABLE_VERSION_BASED_BRIDGE set to true + SparkConf conf = new SparkConf() + .setMaster("local[1]") + .setAppName("test-static-get-disabled") + .set(BulkSparkConf.DISABLE_SSTABLE_VERSION_BASED_BRIDGE, "true"); + SparkContext sc = new SparkContext(conf); + try + { + boolean result = BulkSparkConf.getDisableSSTableVersionBasedBridge(); + assertThat(result) + .describedAs("getDisableSSTableVersionBasedBridge should return true (disabled) when flag is set to true") + .isTrue(); + } + finally + { + sc.stop(); + } + } + + @Test + void testGetDisableSSTableVersionBasedBridgeWhenEnabled() + { + // Create a SparkContext with DISABLE_SSTABLE_VERSION_BASED_BRIDGE explicitly set to false + SparkConf conf = new SparkConf() + .setMaster("local[1]") + .setAppName("test-static-get-enabled") + .set(BulkSparkConf.DISABLE_SSTABLE_VERSION_BASED_BRIDGE, "false"); + SparkContext sc = new SparkContext(conf); + try + { + boolean result = BulkSparkConf.getDisableSSTableVersionBasedBridge(); + assertThat(result) + .describedAs("getDisableSSTableVersionBasedBridge should return false (enabled) when flag is explicitly set to false") + .isFalse(); + } + finally + { + sc.stop(); + } + } + private Map copyDefaultOptions() { TreeMap map = new TreeMap<>(String.CASE_INSENSITIVE_ORDER); diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/CassandraClusterInfoTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/CassandraClusterInfoTest.java index 5ed4c5221..498a2119b 100644 --- a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/CassandraClusterInfoTest.java +++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/CassandraClusterInfoTest.java @@ -28,6 +28,7 @@ import org.junit.jupiter.api.Test; import o.a.c.sidecar.client.shaded.common.response.TimeSkewResponse; +import org.apache.cassandra.bridge.CassandraVersion; import org.apache.cassandra.spark.bulkwriter.token.TokenRangeMapping; import org.apache.cassandra.spark.exception.TimeSkewTooLargeException; @@ -81,7 +82,7 @@ private static class MockClusterInfoForTimeSkew extends CassandraClusterInfo MockClusterInfoForTimeSkew(int allowanceMinutes, Instant remoteNow) { - super((BulkSparkConf) null); + super((BulkSparkConf) null, CassandraVersion.FIVEZERO); mockCassandraContext(allowanceMinutes, remoteNow); } @@ -107,4 +108,6 @@ private void mockCassandraContext(int allowanceMinutes, Instant remoteNow) when(cassandraContext.sidecarPort()).thenReturn(9043); } } + } + diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/MockBulkWriterContext.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/MockBulkWriterContext.java index d9cefac43..98e01f743 100644 --- a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/MockBulkWriterContext.java +++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/MockBulkWriterContext.java @@ -43,6 +43,7 @@ import org.apache.cassandra.bridge.CassandraBridge; import org.apache.cassandra.bridge.CassandraBridgeFactory; +import org.apache.cassandra.bridge.CassandraVersion; import org.apache.cassandra.spark.bulkwriter.cloudstorage.coordinated.CoordinatedWriteConf; import org.apache.cassandra.spark.bulkwriter.token.ConsistencyLevel; import org.apache.cassandra.spark.bulkwriter.token.ReplicaAwareFailureHandler; @@ -376,12 +377,26 @@ public void checkBulkWriterIsEnabledOrThrow() BulkFeatures.BULK_WRITER)); } - @Override public String getLowestCassandraVersion() { return cassandraVersion; } + public Set getSSTableVersionsOnCluster() + { + // Return SSTable versions based on Cassandra version for testing + CassandraVersion version = CassandraVersion.fromVersion(cassandraVersion) + .orElse(CassandraVersion.FIVEZERO); + return new HashSet<>(version.getNativeSStableVersions()); + } + + @Override + public CassandraVersion bridgeVersion() + { + return CassandraVersion.fromVersion(cassandraVersion) + .orElse(CassandraVersion.FIVEZERO); + } + private List buildCompleteBatchIds(List uuids) { return uuids.stream().map(uuid -> uuid + "-" + jobId).collect(Collectors.toList()); diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/SortedSSTableWriterTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/SortedSSTableWriterTest.java index 6328f1252..027163fab 100644 --- a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/SortedSSTableWriterTest.java +++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/SortedSSTableWriterTest.java @@ -116,7 +116,7 @@ public void canCreateWriterForVersion(String version) throws IOException break; case 50: // Format is "oa--big" or "da--bti" - if ("big".equals(CassandraVersion.sstableFormat())) + if ("big".equals(CassandraVersion.configuredSSTableFormat())) { assertThat(baseFileName).matches("oa-\\d+-big"); } diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/TableSchemaTestCommon.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/TableSchemaTestCommon.java index 32fddffc5..b5c5bb69f 100644 --- a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/TableSchemaTestCommon.java +++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/TableSchemaTestCommon.java @@ -35,6 +35,7 @@ import org.apache.cassandra.bridge.CassandraBridge; import org.apache.cassandra.bridge.CassandraBridgeFactory; +import org.apache.cassandra.bridge.CassandraVersion; import org.apache.cassandra.spark.common.schema.ColumnType; import org.apache.cassandra.spark.data.CqlField; import org.apache.spark.sql.types.DataTypes; @@ -301,7 +302,9 @@ public TableSchema build() { dataFrameSchema = dataFrameSchema.add("timestamp", DataTypes.IntegerType); } - return new TableSchema(dataFrameSchema, tableInfoProvider, writeMode, ttlOption, timestampOption, cassandraVersion, quoteIdentifiers); + CassandraVersion bridgeVersion = CassandraVersion.fromVersion(cassandraVersion) + .orElseThrow(() -> new IllegalArgumentException("Unsupported Cassandra version: " + cassandraVersion)); + return new TableSchema(dataFrameSchema, tableInfoProvider, writeMode, ttlOption, timestampOption, bridgeVersion, quoteIdentifiers); } } diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/cloudstorage/coordinated/CassandraClusterInfoGroupTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/cloudstorage/coordinated/CassandraClusterInfoGroupTest.java index 4825c8dea..783f05968 100644 --- a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/cloudstorage/coordinated/CassandraClusterInfoGroupTest.java +++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/cloudstorage/coordinated/CassandraClusterInfoGroupTest.java @@ -34,6 +34,7 @@ import org.junit.jupiter.api.Test; import o.a.c.sidecar.client.shaded.common.response.TokenRangeReplicasResponse; +import org.apache.cassandra.bridge.CassandraVersion; import org.apache.cassandra.spark.bulkwriter.BroadcastableClusterInfoGroup; import org.apache.cassandra.spark.bulkwriter.BulkSparkConf; import org.apache.cassandra.spark.bulkwriter.CassandraClusterInfo; @@ -90,12 +91,10 @@ void testDelegationOfSingleCluster() () -> Partitioner.Murmur3Partitioner, RingInstance::new); when(clusterInfo.getTokenRangeMapping(anyBoolean())).thenReturn(expectedTokenRangeMapping); - when(clusterInfo.getLowestCassandraVersion()).thenReturn("lowestCassandraVersion"); when(clusterInfo.clusterWriteAvailability()).thenReturn(Collections.emptyMap()); CassandraClusterInfoGroup group = mockClusterGroup(1, index -> clusterInfo); // Since there is a single clusterInfo in the group. It behaves as a simple delegation to the sole clusterInfo assertThat(group.clusterWriteAvailability()).isSameAs(clusterInfo.clusterWriteAvailability()); - assertThat(group.getLowestCassandraVersion()).isSameAs(clusterInfo.getLowestCassandraVersion()); assertThat(group.getTokenRangeMapping(true)).isSameAs(clusterInfo.getTokenRangeMapping(true)); } @@ -138,30 +137,6 @@ void testAggregateWriteAvailability() .containsValues(WriteAvailability.AVAILABLE, WriteAvailability.UNAVAILABLE_DOWN); } - @Test - void testAggregateLowestCassandraVersion() - { - CassandraClusterInfoGroup goodGroup = mockClusterGroup(2, index -> { - CassandraClusterInfo clusterInfo = mockClusterInfo("cluster" + index); - when(clusterInfo.getLowestCassandraVersion()).thenReturn("4.0." + index); - return clusterInfo; - }); - assertThat(goodGroup.getLowestCassandraVersion()).isEqualTo("4.0.0"); - } - - @Test - void testAggregateLowestCassandraVersionFailDueToDifference() - { - CassandraClusterInfoGroup badGroup = mockClusterGroup(2, index -> { - CassandraClusterInfo clusterInfo = mockClusterInfo("cluster" + index); - when(clusterInfo.getLowestCassandraVersion()).thenReturn((4 + index) + ".0.0"); - return clusterInfo; - }); - assertThatThrownBy(badGroup::getLowestCassandraVersion) - .isExactlyInstanceOf(IllegalStateException.class) - .hasMessage("Cluster versions are not compatible. lowest=4.0.0 and highest=5.0.0"); - } - @Test void testCheckBulkWriterIsEnabledOrThrow() { @@ -240,7 +215,7 @@ void testTimeSkewTooLarge() void testCreateClusterInfoListFailsDueToAbsentConfiguration() { BulkSparkConf conf = mock(BulkSparkConf.class); - assertThatThrownBy(() -> fromBulkSparkConf(conf)) + assertThatThrownBy(() -> fromBulkSparkConf(conf, (CassandraVersion) null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("In order to create an instance of CassandraCoordinatedBulkWriterContext, " + "you must provide the appropriate coordinated write configuration by " + @@ -252,7 +227,7 @@ void testCreateClusterInfoListFailsDueToEmptyConfiguration() { BulkSparkConf conf = mock(BulkSparkConf.class, RETURNS_DEEP_STUBS); when(conf.coordinatedWriteConf().clusters()).thenReturn(Collections.emptyMap()); - assertThatThrownBy(() -> fromBulkSparkConf(conf)) + assertThatThrownBy(() -> fromBulkSparkConf(conf, (CassandraVersion) null)) .isInstanceOf(IllegalStateException.class) .hasMessageContaining("No cluster info is built from"); } @@ -263,7 +238,7 @@ void testCreateClusterInfoListFailsDueToEmptyClusterId() BulkSparkConf conf = mock(BulkSparkConf.class); CoordinatedWriteConf.ClusterConf clusterConf = new CoordinatedWriteConf.SimpleClusterConf(Collections.singletonList("localhost:9043"), "localDc"); when(conf.coordinatedWriteConf()).thenReturn(new CoordinatedWriteConf(Collections.singletonMap("", clusterConf))); - assertThatThrownBy(() -> fromBulkSparkConf(conf)) + assertThatThrownBy(() -> fromBulkSparkConf(conf, (CassandraVersion) null)) .isInstanceOf(IllegalStateException.class) .describedAs("The exception message should include the original json to help spot the wrong configuration (empty clusterId)") .hasMessage("Found coordinatedWriteConf with empty or null clusterId. " + @@ -277,7 +252,6 @@ void testSerDeser() CassandraClusterInfoGroup originalGroup = mockClusterGroup(2, index -> { CassandraClusterInfo clusterInfo = mockClusterInfo("cluster" + index); when(clusterInfo.getPartitioner()).thenReturn(Partitioner.Murmur3Partitioner); - when(clusterInfo.getLowestCassandraVersion()).thenReturn("4.0.0"); return clusterInfo; }); @@ -300,10 +274,6 @@ void testSerDeser() .describedAs("Partitioner should be preserved after serialization") .isEqualTo(Partitioner.Murmur3Partitioner); - assertThat(deserializedBroadcastable.getLowestCassandraVersion()) - .describedAs("Lowest Cassandra version should be preserved after serialization") - .isEqualTo("4.0.0"); - assertThat(deserializedBroadcastable.size()) .describedAs("Number of clusters should be preserved after serialization") .isEqualTo(2); diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/CassandraDataLayerValidationTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/CassandraDataLayerValidationTest.java new file mode 100644 index 000000000..dc3dbba8e --- /dev/null +++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/CassandraDataLayerValidationTest.java @@ -0,0 +1,467 @@ +/* + * 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.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import org.junit.jupiter.api.Test; + +import org.apache.cassandra.bridge.CassandraVersion; +import org.apache.cassandra.clients.Sidecar; +import org.apache.cassandra.spark.data.partitioner.ConsistencyLevel; +import org.apache.cassandra.spark.data.partitioner.TokenPartitioner; +import org.apache.cassandra.spark.utils.TimeProvider; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatNoException; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * Unit tests for CassandraDataLayer validation methods + */ +public class CassandraDataLayerValidationTest +{ + @Test + void testValidateSStableVersionsWithAllSupportedVersions() + { + CassandraDataLayer dataLayer = createTestDataLayer(); + Set sstableVersions = new HashSet<>(Arrays.asList("big-na", "big-nb")); + + assertThatNoException() + .describedAs("All versions are supported by FOURZERO") + .isThrownBy(() -> dataLayer.validateSStableVersions(sstableVersions, CassandraVersion.FOURZERO, false)); + } + + @Test + void testValidateSStableVersionsWithUnsupportedVersion() + { + CassandraDataLayer dataLayer = createTestDataLayer(); + // C* 4.0 cannot read C* 5.0 SSTable versions + Set sstableVersions = new HashSet<>(Arrays.asList("big-na", "big-oa")); + + assertThatThrownBy(() -> dataLayer.validateSStableVersions(sstableVersions, CassandraVersion.FOURZERO, false)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("Detected unsupported SSTable version(s)") + .hasMessageContaining("big-oa") + .hasMessageContaining("FOURZERO"); + } + + @Test + void testValidateSStableVersionsWithNullVersionsThrowsException() + { + CassandraDataLayer dataLayer = createTestDataLayer(); + + assertThatThrownBy(() -> dataLayer.validateSStableVersions(null, CassandraVersion.FOURZERO, false)) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("Unable to retrieve SSTable versions from cluster"); + } + + @Test + void testValidateSStableVersionsWithEmptyVersionsThrowsException() + { + CassandraDataLayer dataLayer = createTestDataLayer(); + + assertThatThrownBy(() -> dataLayer.validateSStableVersions(Collections.emptySet(), CassandraVersion.FOURZERO, false)) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("Unable to retrieve SSTable versions from cluster"); + } + + @Test + void testValidateSStableVersionsSkipsValidationWhenFallbackEnabled() + { + CassandraDataLayer dataLayer = createTestDataLayer(); + + // Test with invalid versions - should not throw when fallback enabled + Set invalidVersions = new HashSet<>(List.of("invalid-version")); + assertThatNoException() + .describedAs("Validation should be skipped with invalid versions when fallback mode is enabled") + .isThrownBy(() -> dataLayer.validateSStableVersions(invalidVersions, CassandraVersion.FOURZERO, true)); + + // Test with null versions - should not throw when fallback enabled + assertThatNoException() + .describedAs("Validation should be skipped with null versions when fallback enabled") + .isThrownBy(() -> dataLayer.validateSStableVersions(null, CassandraVersion.FOURZERO, true)); + } + + @Test + void testValidateSStableVersionsForFiveZeroWithBackwardCompatibility() + { + CassandraDataLayer dataLayer = createTestDataLayer(); + // C* 5.0 should be able to read C* 4.0 SSTable versions + Set sstableVersions = new HashSet<>(Arrays.asList("big-na", "big-nb", "big-oa")); + + assertThatNoException() + .describedAs("FIVEZERO should support reading FOURZERO versions") + .isThrownBy(() -> dataLayer.validateSStableVersions(sstableVersions, CassandraVersion.FIVEZERO, false)); + } + + @Test + void testValidateSStableVersionsForFiveZeroWithBtiFormat() + { + CassandraDataLayer dataLayer = createTestDataLayer(); + Set sstableVersions = new HashSet<>(Arrays.asList("big-oa", "bti-da")); + + assertThatNoException() + .describedAs("FIVEZERO should support both big and bti formats") + .isThrownBy(() -> dataLayer.validateSStableVersions(sstableVersions, CassandraVersion.FIVEZERO, false)); + } + + @Test + void testValidateSStableVersionsErrorMessageIncludesAllDetails() + { + CassandraDataLayer dataLayer = createTestDataLayer(); + Set sstableVersions = new HashSet<>(List.of("big-oa")); + + assertThatThrownBy(() -> dataLayer.validateSStableVersions(sstableVersions, CassandraVersion.FOURZERO, false)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("Detected unsupported SSTable version(s)") + .hasMessageContaining("Supported versions:") + .hasMessageContaining("Observed SSTable versions in the cluster:"); + } + + @Test + void testValidateSStableVersionsListWithValidVersions() + { + Set expectedVersions = new HashSet<>(Arrays.asList("big-na", "big-nb")); + CassandraDataLayer dataLayer = createTestDataLayerWithVersions(expectedVersions); + + SSTable ssTable1 = createMockSSTable("big", "na", "test1-big-na-Data.db"); + SSTable ssTable2 = createMockSSTable("big", "nb", "test2-big-nb-Data.db"); + List sstables = Arrays.asList(ssTable1, ssTable2); + + assertThatNoException() + .describedAs("All SSTables have expected versions") + .isThrownBy(() -> dataLayer.validateSStableVersions(sstables)); + } + + @Test + void testValidateSStableVersionsListWithUnexpectedVersion() + { + Set expectedVersions = new HashSet<>(List.of("big-na")); + CassandraDataLayer dataLayer = createTestDataLayerWithVersions(expectedVersions); + + SSTable ssTable1 = createMockSSTable("big", "na", "test1-big-na-Data.db"); + SSTable ssTable2 = createMockSSTable("big", "nb", "test2-big-nb-Data.db"); + List sstables = Arrays.asList(ssTable1, ssTable2); + + assertThatThrownBy(() -> dataLayer.validateSStableVersions(sstables)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("has version 'big-nb' which was not observed in cluster gossip info") + .hasMessageContaining("test2-big-nb-Data.db"); + } + + @Test + void testValidateSStableVersionsListWithEmptyList() + { + Set expectedVersions = new HashSet<>(List.of("big-na")); + CassandraDataLayer dataLayer = createTestDataLayerWithVersions(expectedVersions); + + assertThatNoException() + .describedAs("Empty SSTable list should not throw exception") + .isThrownBy(() -> dataLayer.validateSStableVersions(Collections.emptyList())); + } + + @Test + void testValidateSStableVersionsListErrorMessageIncludesFileName() + { + Set expectedVersions = new HashSet<>(List.of("big-na")); + CassandraDataLayer dataLayer = createTestDataLayerWithVersions(expectedVersions); + + SSTable ssTable = createMockSSTable("big", "oa", "keyspace-table-big-oa-Data.db"); + List sstables = Collections.singletonList(ssTable); + + assertThatThrownBy(() -> dataLayer.validateSStableVersions(sstables)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("keyspace-table-big-oa-Data.db") + .hasMessageContaining("Expected versions from gossip:"); + } + + @Test + void testValidateSStableVersionsListWithMixedBigAndBtiFormats() + { + Set expectedVersions = new HashSet<>(Arrays.asList("big-oa", "bti-da")); + CassandraDataLayer dataLayer = createTestDataLayerWithVersions(expectedVersions); + + SSTable ssTable1 = createMockSSTable("big", "oa", "test1-big-oa-Data.db"); + SSTable ssTable2 = createMockSSTable("bti", "da", "test2-bti-da-Data.db"); + List sstables = Arrays.asList(ssTable1, ssTable2); + + assertThatNoException() + .describedAs("Mixed big and bti format SSTables should be validated successfully") + .isThrownBy(() -> dataLayer.validateSStableVersions(sstables)); + } + + @Test + void testValidateSStableVersionsListWithUnexpectedBtiVersion() + { + Set expectedVersions = new HashSet<>(List.of("big-oa")); + CassandraDataLayer dataLayer = createTestDataLayerWithVersions(expectedVersions); + + SSTable ssTable1 = createMockSSTable("big", "oa", "test1-big-oa-Data.db"); + SSTable ssTable2 = createMockSSTable("bti", "da", "keyspace-table-bti-da-Data.db"); + List sstables = Arrays.asList(ssTable1, ssTable2); + + assertThatThrownBy(() -> dataLayer.validateSStableVersions(sstables)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("has version 'bti-da' which was not observed in cluster gossip info") + .hasMessageContaining("keyspace-table-bti-da-Data.db") + .hasMessageContaining("Expected versions from gossip:"); + } + + @Test + void testValidateSStableVersionsListSkipsValidationWhenFeatureDisabled() + { + Set expectedVersions = new HashSet<>(List.of("big-na")); + // Create a special test instance that returns true for isSSTableVersionBasedBridgeDisabled + CassandraDataLayer dataLayer = new TestCassandraDataLayerWithFeatureDisabled(expectedVersions); + + // Create SSTable with unexpected version + SSTable ssTable = createMockSSTable("big", "oa", "test-big-oa-Data.db"); + List sstables = Collections.singletonList(ssTable); + + // Should NOT throw exception because feature is disabled + assertThatNoException() + .describedAs("Validation should be skipped when feature is disabled") + .isThrownBy(() -> dataLayer.validateSStableVersions(sstables)); + } + + @Test + void testValidateSStableVersionsListWithEmptyExpectedVersionsThrowsException() + { + CassandraDataLayer dataLayer = createTestDataLayerWithVersions(Collections.emptySet()); + + SSTable ssTable = createMockSSTable("big", "na", "test-big-na-Data.db"); + List sstables = Collections.singletonList(ssTable); + + assertThatThrownBy(() -> dataLayer.validateSStableVersions(sstables)) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("Unable to validate SSTable versions - no expected versions available from cluster"); + } + + // Tests for initializeSSTableVersionsAndBridgeVersion (called from initBulkReader - lines 283-313) + + @Test + void testInitializeSSTableVersionsAndBridgeVersionWithFeatureEnabled() + { + Set mockVersions = new HashSet<>(Arrays.asList("big-na", "big-nb")); + CassandraDataLayer dataLayer = new TestCassandraDataLayerForInitTests(mockVersions, false); + + CassandraVersion result = dataLayer.initializeSSTableVersionsAndBridgeVersion("4.0.0"); + + // Should return FOURZERO bridge version + assertThat((Object) result).isEqualTo(CassandraVersion.FOURZERO); + + // Should set sstableVersionsOnCluster to the retrieved versions + assertThat((Object) dataLayer.sstableVersionsOnCluster) + .isNotNull(); + assertThat(dataLayer.sstableVersionsOnCluster) + .containsExactlyInAnyOrder("big-na", "big-nb"); + } + + @Test + void testInitializeSSTableVersionsAndBridgeVersionWithFeatureDisabled() + { + // Versions won't be retrieved when feature is disabled + CassandraDataLayer dataLayer = new TestCassandraDataLayerForInitTests(null, true); + + CassandraVersion result = dataLayer.initializeSSTableVersionsAndBridgeVersion("4.0.0"); + + // Should return FOURZERO bridge version (fallback to cassandra.version) + assertThat((Object) result).isEqualTo(CassandraVersion.FOURZERO); + + // Should set sstableVersionsOnCluster to null (skipped retrieval) + assertThat((Object) dataLayer.sstableVersionsOnCluster).isNull(); + } + + @Test + void testInitializeSSTableVersionsAndBridgeVersionSelectsFiveZeroBridge() + { + Set mockVersions = new HashSet<>(Arrays.asList("big-oa", "bti-da")); + CassandraDataLayer dataLayer = new TestCassandraDataLayerForInitTests(mockVersions, false); + + CassandraVersion result = dataLayer.initializeSSTableVersionsAndBridgeVersion("5.0.0"); + + // Should return FIVEZERO bridge version based on highest SSTable version + assertThat((Object) result).isEqualTo(CassandraVersion.FIVEZERO); + + // Should set sstableVersionsOnCluster to the retrieved versions + assertThat((Object) dataLayer.sstableVersionsOnCluster) + .isNotNull(); + assertThat(dataLayer.sstableVersionsOnCluster) + .containsExactlyInAnyOrder("big-oa", "bti-da"); + } + + @Test + void testInitializeSSTableVersionsAndBridgeVersionWithMixedVersions() + { + // Cluster has both C* 4.0 and C* 5.0 versions (during rolling upgrade) + Set mockVersions = new HashSet<>(Arrays.asList("big-na", "big-oa")); + CassandraDataLayer dataLayer = new TestCassandraDataLayerForInitTests(mockVersions, false); + + CassandraVersion result = dataLayer.initializeSSTableVersionsAndBridgeVersion("5.0.0"); + + // Should return FIVEZERO bridge (highest version) + assertThat((Object) result).isEqualTo(CassandraVersion.FIVEZERO); + + // Should set sstableVersionsOnCluster to all retrieved versions + assertThat((Object) dataLayer.sstableVersionsOnCluster) + .isNotNull(); + assertThat(dataLayer.sstableVersionsOnCluster) + .containsExactlyInAnyOrder("big-na", "big-oa"); + } + + private CassandraDataLayer createTestDataLayer() + { + // Use TestCassandraDataLayer to avoid SparkContext initialization in unit tests + return new TestCassandraDataLayer(null); + } + + private CassandraDataLayer createTestDataLayerWithVersions(Set sstableVersions) + { + return new TestCassandraDataLayer(sstableVersions); + } + + private SSTable createMockSSTable(String format, String version, String fileName) + { + SSTable ssTable = mock(SSTable.class); + when(ssTable.getFormat()).thenReturn(format); + when(ssTable.getVersion()).thenReturn(version); + when(ssTable.getDataFileName()).thenReturn(fileName); + return ssTable; + } + + /** + * Test subclass to allow setting sstableVersionsOnCluster for testing + * This subclass uses the serialization constructor and overrides problematic methods + */ + private static class TestCassandraDataLayer extends CassandraDataLayer + { + TestCassandraDataLayer(Set sstableVersions) + { + // Use the serialization constructor which doesn't call initialize() + super("test_keyspace", // keyspace + "test_table", // table + false, // quoteIdentifiers + "", // snapshotName + null, // datacenter + Sidecar.ClientConfig.create(), // sidecarClientConfig + null, // sslConfig + mock(CqlTable.class), // cqlTable + mock(TokenPartitioner.class), // tokenPartitioner + CassandraVersion.FOURZERO, // bridgeVersion + ConsistencyLevel.LOCAL_QUORUM, // consistencyLevel + "127.0.0.1", // sidecarInstances + 9043, // sidecarPort + Collections.emptyMap(), // availabilityHints + Collections.emptyMap(), // bigNumberConfigMap + false, // enableStats + false, // readIndexOffset + false, // useIncrementalRepair + null, // lastModifiedTimestampField + Collections.emptyList(), // requestedFeatures + Collections.emptyMap(), // rfMap + mock(TimeProvider.class), // timeProvider + null, // sstableTimeRangeFilter + sstableVersions); // sstableVersionsOnCluster + this.sstableVersionsOnCluster = sstableVersions; + } + + @Override + public void startupValidate() + { + // Skip startup validation in tests to avoid SparkContext initialization + } + + @Override + protected void initSidecarClient() + { + // Skip sidecar client initialization in tests + } + + @Override + protected void initInstanceMap() + { + // Skip instance map initialization in tests + } + + @Override + protected boolean isSSTableVersionBasedBridgeDisabled() + { + // Override to avoid calling BulkSparkConf.getDisableSSTableVersionBasedBridge() + // which would try to initialize SparkContext in unit tests + // Always return false to test the validation logic + return false; + } + } + + /** + * Test subclass with feature disabled to test the skip validation path + */ + private static class TestCassandraDataLayerWithFeatureDisabled extends TestCassandraDataLayer + { + TestCassandraDataLayerWithFeatureDisabled(Set sstableVersions) + { + super(sstableVersions); + } + + @Override + protected boolean isSSTableVersionBasedBridgeDisabled() + { + // Return true to test the feature-disabled path + return true; + } + } + + /** + * Test subclass for testing initializeSSTableVersionsAndBridgeVersion method + * Mocks the Sidecar interaction to avoid actual cluster calls + */ + private static class TestCassandraDataLayerForInitTests extends TestCassandraDataLayer + { + private final Set mockSSTableVersions; + private final boolean featureDisabled; + + TestCassandraDataLayerForInitTests(Set mockSSTableVersions, boolean featureDisabled) + { + super(null); + this.mockSSTableVersions = mockSSTableVersions; + this.featureDisabled = featureDisabled; + } + + @Override + protected boolean isSSTableVersionBasedBridgeDisabled() + { + return featureDisabled; + } + + @Override + protected Set retrieveSSTableVersionsFromCluster() + { + // Mock the Sidecar call - return the mock versions instead of calling actual sidecar + return mockSSTableVersions; + } + } +} diff --git a/cassandra-analytics-integration-tests/src/test/java/org/apache/cassandra/analytics/BulkReaderSSTableVersionBridgeDisabledTest.java b/cassandra-analytics-integration-tests/src/test/java/org/apache/cassandra/analytics/BulkReaderSSTableVersionBridgeDisabledTest.java new file mode 100644 index 000000000..6a601a3d2 --- /dev/null +++ b/cassandra-analytics-integration-tests/src/test/java/org/apache/cassandra/analytics/BulkReaderSSTableVersionBridgeDisabledTest.java @@ -0,0 +1,107 @@ +/* + * 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.analytics; + +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.stream.Collectors; + +import org.junit.jupiter.api.Test; + +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.IInstance; +import org.apache.cassandra.sidecar.testing.QualifiedName; +import org.apache.cassandra.testing.ClusterBuilderConfiguration; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; + +import static org.apache.cassandra.testing.TestUtils.DC1_RF3; +import static org.apache.cassandra.testing.TestUtils.TEST_KEYSPACE; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Integration tests for SSTable version-based bridge selection in bulk reader (feature DISABLED) + * This test class has the feature flag disabled via SparkConf override. + */ +class BulkReaderSSTableVersionBridgeDisabledTest extends SharedClusterSparkIntegrationTestBase +{ + static final List DATASET = Arrays.asList("alpha", "beta", "gamma", "delta", "epsilon"); + static final QualifiedName TABLE_DISABLED = new QualifiedName(TEST_KEYSPACE, "test_disabled"); + + @Override + protected ClusterBuilderConfiguration testClusterConfiguration() + { + return super.testClusterConfiguration() + .nodesPerDc(3); + } + + @Override + protected SparkConf getOrCreateSparkConf() + { + SparkConf conf = super.getOrCreateSparkConf(); + // Disable SSTable version-based bridge selection for this entire test class + conf.set("spark.cassandra_analytics.bridge.disable_sstable_version_based", "true"); + return conf; + } + + @Override + protected void initializeSchemaForTest() + { + createTestKeyspace(TEST_KEYSPACE, DC1_RF3); + + // Create table + createTestTable(TABLE_DISABLED, "CREATE TABLE %s (id int PRIMARY KEY, value text) WITH compression = {'enabled': false};"); + + IInstance firstRunningInstance = cluster.getFirstRunningInstance(); + + // Insert data for TABLE_DISABLED + for (int i = 0; i < DATASET.size(); i++) + { + String query = String.format("INSERT INTO %s (id, value) VALUES (%d, '%s');", TABLE_DISABLED, i, DATASET.get(i)); + firstRunningInstance.coordinator().execute(query, ConsistencyLevel.ALL); + } + + // Flush to create SSTables + cluster.stream().forEach(instance -> { + instance.nodetool("flush", TEST_KEYSPACE, TABLE_DISABLED.table()); + }); + } + + @Test + void testBulkReadWithSSTableVersionBasedBridgeDisabled() + { + // Read with feature disabled (fallback to cassandra.version) + Dataset df = bulkReaderDataFrame(TABLE_DISABLED).load(); + + List rows = df.collectAsList() + .stream() + .sorted(Comparator.comparing(row -> row.getInt(0))) + .collect(Collectors.toList()); + + assertThat(rows).hasSize(DATASET.size()); + for (int i = 0; i < DATASET.size(); i++) + { + assertThat(rows.get(i).getInt(0)).isEqualTo(i); + assertThat(rows.get(i).getString(1)).isEqualTo(DATASET.get(i)); + } + } +} diff --git a/cassandra-analytics-integration-tests/src/test/java/org/apache/cassandra/analytics/BulkReaderSSTableVersionBridgeEnabledTest.java b/cassandra-analytics-integration-tests/src/test/java/org/apache/cassandra/analytics/BulkReaderSSTableVersionBridgeEnabledTest.java new file mode 100644 index 000000000..295359fa6 --- /dev/null +++ b/cassandra-analytics-integration-tests/src/test/java/org/apache/cassandra/analytics/BulkReaderSSTableVersionBridgeEnabledTest.java @@ -0,0 +1,160 @@ +/* + * 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.analytics; + +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.stream.Collectors; + +import org.junit.jupiter.api.Test; + +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.IInstance; +import org.apache.cassandra.sidecar.testing.QualifiedName; +import org.apache.cassandra.testing.ClusterBuilderConfiguration; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; + +import static org.apache.cassandra.testing.TestUtils.DC1_RF3; +import static org.apache.cassandra.testing.TestUtils.TEST_KEYSPACE; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Integration tests for SSTable version-based bridge selection in bulk reader (feature ENABLED) + * This test class has the feature explicitly enabled via SparkConf override. + */ +class BulkReaderSSTableVersionBridgeEnabledTest extends SharedClusterSparkIntegrationTestBase +{ + static final List DATASET = Arrays.asList("alpha", "beta", "gamma", "delta", "epsilon"); + static final QualifiedName TABLE_MULTIPLE = new QualifiedName(TEST_KEYSPACE, "test_multiple"); + static final QualifiedName TABLE_SNAPSHOT = new QualifiedName(TEST_KEYSPACE, "test_snapshot"); + + @Override + protected ClusterBuilderConfiguration testClusterConfiguration() + { + return super.testClusterConfiguration() + .nodesPerDc(3); + } + + @Override + protected SparkConf getOrCreateSparkConf() + { + SparkConf conf = super.getOrCreateSparkConf(); + // Explicitly enable SSTable version-based bridge selection for this entire test class + conf.set("spark.cassandra_analytics.bridge.disable_sstable_version_based", "false"); + return conf; + } + + @Override + protected void initializeSchemaForTest() + { + createTestKeyspace(TEST_KEYSPACE, DC1_RF3); + + // Create tables + createTestTable(TABLE_MULTIPLE, "CREATE TABLE %s (id int PRIMARY KEY, value text) WITH compression = {'enabled': false};"); + createTestTable(TABLE_SNAPSHOT, "CREATE TABLE %s (id int PRIMARY KEY, value text) WITH compression = {'enabled': false};"); + + IInstance firstRunningInstance = cluster.getFirstRunningInstance(); + + // Insert data for TABLE_MULTIPLE (3 batches to create multiple SSTables) + for (int batch = 0; batch < 3; batch++) + { + for (int i = batch * 10; i < (batch + 1) * 10; i++) + { + String query = String.format("INSERT INTO %s (id, value) VALUES (%d, 'value_%d');", TABLE_MULTIPLE, i, i); + firstRunningInstance.coordinator().execute(query, ConsistencyLevel.ALL); + } + // Flush after each batch to create separate SSTables + cluster.stream().forEach(instance -> instance.nodetool("flush", TEST_KEYSPACE, TABLE_MULTIPLE.table())); + } + + // Insert data for TABLE_SNAPSHOT + for (int i = 0; i < DATASET.size(); i++) + { + String query = String.format("INSERT INTO %s (id, value) VALUES (%d, '%s');", TABLE_SNAPSHOT, i, DATASET.get(i)); + firstRunningInstance.coordinator().execute(query, ConsistencyLevel.ALL); + } + + // Flush TABLE_SNAPSHOT to create SSTables + cluster.stream().forEach(instance -> { + instance.nodetool("flush", TEST_KEYSPACE, TABLE_SNAPSHOT.table()); + }); + } + + @Test + void testBulkReadWithSSTableVersionBasedBridgeMultipleSSTables() + { + // Read with SSTable version-based bridge enabled + Dataset df = bulkReaderDataFrame(TABLE_MULTIPLE).load(); + + List rows = df.collectAsList(); + assertThat(rows).hasSize(30); // 3 batches * 10 rows each + + // Verify all rows are present + List ids = rows.stream() + .map(row -> row.getInt(0)) + .sorted() + .collect(Collectors.toList()); + for (int i = 0; i < 30; i++) + { + assertThat(ids.get(i)).isEqualTo(i); + } + } + + @Test + void testBulkReadWithSnapshot() + { + String snapshotName = "test_snapshot_" + System.currentTimeMillis(); + + // Create snapshot on all nodes + cluster.stream().forEach(instance -> { + instance.nodetool("snapshot", "-t", snapshotName, TEST_KEYSPACE); + }); + + try + { + // Read from snapshot with SSTable version-based bridge + Dataset df = bulkReaderDataFrame(TABLE_SNAPSHOT) + .option("snapshotName", snapshotName) + .load(); + + List rows = df.collectAsList() + .stream() + .sorted(Comparator.comparing(row -> row.getInt(0))) + .collect(Collectors.toList()); + + assertThat(rows).hasSize(DATASET.size()); + for (int i = 0; i < DATASET.size(); i++) + { + assertThat(rows.get(i).getInt(0)).isEqualTo(i); + assertThat(rows.get(i).getString(1)).isEqualTo(DATASET.get(i)); + } + } + finally + { + // Clean up snapshot + cluster.stream().forEach(instance -> { + instance.nodetool("clearsnapshot", "-t", snapshotName, TEST_KEYSPACE); + }); + } + } +} diff --git a/cassandra-analytics-integration-tests/src/test/java/org/apache/cassandra/analytics/BulkWriterSSTableVersionBridgeDisabledTest.java b/cassandra-analytics-integration-tests/src/test/java/org/apache/cassandra/analytics/BulkWriterSSTableVersionBridgeDisabledTest.java new file mode 100644 index 000000000..5b97d60d1 --- /dev/null +++ b/cassandra-analytics-integration-tests/src/test/java/org/apache/cassandra/analytics/BulkWriterSSTableVersionBridgeDisabledTest.java @@ -0,0 +1,117 @@ +/* + * 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.analytics; + +import java.util.Arrays; +import java.util.List; + +import org.junit.jupiter.api.Test; + +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.SimpleQueryResult; +import org.apache.cassandra.sidecar.testing.QualifiedName; +import org.apache.cassandra.testing.ClusterBuilderConfiguration; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +import static org.apache.cassandra.testing.TestUtils.DC1_RF3; +import static org.apache.cassandra.testing.TestUtils.TEST_KEYSPACE; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Integration tests for SSTable version-based bridge selection in bulk writer, with feature DISABLED + */ +class BulkWriterSSTableVersionBridgeDisabledTest extends SharedClusterSparkIntegrationTestBase +{ + static final QualifiedName TABLE_DISABLED = new QualifiedName(TEST_KEYSPACE, "test_disabled"); + + @Override + protected ClusterBuilderConfiguration testClusterConfiguration() + { + return super.testClusterConfiguration() + .nodesPerDc(3); + } + + @Override + protected SparkConf getOrCreateSparkConf() + { + SparkConf conf = super.getOrCreateSparkConf(); + // Disable SSTable version-based bridge selection for this entire test class + conf.set("spark.cassandra_analytics.bridge.disable_sstable_version_based", "true"); + return conf; + } + + @Override + protected void initializeSchemaForTest() + { + createTestKeyspace(TEST_KEYSPACE, DC1_RF3); + cluster.schemaChangeIgnoringStoppedInstances( + "CREATE TABLE " + TABLE_DISABLED + " (id int PRIMARY KEY, value text) WITH compression = {'enabled': false};" + ); + } + + @Test + void testBulkWriteWithSSTableVersionBasedBridgeDisabled() + { + SparkSession spark = getOrCreateSparkSession(); + + // Create test data + List data = Arrays.asList( + RowFactory.create(0, "x"), + RowFactory.create(1, "y"), + RowFactory.create(2, "z") + ); + + StructType schema = DataTypes.createStructType(new StructField[]{ + DataTypes.createStructField("id", DataTypes.IntegerType, false), + DataTypes.createStructField("value", DataTypes.StringType, false) + }); + + Dataset df = spark.createDataFrame(data, schema); + + // Write with feature disabled (fallback to cassandra.version) + bulkWriterDataFrameWriter(df, TABLE_DISABLED).save(); + + // Verify data was written + SimpleQueryResult result = cluster.coordinator(1) + .executeWithResult("SELECT id, value FROM " + TABLE_DISABLED, + ConsistencyLevel.ALL); + Object[][] rows = result.toObjectArrays(); + + assertThat(rows.length).isEqualTo(data.size()); + + // Verify all expected IDs are present + List actualIds = Arrays.stream(rows) + .map(row -> (Integer) row[0]) + .sorted() + .collect(java.util.stream.Collectors.toList()); + List expectedIds = data.stream() + .map(row -> row.getInt(0)) + .sorted() + .collect(java.util.stream.Collectors.toList()); + assertThat(actualIds).isEqualTo(expectedIds); + } +} diff --git a/cassandra-analytics-integration-tests/src/test/java/org/apache/cassandra/analytics/BulkWriterSSTableVersionBridgeEnabledTest.java b/cassandra-analytics-integration-tests/src/test/java/org/apache/cassandra/analytics/BulkWriterSSTableVersionBridgeEnabledTest.java new file mode 100644 index 000000000..f73254d0a --- /dev/null +++ b/cassandra-analytics-integration-tests/src/test/java/org/apache/cassandra/analytics/BulkWriterSSTableVersionBridgeEnabledTest.java @@ -0,0 +1,109 @@ +/* + * 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.analytics; + +import java.util.Arrays; +import java.util.List; + +import org.junit.jupiter.api.Test; + +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.SimpleQueryResult; +import org.apache.cassandra.sidecar.testing.QualifiedName; +import org.apache.cassandra.testing.ClusterBuilderConfiguration; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +import static org.apache.cassandra.testing.TestUtils.DC1_RF3; +import static org.apache.cassandra.testing.TestUtils.TEST_KEYSPACE; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Integration tests for SSTable version-based bridge selection in bulk writer (feature ENABLED) + */ +class BulkWriterSSTableVersionBridgeEnabledTest extends SharedClusterSparkIntegrationTestBase +{ + static final QualifiedName TABLE_ENABLED = new QualifiedName(TEST_KEYSPACE, "test_enabled"); + + @Override + protected ClusterBuilderConfiguration testClusterConfiguration() + { + return super.testClusterConfiguration() + .nodesPerDc(3); + } + + @Override + protected void initializeSchemaForTest() + { + createTestKeyspace(TEST_KEYSPACE, DC1_RF3); + cluster.schemaChangeIgnoringStoppedInstances( + "CREATE TABLE " + TABLE_ENABLED + " (id int PRIMARY KEY, value text) WITH compression = {'enabled': false};" + ); + } + + @Test + void testBulkWriteWithSSTableVersionBasedBridgeEnabled() + { + SparkSession spark = getOrCreateSparkSession(); + + // Create test data + List data = Arrays.asList( + RowFactory.create(0, "a"), + RowFactory.create(1, "b"), + RowFactory.create(2, "c"), + RowFactory.create(3, "d"), + RowFactory.create(4, "e") + ); + + StructType schema = DataTypes.createStructType(new StructField[]{ + DataTypes.createStructField("id", DataTypes.IntegerType, false), + DataTypes.createStructField("value", DataTypes.StringType, false) + }); + + Dataset df = spark.createDataFrame(data, schema); + + // Write with SSTable version-based bridge enabled (default) + bulkWriterDataFrameWriter(df, TABLE_ENABLED).save(); + + // Verify data was written + SimpleQueryResult result = cluster.coordinator(1) + .executeWithResult("SELECT id, value FROM " + TABLE_ENABLED, + ConsistencyLevel.ALL); + Object[][] rows = result.toObjectArrays(); + + assertThat(rows.length).isEqualTo(data.size()); + + // Verify all expected IDs are present + List actualIds = Arrays.stream(rows) + .map(row -> (Integer) row[0]) + .sorted() + .collect(java.util.stream.Collectors.toList()); + List expectedIds = data.stream() + .map(row -> row.getInt(0)) + .sorted() + .collect(java.util.stream.Collectors.toList()); + assertThat(actualIds).isEqualTo(expectedIds); + } +} diff --git a/cassandra-analytics-sidecar-client/build.gradle b/cassandra-analytics-sidecar-client/build.gradle index bedf2c042..ad0004bae 100644 --- a/cassandra-analytics-sidecar-client/build.gradle +++ b/cassandra-analytics-sidecar-client/build.gradle @@ -41,4 +41,23 @@ dependencies { implementation "org.slf4j:slf4j-api:${slf4jApiVersion}" api(project(path: ':analytics-sidecar-vertx-client-shaded', configuration: 'shadow')) -} \ No newline at end of file + + // Test dependencies + testImplementation("org.junit.jupiter:junit-jupiter-api:${project.junitVersion}") + testImplementation("org.junit.jupiter:junit-jupiter-params:${project.junitVersion}") + testImplementation("org.junit.jupiter:junit-jupiter-engine:${project.junitVersion}") + testImplementation("org.assertj:assertj-core:${assertjCoreVersion}") + testImplementation(group: 'org.mockito', name: 'mockito-core', version: "${project.rootProject.mockitoVersion}") + testImplementation(group: 'org.mockito', name: 'mockito-inline', version: "${project.rootProject.mockitoVersion}") +} + +test { + useJUnitPlatform() + testLogging { + events "passed", "skipped", "failed" + showExceptions true + exceptionFormat "full" + showCauses true + showStackTraces true + } +} diff --git a/cassandra-analytics-sidecar-client/src/main/java/org/apache/cassandra/clients/Sidecar.java b/cassandra-analytics-sidecar-client/src/main/java/org/apache/cassandra/clients/Sidecar.java index d5922c2d3..6b905fc84 100644 --- a/cassandra-analytics-sidecar-client/src/main/java/org/apache/cassandra/clients/Sidecar.java +++ b/cassandra-analytics-sidecar-client/src/main/java/org/apache/cassandra/clients/Sidecar.java @@ -20,15 +20,18 @@ package org.apache.cassandra.clients; import java.io.IOException; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import o.a.c.sidecar.client.shaded.common.response.GossipInfoResponse; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,6 +53,7 @@ import org.apache.cassandra.spark.common.model.CassandraInstance; import org.apache.cassandra.spark.data.FileType; import org.apache.cassandra.spark.utils.BuildInfo; +import org.apache.cassandra.spark.utils.FutureUtils; import org.apache.cassandra.spark.utils.MapUtils; import org.apache.cassandra.spark.validation.KeyStoreValidation; import org.apache.cassandra.spark.validation.StartupValidator; @@ -149,6 +153,79 @@ public static List> allNodeSettings(SidecarClien .collect(Collectors.toList()); } + /** + * Retrieve gossip info from all nodes on the cluster + * @param client Sidecar client + * @param instances all Sidecar instances + * @return completable futures with GossipInfoResponse + */ + public static List> gossipInfoFromAllNodes(SidecarClient client, + Set instances) + { + return instances.stream() + .map(instance -> client + .gossipInfo(instance) + .exceptionally(throwable -> { + LOGGER.warn(String.format("Failed to retrieve gossipinfo from instance=%s", + instance), throwable); + return null; + })) + .collect(Collectors.toList()); + } + + /** + * Retrieves SSTable versions from all nodes in the cluster via gossip info. + * This method fetches gossip information from all Sidecar instances and extracts + * the SSTable versions running on each node. + * + * @param client Sidecar client + * @param instances all Sidecar instances in the cluster + * @param maxRetryDelayMillis maximum delay in milliseconds between retries + * @param maxRetries maximum number of retry attempts + * @return a set of SSTable versions across all nodes in the cluster + * @throws RuntimeException if unable to retrieve gossip info from any nodes + */ + public static Set getSSTableVersionsFromCluster(SidecarClient client, + Set instances, + long maxRetryDelayMillis, + int maxRetries) + { + LOGGER.debug("Retrieving SSTable versions from cluster via gossip..."); + + List> gossipInfoFutures = + gossipInfoFromAllNodes(client, instances); + + // Calculate total timeout + final long totalTimeout = maxRetryDelayMillis * maxRetries * gossipInfoFutures.size(); + + List gossipInfoResponses = + FutureUtils.bestEffortGet(gossipInfoFutures, totalTimeout, TimeUnit.MILLISECONDS); + + if (gossipInfoResponses.isEmpty()) + { + LOGGER.warn("Unable to retrieve gossip info from any nodes. 0/{} instances available.", + gossipInfoFutures.size()); + // do not fail here, bridge determination logic checks for feature flag and proceeds accordingly + return Collections.emptySet(); + } + else if (gossipInfoResponses.size() < gossipInfoFutures.size()) + { + LOGGER.warn("{}/{} instances were used to retrieve gossip info and determine SSTable versions", + gossipInfoResponses.size(), gossipInfoFutures.size()); + } + + // Extract and collect SSTable versions from all gossip info responses + Set sstableVersions = gossipInfoResponses.stream() + .flatMap(response -> response.values().stream()) + .map(GossipInfoResponse.GossipInfo::sstableVersions) + .filter(Objects::nonNull) + .flatMap(List::stream) + .collect(Collectors.toSet()); + + LOGGER.info("Detected SSTable versions on cluster: {}", sstableVersions); + return sstableVersions; + } + public static SidecarInstance toSidecarInstance(CassandraInstance instance, int sidecarPort) { return new SidecarInstanceImpl(instance.nodeName(), sidecarPort); diff --git a/cassandra-analytics-sidecar-client/src/test/java/org/apache/cassandra/clients/SidecarTest.java b/cassandra-analytics-sidecar-client/src/test/java/org/apache/cassandra/clients/SidecarTest.java new file mode 100644 index 000000000..82b454868 --- /dev/null +++ b/cassandra-analytics-sidecar-client/src/test/java/org/apache/cassandra/clients/SidecarTest.java @@ -0,0 +1,193 @@ +/* + * 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.clients; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CompletableFuture; + +import org.junit.jupiter.api.Test; + +import o.a.c.sidecar.client.shaded.client.SidecarClient; +import o.a.c.sidecar.client.shaded.client.SidecarInstance; +import o.a.c.sidecar.client.shaded.client.SidecarInstanceImpl; +import o.a.c.sidecar.client.shaded.common.response.GossipInfoResponse; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * Unit tests for Sidecar utility methods + */ +public class SidecarTest +{ + @Test + void testGetSSTableVersionsFromClusterWithSingleNode() + { + SidecarClient client = mock(SidecarClient.class); + SidecarInstance instance = new SidecarInstanceImpl("localhost", 9043); + Set instances = Collections.singleton(instance); + + // Mock gossip response + GossipInfoResponse response = new GossipInfoResponse(); + GossipInfoResponse.GossipInfo info = createGossipInfo(Arrays.asList("big-na", "big-nb")); + response.put("localhost", info); + + when(client.gossipInfo(any(SidecarInstance.class))) + .thenReturn(CompletableFuture.completedFuture(response)); + + Set versions = Sidecar.getSSTableVersionsFromCluster(client, instances, 1000L, 3); + + assertThat(versions) + .describedAs("Should extract SSTable versions from gossip info") + .containsExactlyInAnyOrder("big-na", "big-nb"); + } + + @Test + void testGetSSTableVersionsFromClusterWithMultipleNodesAndMixedVersions() + { + SidecarClient client = mock(SidecarClient.class); + SidecarInstance instance1 = new SidecarInstanceImpl("node1", 9043); + SidecarInstance instance2 = new SidecarInstanceImpl("node2", 9043); + Set instances = new HashSet<>(Arrays.asList(instance1, instance2)); + + // Node1 has C* 4.0 versions (big-na, big-nb) + GossipInfoResponse response1 = new GossipInfoResponse(); + GossipInfoResponse.GossipInfo info1 = createGossipInfo(Arrays.asList("big-na", "big-nb")); + response1.put("node1", info1); + + // Node2 has mixed versions: one C* 4.0 (big-nb) and C* 5.0 versions (big-oa, bti-da) + GossipInfoResponse response2 = new GossipInfoResponse(); + GossipInfoResponse.GossipInfo info2 = createGossipInfo(Arrays.asList("big-nb", "big-oa", "bti-da")); + response2.put("node2", info2); + + when(client.gossipInfo(instance1)) + .thenReturn(CompletableFuture.completedFuture(response1)); + when(client.gossipInfo(instance2)) + .thenReturn(CompletableFuture.completedFuture(response2)); + + Set versions = Sidecar.getSSTableVersionsFromCluster(client, instances, 1000L, 3); + + assertThat(versions) + .describedAs("Should aggregate and deduplicate SSTable versions from multiple nodes with mixed Cassandra versions") + .containsExactlyInAnyOrder("big-na", "big-nb", "big-oa", "bti-da"); + } + + @Test + void testGetSSTableVersionsFromClusterWithFailedNode() + { + SidecarClient client = mock(SidecarClient.class); + SidecarInstance instance1 = new SidecarInstanceImpl("node1", 9043); + SidecarInstance instance2 = new SidecarInstanceImpl("node2", 9043); + Set instances = new HashSet<>(Arrays.asList(instance1, instance2)); + + // Node1 succeeds + GossipInfoResponse response1 = new GossipInfoResponse(); + GossipInfoResponse.GossipInfo info1 = createGossipInfo(Arrays.asList("big-na", "big-nb")); + response1.put("node1", info1); + + when(client.gossipInfo(instance1)) + .thenReturn(CompletableFuture.completedFuture(response1)); + + // Node2 fails + when(client.gossipInfo(instance2)) + .thenReturn(CompletableFuture.failedFuture(new RuntimeException("Connection failed"))); + + Set versions = Sidecar.getSSTableVersionsFromCluster(client, instances, 1000L, 3); + + assertThat(versions) + .describedAs("Should return versions from successful nodes even when some fail") + .containsExactlyInAnyOrder("big-na", "big-nb"); + } + + @Test + void testGetSSTableVersionsFromClusterWhenAllNodesFail() + { + SidecarClient client = mock(SidecarClient.class); + SidecarInstance instance1 = new SidecarInstanceImpl("node1", 9043); + SidecarInstance instance2 = new SidecarInstanceImpl("node2", 9043); + Set instances = new HashSet<>(Arrays.asList(instance1, instance2)); + + // Both nodes fail + when(client.gossipInfo(any(SidecarInstance.class))) + .thenReturn(CompletableFuture.failedFuture(new RuntimeException("Connection failed"))); + + Set versions = Sidecar.getSSTableVersionsFromCluster(client, instances, 1000L, 3); + + assertThat(versions) + .describedAs("Should return empty set when all nodes fail") + .isEmpty(); + } + + @Test + void testGetSSTableVersionsFromClusterWithNullSSTableVersions() + { + SidecarClient client = mock(SidecarClient.class); + SidecarInstance instance = new SidecarInstanceImpl("localhost", 9043); + Set instances = Collections.singleton(instance); + + // Mock gossip response with null SSTable versions + GossipInfoResponse response = new GossipInfoResponse(); + GossipInfoResponse.GossipInfo info = createGossipInfo(null); + response.put("localhost", info); + + when(client.gossipInfo(any(SidecarInstance.class))) + .thenReturn(CompletableFuture.completedFuture(response)); + + Set versions = Sidecar.getSSTableVersionsFromCluster(client, instances, 1000L, 3); + + assertThat(versions) + .describedAs("Should return empty set when SSTable versions are null") + .isEmpty(); + } + + @Test + void testGetSSTableVersionsFromClusterWithEmptyInstancesSet() + { + SidecarClient client = mock(SidecarClient.class); + Set instances = Collections.emptySet(); + + Set versions = Sidecar.getSSTableVersionsFromCluster(client, instances, 1000L, 3); + + assertThat(versions) + .describedAs("Should return empty set when no instances provided") + .isEmpty(); + } + + private GossipInfoResponse.GossipInfo createGossipInfo(List sstableVersions) + { + GossipInfoResponse.GossipInfo info = new GossipInfoResponse.GossipInfo(); + info.put("status", "NORMAL"); + info.put("rack", "RACK1"); + info.put("datacenter", "DC1"); + info.put("releaseVersion", "4.0.0"); + info.put("rpcAddress", "127.0.0.1"); + if (sstableVersions != null && !sstableVersions.isEmpty()) + { + info.put("sstableVersions", String.join(",", sstableVersions)); + } + return info; + } +} diff --git a/cassandra-five-zero-bridge/src/test/java/org/apache/cassandra/io/sstable/format/bti/BtiReaderUtilsTest.java b/cassandra-five-zero-bridge/src/test/java/org/apache/cassandra/io/sstable/format/bti/BtiReaderUtilsTest.java index 8539b3ff2..11236cda8 100644 --- a/cassandra-five-zero-bridge/src/test/java/org/apache/cassandra/io/sstable/format/bti/BtiReaderUtilsTest.java +++ b/cassandra-five-zero-bridge/src/test/java/org/apache/cassandra/io/sstable/format/bti/BtiReaderUtilsTest.java @@ -61,7 +61,7 @@ public class BtiReaderUtilsTest public void testPartitionIndexTokenRange(boolean compression) { // Only test BTI format for Cassandra 5.0+ - assumeThat(CassandraVersion.sstableFormat()).isEqualTo("bti"); + assumeThat(CassandraVersion.configuredSSTableFormat()).isEqualTo("bti"); qt().forAll(arbitrary().enumValues(Partitioner.class)) .checkAssert(partitioner -> { @@ -103,7 +103,7 @@ public void testPartitionIndexTokenRange(boolean compression) public void testPrimaryIndexContainsAnyKey(boolean compression) { // Only test BTI format for Cassandra 5.0+ - assumeThat(CassandraVersion.sstableFormat()).isEqualTo("bti"); + assumeThat(CassandraVersion.configuredSSTableFormat()).isEqualTo("bti"); qt().forAll(arbitrary().enumValues(Partitioner.class)) .checkAssert(partitioner -> { @@ -162,7 +162,7 @@ public void testPrimaryIndexContainsAnyKey(boolean compression) public void testReadPrimaryIndex(boolean compression) { // Only test BTI format for Cassandra 5.0+ - assumeThat(CassandraVersion.sstableFormat()).isEqualTo("bti"); + assumeThat(CassandraVersion.configuredSSTableFormat()).isEqualTo("bti"); qt().forAll(arbitrary().enumValues(Partitioner.class)) .checkAssert(partitioner -> { @@ -221,7 +221,7 @@ public void testReadPrimaryIndex(boolean compression) public void testConsumePrimaryIndex(boolean compression) { // Only test BTI format for Cassandra 5.0+ - assumeThat(CassandraVersion.sstableFormat()).isEqualTo("bti"); + assumeThat(CassandraVersion.configuredSSTableFormat()).isEqualTo("bti"); qt().forAll(arbitrary().enumValues(Partitioner.class)) .checkAssert(partitioner -> { diff --git a/cassandra-five-zero-bridge/src/test/java/org/apache/cassandra/spark/reader/SummaryDbTests.java b/cassandra-five-zero-bridge/src/test/java/org/apache/cassandra/spark/reader/SummaryDbTests.java index b16330e9b..17621e346 100644 --- a/cassandra-five-zero-bridge/src/test/java/org/apache/cassandra/spark/reader/SummaryDbTests.java +++ b/cassandra-five-zero-bridge/src/test/java/org/apache/cassandra/spark/reader/SummaryDbTests.java @@ -86,7 +86,7 @@ public BigInteger tokenAt(int index) public void testSearchSummary() { // Summary.db file is present only in BIG sstables - assumeThat(CassandraVersion.sstableFormat()).isEqualTo("big"); + assumeThat(CassandraVersion.configuredSSTableFormat()).isEqualTo("big"); qt().forAll(arbitrary().enumValues(Partitioner.class)) .checkAssert(partitioner -> { @@ -149,7 +149,7 @@ public void testSearchSummary() public void testSummaryBinarySearch() { // Summary.db file is present only in BIG sstables - assumeThat(CassandraVersion.sstableFormat()).isEqualTo("big"); + assumeThat(CassandraVersion.configuredSSTableFormat()).isEqualTo("big"); SummaryDbUtils.TokenList list = new ArrayTokenList(LongStream.range(5, 10000).boxed().toArray(Long[]::new)); assertThat(SummaryDbUtils.binarySearchSummary(list, BigInteger.valueOf(154L))).isEqualTo(148); @@ -169,7 +169,7 @@ public void testSummaryBinarySearch() public void testSummaryBinarySearchSparse() { // Summary.db file is present only in BIG sstables - assumeThat(CassandraVersion.sstableFormat()).isEqualTo("big"); + assumeThat(CassandraVersion.configuredSSTableFormat()).isEqualTo("big"); SummaryDbUtils.TokenList list = new ArrayTokenList(5L, 10L, 15L, 20L, 25L); assertThat(SummaryDbUtils.binarySearchSummary(list, BigInteger.valueOf(-500L))).isEqualTo(0); diff --git a/cassandra-five-zero-types/src/main/java/org/apache/cassandra/bridge/CassandraTypesImplementation.java b/cassandra-five-zero-types/src/main/java/org/apache/cassandra/bridge/CassandraTypesImplementation.java index a13071c11..6d5804cbf 100644 --- a/cassandra-five-zero-types/src/main/java/org/apache/cassandra/bridge/CassandraTypesImplementation.java +++ b/cassandra-five-zero-types/src/main/java/org/apache/cassandra/bridge/CassandraTypesImplementation.java @@ -54,7 +54,7 @@ public static synchronized void setup(BridgeInitializationParameters params) config.diagnostic_events_enabled = false; config.max_mutation_size = new DataStorageSpec.IntKibibytesBound(config.commitlog_segment_size.toKibibytes() / 2); config.concurrent_compactors = 4; - config.sstable.selected_format = params.getSstableFormat(); + config.sstable.selected_format = params.getConfiguredSSTableFormat(); Path tempDirectory; try {