├── gradle └── wrapper │ ├── gradle-wrapper.jar │ └── gradle-wrapper.properties ├── settings.gradle ├── cdc ├── src │ ├── main │ │ └── java │ │ │ └── org │ │ │ └── apache │ │ │ └── cassandra │ │ │ └── spark │ │ │ ├── cdc │ │ │ ├── RowSource.java │ │ │ ├── RowSink.java │ │ │ ├── ICassandraSource.java │ │ │ ├── CommitLogProvider.java │ │ │ ├── jdk │ │ │ │ ├── JdkValueMetadata.java │ │ │ │ ├── JdkRangeTombstone.java │ │ │ │ └── msg │ │ │ │ │ ├── Column.java │ │ │ │ │ └── RangeTombstone.java │ │ │ ├── ValueWithMetadata.java │ │ │ └── watermarker │ │ │ │ └── DoNothingWatermarker.java │ │ │ ├── CdcKryoRegister.java │ │ │ └── stats │ │ │ └── CdcStats.java │ └── test │ │ └── java │ │ └── org │ │ └── apache │ │ └── cassandra │ │ └── spark │ │ ├── cdc │ │ └── jdk │ │ │ └── msg │ │ │ └── CdcMessageTests.java │ │ └── CdcKryoSerializationTests.java └── build.gradle ├── profiles ├── scala-2.12.gradle ├── scala-2.11.gradle ├── scala-2.11-spark3.gradle └── scala-2.12-spark3.gradle ├── core └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── cassandra │ │ │ └── spark │ │ │ ├── data │ │ │ ├── fourzero │ │ │ │ └── types │ │ │ │ │ └── spark │ │ │ │ │ ├── Counter.java │ │ │ │ │ ├── Duration.java │ │ │ │ │ ├── Int.java │ │ │ │ │ ├── Time.java │ │ │ │ │ ├── BigInt.java │ │ │ │ │ ├── UUID.java │ │ │ │ │ ├── Text.java │ │ │ │ │ ├── Ascii.java │ │ │ │ │ ├── TimeUUID.java │ │ │ │ │ ├── VarChar.java │ │ │ │ │ ├── Blob.java │ │ │ │ │ ├── Inet.java │ │ │ │ │ ├── VarInt.java │ │ │ │ │ ├── SmallInt.java │ │ │ │ │ ├── Timestamp.java │ │ │ │ │ ├── Double.java │ │ │ │ │ ├── Empty.java │ │ │ │ │ ├── Float.java │ │ │ │ │ ├── Boolean.java │ │ │ │ │ ├── TinyInt.java │ │ │ │ │ └── complex │ │ │ │ │ └── CqlList.java │ │ │ └── SSTablesSupplier.java │ │ │ ├── cdc │ │ │ ├── watermarker │ │ │ │ └── SparkInMemoryWatermarker.java │ │ │ ├── SparkRowSource.java │ │ │ ├── SparkRowSink.java │ │ │ ├── fourzero │ │ │ │ ├── CdcEventWriter.java │ │ │ │ ├── LocalCdcEventWriter.java │ │ │ │ └── SparkRangeTombstoneBuilder.java │ │ │ └── AbstractCdcEventWriter.java │ │ │ ├── reader │ │ │ ├── common │ │ │ │ ├── IndexReader.java │ │ │ │ └── ChunkCorruptException.java │ │ │ ├── fourzero │ │ │ │ └── Scannable.java │ │ │ ├── IndexConsumer.java │ │ │ └── IndexEntry.java │ │ │ └── sparksql │ │ │ ├── NoMatchFoundException.java │ │ │ └── filters │ │ │ └── PruneColumnFilter.java │ ├── spark3 │ │ └── org │ │ │ └── apache │ │ │ └── cassandra │ │ │ └── spark │ │ │ └── sparksql │ │ │ ├── LocalDataSource.java │ │ │ └── LocalPartitionSizeSource.java │ └── spark2 │ │ └── org │ │ └── apache │ │ └── cassandra │ │ └── spark │ │ └── sparksql │ │ └── LocalDataSource.java │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── cassandra │ │ └── spark │ │ ├── TestRunnable.java │ │ └── data │ │ ├── fourzero │ │ └── types │ │ │ └── DateTypeTests.java │ │ └── VersionRunner.java │ ├── spark3 │ └── org │ │ └── apache │ │ └── cassandra │ │ └── spark │ │ └── cdc │ │ ├── RequireTwoReplicasLocalDataSource.java │ │ └── SpyWatermarkerDataSource.java │ └── spark2 │ └── org │ └── apache │ └── cassandra │ └── spark │ └── sparksql │ └── RangeFilterTests.java ├── ide └── idea │ └── Project_Default.xml ├── gradle.properties ├── common └── src │ ├── test │ └── java │ │ └── org │ │ └── apache │ │ └── cassandra │ │ └── spark │ │ ├── utils │ │ ├── TimeUtilsTest.java │ │ ├── LoggerHelperTests.java │ │ └── ArrayUtilsTest.java │ │ └── reader │ │ └── fourzero │ │ └── CompressionUtilTests.java │ └── main │ └── java │ └── org │ └── apache │ └── cassandra │ └── spark │ ├── utils │ ├── StatsUtil.java │ ├── TimeUtils.java │ ├── TimeProvider.java │ ├── RandomUtils.java │ ├── streaming │ │ ├── StreamConsumer.java │ │ ├── CassandraFile.java │ │ └── StreamBuffer.java │ ├── IOUtils.java │ └── ThrowableUtils.java │ ├── data │ ├── fourzero │ │ └── types │ │ │ ├── BinaryBased.java │ │ │ ├── LongBased.java │ │ │ ├── Counter.java │ │ │ ├── Duration.java │ │ │ ├── StringBased.java │ │ │ ├── Text.java │ │ │ ├── Ascii.java │ │ │ ├── VarChar.java │ │ │ ├── TimeUUID.java │ │ │ ├── Time.java │ │ │ ├── BigInt.java │ │ │ ├── Empty.java │ │ │ ├── UUID.java │ │ │ ├── Blob.java │ │ │ ├── Timestamp.java │ │ │ ├── Int.java │ │ │ ├── Float.java │ │ │ ├── TinyInt.java │ │ │ ├── Double.java │ │ │ ├── SmallInt.java │ │ │ ├── VarInt.java │ │ │ ├── Boolean.java │ │ │ └── Inet.java │ ├── AvailabilityHint.java │ ├── partitioner │ │ ├── Partitioner.java │ │ └── NotEnoughReplicasException.java │ └── IncompleteSSTableException.java │ ├── reader │ ├── fourzero │ │ ├── UdtBuffer.java │ │ ├── ListBuffer.java │ │ ├── SetBuffer.java │ │ ├── MapBuffer.java │ │ └── BaseFourZeroUtils.java │ ├── CassandraVersion.java │ ├── EmptyScanner.java │ ├── BigNumberConfig.java │ ├── SparkSSTableReader.java │ └── common │ │ └── SSTableStreamException.java │ ├── shaded │ └── fourzero │ │ └── cassandra │ │ └── cql3 │ │ └── functions │ │ └── types │ │ ├── UserTypeHelper.java │ │ └── TupleHelper.java │ ├── stats │ └── IStats.java │ └── sparksql │ └── filters │ └── RangeFilter.java ├── example ├── build.gradle └── src │ └── main │ ├── java │ └── org │ │ └── apache │ │ └── cassandra │ │ └── spark │ │ └── SimpleExample.java │ ├── spark2 │ └── org │ │ └── apache │ │ └── cassandra │ │ └── spark │ │ └── s3 │ │ └── S3DataSource.java │ └── spark3 │ └── org │ └── apache │ └── cassandra │ └── spark │ └── s3 │ └── S3DataSource.java ├── DEV-README.md └── .gitignore /gradle/wrapper/gradle-wrapper.jar: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jberragan/spark-cassandra-bulkreader/HEAD/gradle/wrapper/gradle-wrapper.jar -------------------------------------------------------------------------------- /settings.gradle: -------------------------------------------------------------------------------- 1 | rootProject.name = 'spark-cassandra-bulkreader' 2 | 3 | include 'cdc' 4 | include 'common' 5 | include 'core' 6 | include 'fourzero' 7 | include 'example' -------------------------------------------------------------------------------- /cdc/src/main/java/org/apache/cassandra/spark/cdc/RowSource.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.cdc; 2 | 3 | public interface RowSource 4 | { 5 | RowType toRow(); 6 | } -------------------------------------------------------------------------------- /cdc/src/main/java/org/apache/cassandra/spark/cdc/RowSink.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.cdc; 2 | 3 | public interface RowSink 4 | { 5 | ReturnType fromRow(RowType row); 6 | } -------------------------------------------------------------------------------- /profiles/scala-2.12.gradle: -------------------------------------------------------------------------------- 1 | ext { 2 | jacksonVersion="2.6.7.1" 3 | scalaMajorVersion="2.12" 4 | jacksonCoreVersion="2.9.5" 5 | scalaVersion="2.12.10" 6 | sparkMajorVersion="2" 7 | sparkVersion="2.4.5" 8 | } -------------------------------------------------------------------------------- /profiles/scala-2.11.gradle: -------------------------------------------------------------------------------- 1 | ext { 2 | jacksonVersion="2.6.7.1" 3 | jacksonCoreVersion="2.9.5" 4 | scalaMajorVersion="2.11" 5 | scalaVersion="2.11.12" 6 | sparkVersion="2.4.5" 7 | sparkMajorVersion="2" 8 | } 9 | -------------------------------------------------------------------------------- /profiles/scala-2.11-spark3.gradle: -------------------------------------------------------------------------------- 1 | ext { 2 | jacksonVersion="2.12.0" 3 | jacksonCoreVersion="2.12.0" 4 | scalaMajorVersion="2.12" 5 | scalaVersion="2.12.10" 6 | sparkMajorVersion="3" 7 | sparkVersion="3.0.1" 8 | } -------------------------------------------------------------------------------- /profiles/scala-2.12-spark3.gradle: -------------------------------------------------------------------------------- 1 | ext { 2 | jacksonVersion="2.12.0" 3 | jacksonCoreVersion="2.12.0" 4 | scalaMajorVersion="2.12" 5 | scalaVersion="2.12.10" 6 | sparkMajorVersion="3" 7 | sparkVersion="3.0.1" 8 | } 9 | 10 | -------------------------------------------------------------------------------- /gradle/wrapper/gradle-wrapper.properties: -------------------------------------------------------------------------------- 1 | distributionBase=GRADLE_USER_HOME 2 | distributionPath=wrapper/dists 3 | distributionUrl=https\://services.gradle.org/distributions/gradle-7.3.3-bin.zip 4 | zipStoreBase=GRADLE_USER_HOME 5 | zipStorePath=wrapper/dists 6 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/data/fourzero/types/spark/Counter.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types.spark; 2 | 3 | import org.apache.cassandra.spark.data.SparkCqlField; 4 | 5 | public class Counter extends org.apache.cassandra.spark.data.fourzero.types.Counter implements SparkCqlField.NotImplementedTrait 6 | { 7 | public static final Counter INSTANCE = new Counter(); 8 | } 9 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/data/fourzero/types/spark/Duration.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types.spark; 2 | 3 | import org.apache.cassandra.spark.data.SparkCqlField; 4 | 5 | public class Duration extends org.apache.cassandra.spark.data.fourzero.types.Duration implements SparkCqlField.NotImplementedTrait 6 | { 7 | public static final Duration INSTANCE = new Duration(); 8 | } 9 | -------------------------------------------------------------------------------- /ide/idea/Project_Default.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 7 | -------------------------------------------------------------------------------- /gradle.properties: -------------------------------------------------------------------------------- 1 | group=org.apache.cassandra.spark 2 | version=0.0.1 3 | snapshot=true 4 | 5 | jdkLevel=1.8 6 | scala=2.12 7 | spark=3 8 | intellijVersion=9.0.4 9 | junitVersion=4.12 10 | quickTheoriesVersion=0.26 11 | mockitoVersion=1.10.19 12 | jnaVersion=5.9.0 13 | slf4jApiVersion=1.7.26 14 | commonsLang3Version=3.12.0 15 | commonsLangVersion=2.6 16 | guavaVersion=16.0.1 17 | kryoVersion=4.0.2 18 | commonsCodecVersion=1.15 19 | zStdVersion=1.5.0-4 20 | nettyBoringSslVersion=2.0.48.Final 21 | -------------------------------------------------------------------------------- /common/src/test/java/org/apache/cassandra/spark/utils/TimeUtilsTest.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.utils; 2 | 3 | import java.time.Duration; 4 | 5 | import org.junit.Test; 6 | 7 | import static org.junit.Assert.assertEquals; 8 | 9 | public class TimeUtilsTest 10 | { 11 | @Test 12 | public void testDurationToMicros() 13 | { 14 | assertEquals(1000_000L, TimeUtils.toMicros(Duration.ofSeconds(1))); 15 | assertEquals(1234_000L, TimeUtils.toMicros(Duration.ofMillis(1234))); 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/cdc/watermarker/SparkInMemoryWatermarker.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.cdc.watermarker; 2 | 3 | import org.apache.spark.TaskContext; 4 | 5 | public class SparkInMemoryWatermarker 6 | { 7 | public static final InMemoryWatermarker INSTANCE = new InMemoryWatermarker(new InMemoryWatermarker.TaskContextProvider() 8 | { 9 | public boolean hasTaskContext() 10 | { 11 | return TaskContext.get() != null; 12 | } 13 | 14 | public int partitionId() 15 | { 16 | return TaskContext.getPartitionId(); 17 | } 18 | }); 19 | } 20 | -------------------------------------------------------------------------------- /cdc/src/main/java/org/apache/cassandra/spark/cdc/ICassandraSource.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.cdc; 2 | 3 | import java.nio.ByteBuffer; 4 | import java.util.List; 5 | 6 | public interface ICassandraSource 7 | { 8 | /** 9 | * Read values from Cassandra, instead of using the values from commitlog 10 | * 11 | * @param keyspace name of the keyspace 12 | * @param table name of the Table 13 | * @param columnsToFetch lis of columns to fetch 14 | * @param primaryKeyColumns primary key columns to locate the row 15 | * @return list of values read from cassandra. The size should be the same as columnsToFetch 16 | */ 17 | List readFromCassandra(String keyspace, String table, List columnsToFetch, 18 | List primaryKeyColumns); 19 | } 20 | -------------------------------------------------------------------------------- /example/build.gradle: -------------------------------------------------------------------------------- 1 | project(':example') { 2 | apply plugin: 'java' 3 | apply plugin: 'application' 4 | 5 | apply from: "../profiles/scala-${project.rootProject.ext.crossbuildVersion}${project.rootProject.ext.sparkVersionQualifier}.gradle" 6 | 7 | application { 8 | mainClassName = "org.apache.cassandra.spark.SimpleExample" 9 | } 10 | shadowJar.enabled = false 11 | 12 | dependencies { 13 | implementation project(path: ':common') 14 | implementation project(path: ':cdc') 15 | implementation project(path: ':core') 16 | implementation "org.apache.spark:spark-core_${scalaMajorVersion}:${sparkVersion}" 17 | implementation "org.apache.spark:spark-sql_${scalaMajorVersion}:${sparkVersion}" 18 | implementation platform('com.amazonaws:aws-java-sdk-bom:1.11.989') 19 | implementation 'com.amazonaws:aws-java-sdk-s3:1.11.989' 20 | } 21 | 22 | } -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/utils/StatsUtil.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.utils; 2 | 3 | import java.util.concurrent.Callable; 4 | import java.util.function.LongConsumer; 5 | 6 | public class StatsUtil 7 | { 8 | 9 | /** 10 | * Convenient helper that captures the time taken to run the task and pass it to the {@link LongConsumer} to report. 11 | * It is suitable for the tasks that have simple scope. For the complicated ones that have distinct callsites for 12 | * starting and stopping the timer, the method does not fit. 13 | */ 14 | public static T reportTimeTaken(Callable task, LongConsumer nanosTaken) 15 | { 16 | long current = System.nanoTime(); 17 | try 18 | { 19 | T res = task.call(); 20 | nanosTaken.accept(System.nanoTime() - current); 21 | return res; 22 | } 23 | catch (Exception e) 24 | { 25 | throw new RuntimeException(e); 26 | } 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /example/src/main/java/org/apache/cassandra/spark/SimpleExample.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark; 2 | 3 | import org.apache.cassandra.spark.sparksql.LocalDataSource; 4 | import org.apache.spark.SparkConf; 5 | import org.apache.spark.sql.Dataset; 6 | import org.apache.spark.sql.Row; 7 | import org.apache.spark.sql.SparkSession; 8 | 9 | public class SimpleExample 10 | { 11 | public static void main(final String[] args) 12 | { 13 | final SparkSession spark = SparkSession.builder() 14 | .config(new SparkConf().set("spark.master", "local")) 15 | .getOrCreate(); 16 | 17 | final Dataset df = spark.read().format(LocalDataSource.class.getName()) 18 | .option("keyspace", "test") 19 | .option("createStmt", "CREATE TABLE IF NOT EXISTS test.basic_test (a bigint PRIMARY KEY, b bigint, c bigint);") 20 | .option("dirs", args[0]) 21 | .load(); 22 | 23 | for (Row row : df.collectAsList()) 24 | { 25 | System.out.println("Row: " + row); 26 | } 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/utils/TimeUtils.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.utils; 2 | 3 | import java.time.Duration; 4 | import java.util.concurrent.TimeUnit; 5 | 6 | public class TimeUtils 7 | { 8 | public static long secsToMicros(long seconds) 9 | { 10 | return TimeUnit.SECONDS.toMicros(seconds); 11 | } 12 | 13 | // It drops any precision higher than milliseconds from duration 14 | public static long toMicros(Duration duration) 15 | { 16 | return TimeUnit.MILLISECONDS.toMicros(duration.toMillis()); 17 | } 18 | 19 | public static long toMicros(java.sql.Timestamp timestamp) 20 | { 21 | long millis = timestamp.getTime(); 22 | int nanos = timestamp.getNanos(); // nanos of the fractional seconds component. 23 | nanos = nanos % 1000_000; // only keep the micros component. 24 | return TimeUnit.MILLISECONDS.toMicros(millis) + TimeUnit.NANOSECONDS.toMicros(nanos); 25 | } 26 | 27 | public static long nowMicros() 28 | { 29 | return TimeUnit.MILLISECONDS.toMicros(System.currentTimeMillis()); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /cdc/src/main/java/org/apache/cassandra/spark/cdc/CommitLogProvider.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.cdc; 2 | 3 | import java.util.stream.Stream; 4 | 5 | /* 6 | * 7 | * Licensed to the Apache Software Foundation (ASF) under one 8 | * or more contributor license agreements. See the NOTICE file 9 | * distributed with this work for additional information 10 | * regarding copyright ownership. The ASF licenses this file 11 | * to you under the Apache License, Version 2.0 (the 12 | * "License"); you may not use this file except in compliance 13 | * with the License. You may obtain a copy of the License at 14 | * 15 | * http://www.apache.org/licenses/LICENSE-2.0 16 | * 17 | * Unless required by applicable law or agreed to in writing, 18 | * software distributed under the License is distributed on an 19 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 20 | * KIND, either express or implied. See the License for the 21 | * specific language governing permissions and limitations 22 | * under the License. 23 | * 24 | */ 25 | 26 | public interface CommitLogProvider 27 | { 28 | Stream logs(); 29 | } 30 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/reader/common/IndexReader.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.reader.common; 23 | 24 | import org.apache.cassandra.spark.reader.SparkSSTableReader; 25 | 26 | public interface IndexReader extends SparkSSTableReader 27 | { 28 | } 29 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/BinaryBased.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import org.apache.cassandra.spark.data.fourzero.NativeType; 4 | 5 | /* 6 | * 7 | * Licensed to the Apache Software Foundation (ASF) under one 8 | * or more contributor license agreements. See the NOTICE file 9 | * distributed with this work for additional information 10 | * regarding copyright ownership. The ASF licenses this file 11 | * to you under the Apache License, Version 2.0 (the 12 | * "License"); you may not use this file except in compliance 13 | * with the License. You may obtain a copy of the License at 14 | * 15 | * http://www.apache.org/licenses/LICENSE-2.0 16 | * 17 | * Unless required by applicable law or agreed to in writing, 18 | * software distributed under the License is distributed on an 19 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 20 | * KIND, either express or implied. See the License for the 21 | * specific language governing permissions and limitations 22 | * under the License. 23 | * 24 | */ 25 | 26 | public abstract class BinaryBased extends NativeType 27 | { 28 | } 29 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/reader/fourzero/UdtBuffer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.reader.fourzero; 23 | 24 | public class UdtBuffer extends ComplexTypeBuffer 25 | { 26 | UdtBuffer(int cellCount) 27 | { 28 | super(cellCount, cellCount); 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/reader/fourzero/ListBuffer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.reader.fourzero; 23 | 24 | public class ListBuffer extends ComplexTypeBuffer 25 | { 26 | ListBuffer(int cellCount) 27 | { 28 | super(cellCount, cellCount); 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/reader/fourzero/Scannable.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.reader.fourzero; 2 | 3 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.io.sstable.ISSTableScanner; 4 | 5 | /* 6 | * 7 | * Licensed to the Apache Software Foundation (ASF) under one 8 | * or more contributor license agreements. See the NOTICE file 9 | * distributed with this work for additional information 10 | * regarding copyright ownership. The ASF licenses this file 11 | * to you under the Apache License, Version 2.0 (the 12 | * "License"); you may not use this file except in compliance 13 | * with the License. You may obtain a copy of the License at 14 | * 15 | * http://www.apache.org/licenses/LICENSE-2.0 16 | * 17 | * Unless required by applicable law or agreed to in writing, 18 | * software distributed under the License is distributed on an 19 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 20 | * KIND, either express or implied. See the License for the 21 | * specific language governing permissions and limitations 22 | * under the License. 23 | * 24 | */ 25 | 26 | public interface Scannable 27 | { 28 | ISSTableScanner scanner(); 29 | } 30 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/reader/IndexConsumer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.reader; 23 | 24 | import java.util.function.Consumer; 25 | 26 | public interface IndexConsumer extends Consumer 27 | { 28 | void onFailure(Throwable t); 29 | 30 | void onFinished(long runtimeNanos); 31 | } 32 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/utils/TimeProvider.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.utils; 2 | 3 | /* 4 | * 5 | * Licensed to the Apache Software Foundation (ASF) under one 6 | * or more contributor license agreements. See the NOTICE file 7 | * distributed with this work for additional information 8 | * regarding copyright ownership. The ASF licenses this file 9 | * to you under the Apache License, Version 2.0 (the 10 | * "License"); you may not use this file except in compliance 11 | * with the License. You may obtain a copy of the License at 12 | * 13 | * http://www.apache.org/licenses/LICENSE-2.0 14 | * 15 | * Unless required by applicable law or agreed to in writing, 16 | * software distributed under the License is distributed on an 17 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 18 | * KIND, either express or implied. See the License for the 19 | * specific language governing permissions and limitations 20 | * under the License. 21 | * 22 | */ 23 | 24 | public interface TimeProvider 25 | { 26 | TimeProvider INSTANCE = () -> (int) (System.currentTimeMillis() / 1000L); 27 | 28 | /** 29 | * @return current time in seconds 30 | */ 31 | int now(); 32 | } 33 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/cdc/SparkRowSource.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.cdc; 23 | 24 | import org.apache.spark.sql.catalyst.InternalRow; 25 | 26 | public interface SparkRowSource extends RowSource 27 | { 28 | /** 29 | * @return an {@link InternalRow} 30 | */ 31 | InternalRow toRow(); 32 | } 33 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/reader/common/ChunkCorruptException.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.reader.common; 2 | 3 | import java.io.IOException; 4 | 5 | /* 6 | * 7 | * Licensed to the Apache Software Foundation (ASF) under one 8 | * or more contributor license agreements. See the NOTICE file 9 | * distributed with this work for additional information 10 | * regarding copyright ownership. The ASF licenses this file 11 | * to you under the Apache License, Version 2.0 (the 12 | * "License"); you may not use this file except in compliance 13 | * with the License. You may obtain a copy of the License at 14 | * 15 | * http://www.apache.org/licenses/LICENSE-2.0 16 | * 17 | * Unless required by applicable law or agreed to in writing, 18 | * software distributed under the License is distributed on an 19 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 20 | * KIND, either express or implied. See the License for the 21 | * specific language governing permissions and limitations 22 | * under the License. 23 | * 24 | */ 25 | 26 | public class ChunkCorruptException extends IOException 27 | { 28 | public ChunkCorruptException(final String message) 29 | { 30 | super(message); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/sparksql/NoMatchFoundException.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.sparksql; 2 | 3 | /* 4 | * 5 | * Licensed to the Apache Software Foundation (ASF) under one 6 | * or more contributor license agreements. See the NOTICE file 7 | * distributed with this work for additional information 8 | * regarding copyright ownership. The ASF licenses this file 9 | * to you under the Apache License, Version 2.0 (the 10 | * "License"); you may not use this file except in compliance 11 | * with the License. You may obtain a copy of the License at 12 | * 13 | * http://www.apache.org/licenses/LICENSE-2.0 14 | * 15 | * Unless required by applicable law or agreed to in writing, 16 | * software distributed under the License is distributed on an 17 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 18 | * KIND, either express or implied. See the License for the 19 | * specific language governing permissions and limitations 20 | * under the License. 21 | * 22 | */ 23 | 24 | public class NoMatchFoundException extends Exception 25 | { 26 | public NoMatchFoundException() 27 | { 28 | } 29 | 30 | public NoMatchFoundException(final String message) 31 | { 32 | super(message); 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/data/fourzero/types/spark/Int.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.data.fourzero.types.spark; 23 | 24 | import org.apache.cassandra.spark.data.SparkCqlField; 25 | 26 | public class Int extends org.apache.cassandra.spark.data.fourzero.types.Int implements SparkCqlField.IntTrait 27 | { 28 | public static final Int INSTANCE = new Int(); 29 | } 30 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/data/fourzero/types/spark/Time.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.data.fourzero.types.spark; 23 | 24 | import org.apache.cassandra.spark.data.SparkCqlField; 25 | 26 | public class Time extends org.apache.cassandra.spark.data.fourzero.types.Time implements SparkCqlField.LongTraits 27 | { 28 | public static final Time INSTANCE = new Time(); 29 | } 30 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/data/fourzero/types/spark/BigInt.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.data.fourzero.types.spark; 23 | 24 | import org.apache.cassandra.spark.data.SparkCqlField; 25 | 26 | public class BigInt extends org.apache.cassandra.spark.data.fourzero.types.BigInt implements SparkCqlField.LongTraits 27 | { 28 | public static final BigInt INSTANCE = new BigInt(); 29 | } 30 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/data/fourzero/types/spark/UUID.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.data.fourzero.types.spark; 23 | 24 | import org.apache.cassandra.spark.data.SparkCqlField; 25 | 26 | public class UUID extends org.apache.cassandra.spark.data.fourzero.types.UUID implements SparkCqlField.SparkCqlType, SparkCqlField.UUIDTraits 27 | { 28 | public static final UUID INSTANCE = new UUID(); 29 | } 30 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/data/fourzero/types/spark/Text.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.data.fourzero.types.spark; 23 | 24 | import org.apache.cassandra.spark.data.SparkCqlField; 25 | 26 | public class Text extends org.apache.cassandra.spark.data.fourzero.types.Text implements SparkCqlField.SparkCqlType, SparkCqlField.StringTraits 27 | { 28 | public static final Text INSTANCE = new Text(); 29 | } 30 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/data/fourzero/types/spark/Ascii.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.data.fourzero.types.spark; 23 | 24 | import org.apache.cassandra.spark.data.SparkCqlField; 25 | 26 | public class Ascii extends org.apache.cassandra.spark.data.fourzero.types.Ascii implements SparkCqlField.SparkCqlType, SparkCqlField.StringTraits 27 | { 28 | public static final Ascii INSTANCE = new Ascii(); 29 | } 30 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/reader/CassandraVersion.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.reader; 23 | 24 | public enum CassandraVersion 25 | { 26 | THREEZERO("3.0"), FOURZERO("4.0"); 27 | public final String name; 28 | 29 | CassandraVersion(final String name) 30 | { 31 | this.name = name; 32 | } 33 | 34 | public String versionName() 35 | { 36 | return name; 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/data/fourzero/types/spark/TimeUUID.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.data.fourzero.types.spark; 23 | 24 | import org.apache.cassandra.spark.data.SparkCqlField; 25 | 26 | public class TimeUUID extends org.apache.cassandra.spark.data.fourzero.types.TimeUUID implements SparkCqlField.SparkCqlType, SparkCqlField.UUIDTraits 27 | { 28 | public static final TimeUUID INSTANCE = new TimeUUID(); 29 | } 30 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/data/fourzero/types/spark/VarChar.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.data.fourzero.types.spark; 23 | 24 | import org.apache.cassandra.spark.data.SparkCqlField; 25 | 26 | public class VarChar extends org.apache.cassandra.spark.data.fourzero.types.VarChar implements SparkCqlField.SparkCqlType, SparkCqlField.StringTraits 27 | { 28 | public static final VarChar INSTANCE = new VarChar(); 29 | } 30 | -------------------------------------------------------------------------------- /core/src/test/java/org/apache/cassandra/spark/TestRunnable.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark; 2 | 3 | import java.io.IOException; 4 | import java.nio.file.Path; 5 | 6 | import org.apache.cassandra.spark.data.partitioner.Partitioner; 7 | import org.apache.cassandra.spark.reader.CassandraBridge; 8 | 9 | /* 10 | * 11 | * Licensed to the Apache Software Foundation (ASF) under one 12 | * or more contributor license agreements. See the NOTICE file 13 | * distributed with this work for additional information 14 | * regarding copyright ownership. The ASF licenses this file 15 | * to you under the Apache License, Version 2.0 (the 16 | * "License"); you may not use this file except in compliance 17 | * with the License. You may obtain a copy of the License at 18 | * 19 | * http://www.apache.org/licenses/LICENSE-2.0 20 | * 21 | * Unless required by applicable law or agreed to in writing, 22 | * software distributed under the License is distributed on an 23 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 24 | * KIND, either express or implied. See the License for the 25 | * specific language governing permissions and limitations 26 | * under the License. 27 | * 28 | */ 29 | public interface TestRunnable 30 | { 31 | void run(Partitioner partitioner, Path dir, CassandraBridge bridge) throws IOException; 32 | } 33 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/cdc/SparkRowSink.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.cdc; 23 | 24 | import org.apache.spark.sql.Row; 25 | 26 | public interface SparkRowSink extends RowSink 27 | { 28 | /** 29 | * Consumes the spark row and produce an instance of {@link ReturnType} 30 | * @param row spark row 31 | * @return an instance of T 32 | */ 33 | ReturnType fromRow(Row row); 34 | } 35 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/reader/fourzero/SetBuffer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.reader.fourzero; 23 | 24 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.rows.Cell; 25 | 26 | public class SetBuffer extends ComplexTypeBuffer 27 | { 28 | SetBuffer(int cellCount) 29 | { 30 | super(cellCount, cellCount); 31 | } 32 | 33 | @Override 34 | public void addCell(Cell cell) 35 | { 36 | this.add(cell.path().get(0)); // set - copy over key 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/LongBased.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import org.apache.cassandra.spark.data.fourzero.NativeType; 4 | import org.apache.cassandra.spark.utils.RandomUtils; 5 | 6 | /* 7 | * 8 | * Licensed to the Apache Software Foundation (ASF) under one 9 | * or more contributor license agreements. See the NOTICE file 10 | * distributed with this work for additional information 11 | * regarding copyright ownership. The ASF licenses this file 12 | * to you under the Apache License, Version 2.0 (the 13 | * "License"); you may not use this file except in compliance 14 | * with the License. You may obtain a copy of the License at 15 | * 16 | * http://www.apache.org/licenses/LICENSE-2.0 17 | * 18 | * Unless required by applicable law or agreed to in writing, 19 | * software distributed under the License is distributed on an 20 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 21 | * KIND, either express or implied. See the License for the 22 | * specific language governing permissions and limitations 23 | * under the License. 24 | * 25 | */ 26 | 27 | public abstract class LongBased extends NativeType 28 | { 29 | @Override 30 | public Object randomValue(int minCollectionSize) 31 | { 32 | return (long) RandomUtils.randomPositiveInt(5000000); // keep within bound to avoid overflows 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/Counter.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import org.apache.cassandra.spark.data.fourzero.NativeType; 4 | 5 | /* 6 | * 7 | * Licensed to the Apache Software Foundation (ASF) under one 8 | * or more contributor license agreements. See the NOTICE file 9 | * distributed with this work for additional information 10 | * regarding copyright ownership. The ASF licenses this file 11 | * to you under the Apache License, Version 2.0 (the 12 | * "License"); you may not use this file except in compliance 13 | * with the License. You may obtain a copy of the License at 14 | * 15 | * http://www.apache.org/licenses/LICENSE-2.0 16 | * 17 | * Unless required by applicable law or agreed to in writing, 18 | * software distributed under the License is distributed on an 19 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 20 | * KIND, either express or implied. See the License for the 21 | * specific language governing permissions and limitations 22 | * under the License. 23 | * 24 | */ 25 | 26 | public class Counter extends NativeType 27 | { 28 | public static final Counter INSTANCE = new Counter(); 29 | 30 | @Override 31 | public String name() 32 | { 33 | return "counter"; 34 | } 35 | 36 | @Override 37 | public boolean isSupported() 38 | { 39 | return false; 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/Duration.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import org.apache.cassandra.spark.data.fourzero.NativeType; 4 | 5 | /* 6 | * 7 | * Licensed to the Apache Software Foundation (ASF) under one 8 | * or more contributor license agreements. See the NOTICE file 9 | * distributed with this work for additional information 10 | * regarding copyright ownership. The ASF licenses this file 11 | * to you under the Apache License, Version 2.0 (the 12 | * "License"); you may not use this file except in compliance 13 | * with the License. You may obtain a copy of the License at 14 | * 15 | * http://www.apache.org/licenses/LICENSE-2.0 16 | * 17 | * Unless required by applicable law or agreed to in writing, 18 | * software distributed under the License is distributed on an 19 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 20 | * KIND, either express or implied. See the License for the 21 | * specific language governing permissions and limitations 22 | * under the License. 23 | * 24 | */ 25 | 26 | public class Duration extends NativeType 27 | { 28 | public static final Duration INSTANCE = new Duration(); 29 | 30 | @Override 31 | public String name() 32 | { 33 | return "duration"; 34 | } 35 | 36 | @Override 37 | public boolean isSupported() 38 | { 39 | return false; 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/reader/EmptyScanner.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.reader; 2 | 3 | /* 4 | * 5 | * Licensed to the Apache Software Foundation (ASF) under one 6 | * or more contributor license agreements. See the NOTICE file 7 | * distributed with this work for additional information 8 | * regarding copyright ownership. The ASF licenses this file 9 | * to you under the Apache License, Version 2.0 (the 10 | * "License"); you may not use this file except in compliance 11 | * with the License. You may obtain a copy of the License at 12 | * 13 | * http://www.apache.org/licenses/LICENSE-2.0 14 | * 15 | * Unless required by applicable law or agreed to in writing, 16 | * software distributed under the License is distributed on an 17 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 18 | * KIND, either express or implied. See the License for the 19 | * specific language governing permissions and limitations 20 | * under the License. 21 | * 22 | */ 23 | 24 | public class EmptyScanner implements IStreamScanner 25 | { 26 | public static final EmptyScanner INSTANCE = new EmptyScanner(); 27 | 28 | public Rid data() 29 | { 30 | return null; 31 | } 32 | 33 | public boolean next() 34 | { 35 | return false; 36 | } 37 | 38 | public void advanceToNextColumn() 39 | { 40 | } 41 | 42 | public void close() 43 | { 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/cdc/fourzero/CdcEventWriter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.cdc.fourzero; 23 | 24 | import org.apache.cassandra.spark.cdc.AbstractCdcEventWriter; 25 | import org.apache.cassandra.spark.cdc.SparkCdcEvent; 26 | import org.apache.spark.sql.Row; 27 | 28 | public abstract class CdcEventWriter extends AbstractCdcEventWriter 29 | { 30 | // Add final to prohibit overriding 31 | @Override 32 | public final void process(Row row) 33 | { 34 | processEvent(SparkCdcEvent.Builder.EMPTY.fromRow(row)); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /core/src/main/spark3/org/apache/cassandra/spark/sparksql/LocalDataSource.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.sparksql; 2 | 3 | import org.apache.cassandra.spark.data.DataLayer; 4 | import org.apache.cassandra.spark.data.LocalDataLayer; 5 | import org.apache.spark.sql.util.CaseInsensitiveStringMap; 6 | 7 | /* 8 | * 9 | * Licensed to the Apache Software Foundation (ASF) under one 10 | * or more contributor license agreements. See the NOTICE file 11 | * distributed with this work for additional information 12 | * regarding copyright ownership. The ASF licenses this file 13 | * to you under the Apache License, Version 2.0 (the 14 | * "License"); you may not use this file except in compliance 15 | * with the License. You may obtain a copy of the License at 16 | * 17 | * http://www.apache.org/licenses/LICENSE-2.0 18 | * 19 | * Unless required by applicable law or agreed to in writing, 20 | * software distributed under the License is distributed on an 21 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 22 | * KIND, either express or implied. See the License for the 23 | * specific language governing permissions and limitations 24 | * under the License. 25 | * 26 | */ 27 | 28 | @SuppressWarnings("unused") 29 | public class LocalDataSource extends CassandraTableProvider 30 | { 31 | @Override 32 | public String shortName() 33 | { 34 | return "localsstabledatasource"; 35 | } 36 | 37 | @Override 38 | public DataLayer getDataLayer(final CaseInsensitiveStringMap options) 39 | { 40 | return LocalDataLayer.from(options); 41 | } 42 | } -------------------------------------------------------------------------------- /core/src/main/spark3/org/apache/cassandra/spark/sparksql/LocalPartitionSizeSource.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.sparksql; 23 | 24 | import org.apache.cassandra.spark.data.DataLayer; 25 | import org.apache.cassandra.spark.data.LocalDataLayer; 26 | import org.apache.spark.sql.util.CaseInsensitiveStringMap; 27 | 28 | public class LocalPartitionSizeSource extends PartitionSizeTableProvider 29 | { 30 | @Override 31 | public String shortName() 32 | { 33 | return "localpartitionsizesource"; 34 | } 35 | 36 | @Override 37 | public DataLayer getDataLayer(final CaseInsensitiveStringMap options) 38 | { 39 | return LocalDataLayer.from(options); 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/utils/RandomUtils.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.utils; 2 | 3 | import java.nio.ByteBuffer; 4 | import java.util.Random; 5 | 6 | /* 7 | * 8 | * Licensed to the Apache Software Foundation (ASF) under one 9 | * or more contributor license agreements. See the NOTICE file 10 | * distributed with this work for additional information 11 | * regarding copyright ownership. The ASF licenses this file 12 | * to you under the Apache License, Version 2.0 (the 13 | * "License"); you may not use this file except in compliance 14 | * with the License. You may obtain a copy of the License at 15 | * 16 | * http://www.apache.org/licenses/LICENSE-2.0 17 | * 18 | * Unless required by applicable law or agreed to in writing, 19 | * software distributed under the License is distributed on an 20 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 21 | * KIND, either express or implied. See the License for the 22 | * specific language governing permissions and limitations 23 | * under the License. 24 | * 25 | */ 26 | 27 | public class RandomUtils 28 | { 29 | 30 | public static final Random RANDOM = new Random(); 31 | 32 | public static int randomPositiveInt(int bound) 33 | { 34 | return RANDOM.nextInt(bound - 1) + 1; 35 | } 36 | 37 | public static byte[] randomBytes(int len) 38 | { 39 | final byte[] b = new byte[len]; 40 | RANDOM.nextBytes(b); 41 | return b; 42 | } 43 | 44 | public static ByteBuffer randomByteBuffer(int len) 45 | { 46 | return ByteBuffer.wrap(randomBytes(len)); 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /example/src/main/spark2/org/apache/cassandra/spark/s3/S3DataSource.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.s3; 2 | 3 | import org.apache.cassandra.spark.data.DataLayer; 4 | import org.apache.cassandra.spark.data.partitioner.ConsistencyLevel; 5 | import org.apache.cassandra.spark.sparksql.CassandraDataSource; 6 | import org.apache.spark.sql.sources.v2.DataSourceOptions; 7 | 8 | // S3DataSource that implements Spark 2 CassandraDataSource 9 | public class S3DataSource extends CassandraDataSource 10 | { 11 | public DataLayer getDataLayer(DataSourceOptions options) 12 | { 13 | return new S3DataLayer( 14 | options.get("consistencyLevel").map(ConsistencyLevel::valueOf).orElse(ConsistencyLevel.LOCAL_QUORUM), 15 | options.get("clusterName").orElseThrow(() -> new RuntimeException("No cluster name specified")), 16 | options.get("keyspace").orElseThrow(() -> new RuntimeException("No keyspace specified")), 17 | options.get("table").orElseThrow(() -> new RuntimeException("No table specified")), 18 | options.get("tableCreateStmt").orElseThrow(() -> new RuntimeException("No tableCreateStmt specified")), 19 | options.get("DC").orElseThrow(() -> new RuntimeException("No DC specified")), 20 | options.get("s3-region").orElseThrow(() -> new RuntimeException("No S3 region specified")), 21 | options.get("s3-bucket").orElseThrow(() -> new RuntimeException("No S3 bucket specified")), 22 | options.getInt("defaultParallelism", 1), 23 | options.getInt("numCores", 1) 24 | ); 25 | } 26 | 27 | public String shortName() 28 | { 29 | return "s3-datasource"; 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /core/src/main/spark2/org/apache/cassandra/spark/sparksql/LocalDataSource.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.sparksql; 2 | 3 | import org.apache.cassandra.spark.data.DataLayer; 4 | import org.apache.cassandra.spark.data.LocalDataLayer; 5 | import org.apache.spark.sql.sources.v2.DataSourceOptions; 6 | 7 | /* 8 | * 9 | * Licensed to the Apache Software Foundation (ASF) under one 10 | * or more contributor license agreements. See the NOTICE file 11 | * distributed with this work for additional information 12 | * regarding copyright ownership. The ASF licenses this file 13 | * to you under the Apache License, Version 2.0 (the 14 | * "License"); you may not use this file except in compliance 15 | * with the License. You may obtain a copy of the License at 16 | * 17 | * http://www.apache.org/licenses/LICENSE-2.0 18 | * 19 | * Unless required by applicable law or agreed to in writing, 20 | * software distributed under the License is distributed on an 21 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 22 | * KIND, either express or implied. See the License for the 23 | * specific language governing permissions and limitations 24 | * under the License. 25 | * 26 | */ 27 | 28 | @SuppressWarnings("unused") 29 | public class LocalDataSource extends CassandraDataSource 30 | { 31 | @Override 32 | public String shortName() 33 | { 34 | return "localsstabledatasource"; 35 | } 36 | 37 | @Override 38 | public DataLayer getDataLayer(final DataSourceOptions options) 39 | { 40 | // options.asMap() returns the keyLowerCasedMap. All the keys need to be lower cased 41 | return LocalDataLayer.from(options.asMap()); 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/reader/fourzero/MapBuffer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.reader.fourzero; 23 | 24 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.rows.Cell; 25 | 26 | public class MapBuffer extends ComplexTypeBuffer 27 | { 28 | MapBuffer(int cellCount) 29 | { 30 | super(cellCount, cellCount * 2); 31 | } 32 | 33 | @Override 34 | public void addCell(Cell cell) 35 | { 36 | this.add(cell.path().get(0)); // map - copy over key and value 37 | super.addCell(cell); 38 | } 39 | 40 | @Override 41 | protected int elements() 42 | { 43 | // divide 2 because we add key and value to the buffer, which makes it twice as big as the map entries. 44 | return super.elements() / 2; 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/reader/BigNumberConfig.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.reader; 23 | 24 | public interface BigNumberConfig 25 | { 26 | BigNumberConfig DEFAULT = new BigNumberConfig() 27 | { 28 | public int bigIntegerPrecision() 29 | { 30 | return 38; 31 | } 32 | 33 | public int bigIntegerScale() 34 | { 35 | return 0; 36 | } 37 | 38 | public int bigDecimalPrecision() 39 | { 40 | return 38; 41 | } 42 | 43 | public int bigDecimalScale() 44 | { 45 | return 19; 46 | } 47 | }; 48 | 49 | int bigIntegerPrecision(); 50 | 51 | int bigIntegerScale(); 52 | 53 | int bigDecimalPrecision(); 54 | 55 | int bigDecimalScale(); 56 | } 57 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/data/fourzero/types/spark/Blob.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.data.fourzero.types.spark; 23 | 24 | import java.nio.ByteBuffer; 25 | 26 | import org.apache.cassandra.spark.data.SparkCqlField; 27 | import org.apache.cassandra.spark.utils.ByteBufUtils; 28 | import org.jetbrains.annotations.NotNull; 29 | 30 | public class Blob extends org.apache.cassandra.spark.data.fourzero.types.Blob implements SparkCqlField.BinaryTraits 31 | { 32 | public static final Blob INSTANCE = new Blob(); 33 | 34 | @Override 35 | public Object toSparkSqlType(@NotNull Object o, boolean isFrozen) 36 | { 37 | return ByteBufUtils.getArray((ByteBuffer) o); // byte[] 38 | } 39 | 40 | @Override 41 | public Object toTestRowType(Object value) 42 | { 43 | return ByteBuffer.wrap((byte[]) value); 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/StringBased.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import org.apache.commons.lang3.RandomStringUtils; 4 | 5 | import org.apache.cassandra.spark.data.fourzero.NativeType; 6 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.SettableByIndexData; 7 | import org.apache.cassandra.spark.utils.RandomUtils; 8 | 9 | /* 10 | * 11 | * Licensed to the Apache Software Foundation (ASF) under one 12 | * or more contributor license agreements. See the NOTICE file 13 | * distributed with this work for additional information 14 | * regarding copyright ownership. The ASF licenses this file 15 | * to you under the Apache License, Version 2.0 (the 16 | * "License"); you may not use this file except in compliance 17 | * with the License. You may obtain a copy of the License at 18 | * 19 | * http://www.apache.org/licenses/LICENSE-2.0 20 | * 21 | * Unless required by applicable law or agreed to in writing, 22 | * software distributed under the License is distributed on an 23 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 24 | * KIND, either express or implied. See the License for the 25 | * specific language governing permissions and limitations 26 | * under the License. 27 | * 28 | */ 29 | 30 | public abstract class StringBased extends NativeType 31 | { 32 | @Override 33 | public Object randomValue(int minCollectionSize) 34 | { 35 | return RandomStringUtils.randomAlphanumeric(RandomUtils.randomPositiveInt(32)); 36 | } 37 | 38 | @Override 39 | public void setInnerValue(SettableByIndexData udtValue, int pos, Object value) 40 | { 41 | udtValue.setString(pos, (String) value); 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /cdc/src/main/java/org/apache/cassandra/spark/cdc/jdk/JdkValueMetadata.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.cdc.jdk; 23 | 24 | import java.nio.ByteBuffer; 25 | 26 | import org.apache.cassandra.spark.cdc.RowSource; 27 | import org.apache.cassandra.spark.cdc.ValueWithMetadata; 28 | import org.apache.cassandra.spark.cdc.jdk.msg.Column; 29 | import org.apache.cassandra.spark.data.fourzero.FourZeroTypes; 30 | 31 | public class JdkValueMetadata extends ValueWithMetadata implements RowSource 32 | { 33 | public JdkValueMetadata(String columnName, 34 | String columnType, 35 | ByteBuffer value) 36 | { 37 | super(columnName, columnType, value); 38 | } 39 | 40 | public Column toRow() 41 | { 42 | return new Column(columnName, FourZeroTypes.INSTANCE.parseType(columnType), getValue()); 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/AvailabilityHint.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data; 2 | 3 | import java.util.Comparator; 4 | 5 | public enum AvailabilityHint 6 | { 7 | // 0 means high priority 8 | UP(0), MOVING(1), LEAVING(1), UNKNOWN(2), JOINING(2), DOWN(2); 9 | 10 | private final int priority; 11 | 12 | AvailabilityHint(int priority) 13 | { 14 | this.priority = priority; 15 | } 16 | 17 | public static final Comparator AVAILABILITY_HINT_COMPARATOR = Comparator.comparingInt((AvailabilityHint o) -> o.priority).reversed(); 18 | 19 | public static AvailabilityHint fromState(String status, String state) 20 | { 21 | if (status.equalsIgnoreCase(AvailabilityHint.DOWN.name())) 22 | { 23 | return AvailabilityHint.DOWN; 24 | } 25 | 26 | if (status.equalsIgnoreCase(AvailabilityHint.UNKNOWN.name())) 27 | { 28 | return AvailabilityHint.UNKNOWN; 29 | } 30 | 31 | if (state.equalsIgnoreCase("NORMAL")) 32 | { 33 | return AvailabilityHint.valueOf(status); 34 | } 35 | if (state.equalsIgnoreCase(AvailabilityHint.MOVING.name())) 36 | { 37 | return AvailabilityHint.MOVING; 38 | } 39 | if (state.equalsIgnoreCase(AvailabilityHint.LEAVING.name())) 40 | { 41 | return AvailabilityHint.LEAVING; 42 | } 43 | if (state.equalsIgnoreCase("STARTING")) 44 | { 45 | return AvailabilityHint.valueOf(status); 46 | } 47 | if (state.equalsIgnoreCase(AvailabilityHint.JOINING.name())) 48 | { 49 | return AvailabilityHint.JOINING; 50 | } 51 | 52 | return AvailabilityHint.UNKNOWN; 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/sparksql/filters/PruneColumnFilter.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.sparksql.filters; 2 | 3 | import java.util.Set; 4 | 5 | import org.jetbrains.annotations.NotNull; 6 | 7 | /* 8 | * 9 | * Licensed to the Apache Software Foundation (ASF) under one 10 | * or more contributor license agreements. See the NOTICE file 11 | * distributed with this work for additional information 12 | * regarding copyright ownership. The ASF licenses this file 13 | * to you under the Apache License, Version 2.0 (the 14 | * "License"); you may not use this file except in compliance 15 | * with the License. You may obtain a copy of the License at 16 | * 17 | * http://www.apache.org/licenses/LICENSE-2.0 18 | * 19 | * Unless required by applicable law or agreed to in writing, 20 | * software distributed under the License is distributed on an 21 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 22 | * KIND, either express or implied. See the License for the 23 | * specific language governing permissions and limitations 24 | * under the License. 25 | * 26 | */ 27 | 28 | /** 29 | * Prune column push-down filter to skip reading columns that are not needed. 30 | */ 31 | public class PruneColumnFilter 32 | { 33 | private final Set requiredColumns; 34 | 35 | public PruneColumnFilter(@NotNull final Set requiredColumns) 36 | { 37 | this.requiredColumns = requiredColumns; 38 | } 39 | 40 | public Set requiredColumns() { 41 | return requiredColumns; 42 | } 43 | 44 | public int size() { 45 | return requiredColumns.size(); 46 | } 47 | 48 | public boolean includeColumn(String columnName) 49 | { 50 | return requiredColumns.contains(columnName); 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /DEV-README.md: -------------------------------------------------------------------------------- 1 | # Spark Cassandra Bulkreader 2 | 3 | SBR supports Spark 2 (Scala 2.11 and 2.12) and Spark 3 (Scala 2.12). 4 | This project uses Gradle as the dependency management and build framework. 5 | 6 | ## Building 7 | 8 | SBR will build for Spark 2 and Scala 2.11 by default. Navigate to the 9 | top-level directory for this project. 10 | 11 | ```shell 12 | ./gradlew clean package 13 | ``` 14 | 15 | ## Spark 2 and Scala 2.12 16 | 17 | To build for Scala 2.12, export `SCALA_VERSION=2.12`: 18 | 19 | ```shell 20 | export SCALA_VERSION=2.12 21 | ./gradlew clean package 22 | ``` 23 | 24 | ## Spark 3 and Scala 2.12 25 | 26 | To build for Spark 3 and Scala 2.12, export both 27 | `SCALA_VERSION=2.12` and `SPARK_VERSION=3`: 28 | 29 | ```shell 30 | export SCALA_VERSION=2.12 31 | export SPARK_VERSION=3 32 | ./gradlew clean package 33 | ``` 34 | 35 | ## IntelliJ 36 | 37 | The project is well-supported in IntelliJ. Run the following profile to 38 | copy code style used for this project: 39 | 40 | ```shell 41 | ./gradlew copyCodeStyle 42 | ``` 43 | 44 | The project has different sources for Spark 2 and Spark 3. Spark 2 uses 45 | the `org.apache.spark.sql.sources.v2` APIs that have been deprecated in Spark 3. 46 | Spark 3 uses new APIs that live in the `org.apache.spark.sql.connector.read` 47 | namespace. By default, the project will load Spark 2 sources, but you can switch 48 | between sources by modifying the `gradle.properties` file. 49 | 50 | For Spark 3, use the following in `gradle.properties`: 51 | 52 | ```properties 53 | scala=2.12 54 | spark=3 55 | ``` 56 | 57 | And then Load Gradle Changes. In Mac, the shortcut to load gradle changes is 58 | Command + Shift + I. This will make the IDE 59 | pick up the Spark 3 sources, and you should now be able to develop against 60 | Spark 3 as well. -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/cdc/fourzero/LocalCdcEventWriter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.cdc.fourzero; 23 | 24 | import java.util.ArrayList; 25 | import java.util.List; 26 | 27 | import com.google.common.annotations.VisibleForTesting; 28 | 29 | import org.apache.cassandra.spark.cdc.SparkCdcEvent; 30 | 31 | /** 32 | * Local impl for _testing_ only 33 | */ 34 | @VisibleForTesting 35 | public class LocalCdcEventWriter extends CdcEventWriter 36 | { 37 | // Declared as `static` because the writer is serialized and deserialized to worker(s). 38 | // The instance passed to spark is not the instance that runs in worker(s). 39 | public static final List events = new ArrayList<>(); 40 | 41 | @Override 42 | public void processEvent(SparkCdcEvent event) 43 | { 44 | // simply collect the events produced in the test 45 | events.add(event); 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /cdc/src/main/java/org/apache/cassandra/spark/CdcKryoRegister.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark; 23 | 24 | import com.esotericsoftware.kryo.Kryo; 25 | import org.apache.cassandra.spark.sparksql.filters.CdcOffset; 26 | import org.apache.cassandra.spark.sparksql.filters.InstanceLogs; 27 | 28 | public class CdcKryoRegister extends BaseKryoRegister 29 | { 30 | private static final ThreadLocal KRYO = ThreadLocal.withInitial(() -> { 31 | final Kryo kryo = new Kryo(); 32 | new CdcKryoRegister().registerClasses(kryo); 33 | return kryo; 34 | }); 35 | 36 | public static Kryo kryo() 37 | { 38 | return KRYO.get(); 39 | } 40 | 41 | @Override 42 | public void registerClasses(Kryo kryo) 43 | { 44 | super.registerClasses(kryo); 45 | kryo.register(CdcOffset.class, CdcOffset.SERIALIZER); 46 | kryo.register(InstanceLogs.class, InstanceLogs.SERIALIZER); 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /cdc/src/main/java/org/apache/cassandra/spark/cdc/jdk/JdkRangeTombstone.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.cdc.jdk; 23 | 24 | import java.util.List; 25 | 26 | import org.apache.cassandra.spark.cdc.RangeTombstone; 27 | import org.apache.cassandra.spark.cdc.RowSource; 28 | import org.jetbrains.annotations.NotNull; 29 | 30 | public class JdkRangeTombstone extends RangeTombstone implements RowSource 31 | { 32 | public JdkRangeTombstone(@NotNull List startBound, boolean startInclusive, 33 | @NotNull List endBound, boolean endInclusive) 34 | { 35 | super(startBound, startInclusive, endBound, endInclusive); 36 | } 37 | 38 | public org.apache.cassandra.spark.cdc.jdk.msg.RangeTombstone toRow() 39 | { 40 | return new org.apache.cassandra.spark.cdc.jdk.msg.RangeTombstone(this); 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/Text.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType; 4 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.AbstractType; 5 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.UTF8Type; 6 | 7 | /* 8 | * 9 | * Licensed to the Apache Software Foundation (ASF) under one 10 | * or more contributor license agreements. See the NOTICE file 11 | * distributed with this work for additional information 12 | * regarding copyright ownership. The ASF licenses this file 13 | * to you under the Apache License, Version 2.0 (the 14 | * "License"); you may not use this file except in compliance 15 | * with the License. You may obtain a copy of the License at 16 | * 17 | * http://www.apache.org/licenses/LICENSE-2.0 18 | * 19 | * Unless required by applicable law or agreed to in writing, 20 | * software distributed under the License is distributed on an 21 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 22 | * KIND, either express or implied. See the License for the 23 | * specific language governing permissions and limitations 24 | * under the License. 25 | * 26 | */ 27 | 28 | public class Text extends StringBased 29 | { 30 | public static final Text INSTANCE = new Text(); 31 | 32 | @Override 33 | public String name() 34 | { 35 | return "text"; 36 | } 37 | 38 | @Override 39 | public AbstractType dataType() 40 | { 41 | return UTF8Type.instance; 42 | } 43 | 44 | @Override 45 | public DataType driverDataType(boolean isFrozen) 46 | { 47 | return org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType.text(); 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | # Compiled source # 2 | ################### 3 | *.com 4 | *.class 5 | *.dll 6 | *.exe 7 | *.o 8 | *.so 9 | *.pyc 10 | 11 | # Packages # 12 | ############ 13 | *.7z 14 | *.dmg 15 | *.gz 16 | *.iso 17 | *.rar 18 | *.tar 19 | *.zip 20 | 21 | # Logs and databases # 22 | ###################### 23 | *.log 24 | 25 | # OS generated files # 26 | ###################### 27 | .DS_Store* 28 | ehthumbs.db 29 | Icon? 30 | Thumbs.db 31 | 32 | # Editor Files # 33 | ################ 34 | *~ 35 | *.swp 36 | 37 | # Gradle Files # 38 | ################ 39 | .gradle 40 | .m2 41 | 42 | # Build output directories 43 | .dist 44 | /target 45 | */target 46 | */build 47 | 48 | # IntelliJ specific files/directories 49 | .out 50 | out 51 | .idea 52 | *.ipr 53 | *.iws 54 | *.iml 55 | atlassian-ide-plugin.xml 56 | 57 | # NetBeans specific files/directories 58 | .nbattrs 59 | *.rar 60 | *.tar 61 | *.zip 62 | 63 | # Logs and databases # 64 | ###################### 65 | *.log 66 | 67 | # OS generated files # 68 | ###################### 69 | .DS_Store* 70 | ehthumbs.db 71 | Icon? 72 | Thumbs.db 73 | 74 | # Editor Files # 75 | ################ 76 | *~ 77 | *.swp 78 | 79 | # Gradle Files # 80 | ################ 81 | .gradle 82 | .m2 83 | 84 | # Build output directories 85 | /target 86 | */target 87 | /build 88 | */build 89 | 90 | # IntelliJ specific files/directories 91 | out 92 | .idea 93 | *.ipr 94 | *.iws 95 | *.iml 96 | atlassian-ide-plugin.xml 97 | 98 | # NetBeans specific files/directories 99 | .nbattrs 100 | *.DS_Store 101 | ideagen/ 102 | tools/ideagen/ 103 | logs/ 104 | ivy.xml 105 | 106 | protogen/ 107 | /protogen/ 108 | */protogen/* 109 | 110 | GPATH 111 | GRTAGS 112 | GTAGS 113 | /pkg 114 | /dist 115 | 116 | /out 117 | /.classpath* 118 | /.idea_build 119 | 120 | 121 | 122 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/Ascii.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType; 4 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.AbstractType; 5 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.AsciiType; 6 | 7 | /* 8 | * 9 | * Licensed to the Apache Software Foundation (ASF) under one 10 | * or more contributor license agreements. See the NOTICE file 11 | * distributed with this work for additional information 12 | * regarding copyright ownership. The ASF licenses this file 13 | * to you under the Apache License, Version 2.0 (the 14 | * "License"); you may not use this file except in compliance 15 | * with the License. You may obtain a copy of the License at 16 | * 17 | * http://www.apache.org/licenses/LICENSE-2.0 18 | * 19 | * Unless required by applicable law or agreed to in writing, 20 | * software distributed under the License is distributed on an 21 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 22 | * KIND, either express or implied. See the License for the 23 | * specific language governing permissions and limitations 24 | * under the License. 25 | * 26 | */ 27 | 28 | public class Ascii extends StringBased 29 | { 30 | public static final Ascii INSTANCE = new Ascii(); 31 | 32 | @Override 33 | public String name() 34 | { 35 | return "ascii"; 36 | } 37 | 38 | @Override 39 | public AbstractType dataType() 40 | { 41 | return AsciiType.instance; 42 | } 43 | 44 | @Override 45 | public DataType driverDataType(boolean isFrozen) 46 | { 47 | return org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType.ascii(); 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/reader/SparkSSTableReader.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.reader; 2 | 3 | import java.math.BigInteger; 4 | 5 | import com.google.common.collect.Range; 6 | 7 | /* 8 | * 9 | * Licensed to the Apache Software Foundation (ASF) under one 10 | * or more contributor license agreements. See the NOTICE file 11 | * distributed with this work for additional information 12 | * regarding copyright ownership. The ASF licenses this file 13 | * to you under the Apache License, Version 2.0 (the 14 | * "License"); you may not use this file except in compliance 15 | * with the License. You may obtain a copy of the License at 16 | * 17 | * http://www.apache.org/licenses/LICENSE-2.0 18 | * 19 | * Unless required by applicable law or agreed to in writing, 20 | * software distributed under the License is distributed on an 21 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 22 | * KIND, either express or implied. See the License for the 23 | * specific language governing permissions and limitations 24 | * under the License. 25 | * 26 | */ 27 | public interface SparkSSTableReader 28 | { 29 | BigInteger firstToken(); 30 | 31 | BigInteger lastToken(); 32 | 33 | default Range range() 34 | { 35 | return Range.closed(firstToken(), lastToken()); 36 | } 37 | 38 | /** 39 | * @return true if this sstable should not be read as part of this Spark partition 40 | */ 41 | boolean ignore(); 42 | 43 | /** 44 | * @param reader sstable reader 45 | * @param range token range 46 | * @return true if SSTable reader overlaps with a given token range 47 | */ 48 | public static boolean overlaps(final SparkSSTableReader reader, final Range range) 49 | { 50 | return range.isConnected(reader.range()); 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/VarChar.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType; 4 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.AbstractType; 5 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.UTF8Type; 6 | 7 | /* 8 | * 9 | * Licensed to the Apache Software Foundation (ASF) under one 10 | * or more contributor license agreements. See the NOTICE file 11 | * distributed with this work for additional information 12 | * regarding copyright ownership. The ASF licenses this file 13 | * to you under the Apache License, Version 2.0 (the 14 | * "License"); you may not use this file except in compliance 15 | * with the License. You may obtain a copy of the License at 16 | * 17 | * http://www.apache.org/licenses/LICENSE-2.0 18 | * 19 | * Unless required by applicable law or agreed to in writing, 20 | * software distributed under the License is distributed on an 21 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 22 | * KIND, either express or implied. See the License for the 23 | * specific language governing permissions and limitations 24 | * under the License. 25 | * 26 | */ 27 | 28 | public class VarChar extends StringBased 29 | { 30 | public static final VarChar INSTANCE = new VarChar(); 31 | 32 | @Override 33 | public String name() 34 | { 35 | return "varchar"; 36 | } 37 | 38 | @Override 39 | public AbstractType dataType() 40 | { 41 | return UTF8Type.instance; 42 | } 43 | 44 | @Override 45 | public DataType driverDataType(boolean isFrozen) 46 | { 47 | return org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType.varchar(); 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /core/src/test/java/org/apache/cassandra/spark/data/fourzero/types/DateTypeTests.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import java.time.LocalDate; 4 | 5 | import org.junit.Test; 6 | 7 | import org.apache.cassandra.spark.reader.CassandraVersion; 8 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.serializers.SimpleDateSerializer; 9 | 10 | import static org.junit.Assert.assertEquals; 11 | import static org.junit.Assert.assertTrue; 12 | 13 | public class DateTypeTests 14 | { 15 | @Test 16 | public void testDateConversion() 17 | { 18 | final int cassandraDate = SimpleDateSerializer.dateStringToDays("2021-07-16"); 19 | assertTrue(cassandraDate < 0); 20 | assertEquals("2021-07-16", SimpleDateSerializer.instance.toString(cassandraDate)); 21 | final Object sparkSqlDate = org.apache.cassandra.spark.data.fourzero.types.spark.Date.INSTANCE.toSparkSqlType(cassandraDate, false); 22 | assertTrue(sparkSqlDate instanceof Integer); 23 | final int numDays = (int) sparkSqlDate; 24 | assertTrue(numDays > 0); 25 | final LocalDate end = LocalDate.of(1970, 1, 1) 26 | .plusDays(numDays); 27 | assertEquals(2021, end.getYear()); 28 | assertEquals(7, end.getMonthValue()); 29 | assertEquals(16, end.getDayOfMonth()); 30 | final Object cqlWriterObj = Date.INSTANCE.convertForCqlWriter(numDays, CassandraVersion.FOURZERO); 31 | final org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.LocalDate cqlWriterDate = (org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.LocalDate) cqlWriterObj; 32 | assertEquals(2021, cqlWriterDate.getYear()); 33 | assertEquals(7, cqlWriterDate.getMonth()); 34 | assertEquals(16, cqlWriterDate.getDay()); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /example/src/main/spark3/org/apache/cassandra/spark/s3/S3DataSource.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.s3; 2 | 3 | import java.util.Map; 4 | import java.util.function.Supplier; 5 | 6 | import org.apache.cassandra.spark.data.DataLayer; 7 | import org.apache.cassandra.spark.data.partitioner.ConsistencyLevel; 8 | import org.apache.cassandra.spark.sparksql.CassandraTableProvider; 9 | import org.apache.spark.sql.util.CaseInsensitiveStringMap; 10 | 11 | // S3DataSource that implements Spark 3 CassandraTableProvider 12 | public class S3DataSource extends CassandraTableProvider 13 | { 14 | public DataLayer getDataLayer(CaseInsensitiveStringMap options) 15 | { 16 | return new S3DataLayer( 17 | ConsistencyLevel.valueOf(options.getOrDefault("consistencyLevel", ConsistencyLevel.LOCAL_QUORUM.toString())), 18 | getOrThrow(options, "clusterName"), 19 | getOrThrow(options, "keyspace"), 20 | getOrThrow(options, "table"), 21 | getOrThrow(options, "tableCreateStmt"), 22 | getOrThrow(options, "DC"), 23 | getOrThrow(options, "s3-region"), 24 | getOrThrow(options, "s3-bucket"), 25 | options.getInt("defaultParallelism", 1), 26 | options.getInt("numCores", 1) 27 | ); 28 | } 29 | 30 | static String getOrThrow(Map options, String key) 31 | { 32 | return getOrThrow(options, key, () -> new RuntimeException("No " + key + " specified")); 33 | } 34 | 35 | static String getOrThrow(Map options, String key, Supplier throwable) 36 | { 37 | final String value = options.get(key); 38 | if (value == null) 39 | { 40 | throw throwable.get(); 41 | } 42 | return value; 43 | } 44 | 45 | public String shortName() 46 | { 47 | return "s3-datasource"; 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /cdc/src/test/java/org/apache/cassandra/spark/cdc/jdk/msg/CdcMessageTests.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.cdc.jdk.msg; 2 | 3 | import java.util.Date; 4 | 5 | import com.google.common.collect.ImmutableList; 6 | import org.junit.Test; 7 | 8 | import org.apache.cassandra.spark.cdc.AbstractCdcEvent; 9 | import org.apache.cassandra.spark.data.fourzero.types.BigInt; 10 | import org.apache.cassandra.spark.data.fourzero.types.Text; 11 | import org.apache.cassandra.spark.data.fourzero.types.Timestamp; 12 | import org.apache.cassandra.spark.data.fourzero.types.UUID; 13 | import org.apache.cassandra.spark.utils.TimeUtils; 14 | 15 | import static org.junit.Assert.assertEquals; 16 | 17 | public class CdcMessageTests 18 | { 19 | @Test 20 | public void testCdcMessage() 21 | { 22 | final long colA = (long) BigInt.INSTANCE.randomValue(1024); 23 | final java.util.UUID colB = java.util.UUID.randomUUID(); 24 | final String colC = Text.INSTANCE.randomValue(1024).toString(); 25 | final Date colD = (Date) Timestamp.INSTANCE.randomValue(1024); 26 | final long now = TimeUtils.nowMicros(); 27 | 28 | final CdcMessage msg = new CdcMessage( 29 | "ks", "tb", 30 | ImmutableList.of(new Column("a", BigInt.INSTANCE, colA)), 31 | ImmutableList.of(new Column("b", UUID.INSTANCE, colB)), 32 | ImmutableList.of(), 33 | ImmutableList.of(new Column("c", Text.INSTANCE, colC), new Column("d", Timestamp.INSTANCE, colD)), 34 | now, 35 | AbstractCdcEvent.Kind.INSERT, 36 | ImmutableList.of(), null, null 37 | ); 38 | assertEquals(4, msg.allColumns().size()); 39 | assertEquals("{\"operation\": INSERT, \"lastModifiedTimestamp\": " + now + ", \"a\": \"" + colA + "\", \"b\": \"" + colB + "\", \"c\": \"" + colC + "\", \"d\": \"" + colD + "\"}", 40 | msg.toString()); 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/data/SSTablesSupplier.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data; 2 | 3 | import java.io.IOException; 4 | import java.util.Set; 5 | 6 | import org.apache.cassandra.spark.reader.SparkSSTableReader; 7 | 8 | /* 9 | * 10 | * Licensed to the Apache Software Foundation (ASF) under one 11 | * or more contributor license agreements. See the NOTICE file 12 | * distributed with this work for additional information 13 | * regarding copyright ownership. The ASF licenses this file 14 | * to you under the Apache License, Version 2.0 (the 15 | * "License"); you may not use this file except in compliance 16 | * with the License. You may obtain a copy of the License at 17 | * 18 | * http://www.apache.org/licenses/LICENSE-2.0 19 | * 20 | * Unless required by applicable law or agreed to in writing, 21 | * software distributed under the License is distributed on an 22 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 23 | * KIND, either express or implied. See the License for the 24 | * specific language governing permissions and limitations 25 | * under the License. 26 | * 27 | */ 28 | public abstract class SSTablesSupplier 29 | { 30 | /** 31 | * Interface that opens a given SSTable and returns an SparkSSTableReader 32 | * 33 | * @param SparkSSTableReader 34 | */ 35 | public interface ReaderOpener 36 | { 37 | T openReader(final SSTable ssTable, final boolean isRepairPrimary) throws IOException; 38 | } 39 | 40 | /** 41 | * Open an SparkSSTableReader for each SSTable 42 | * 43 | * @param readerOpener open SparkSSTableReader for a given SSTable 44 | * @return set of open SparkSSTableReaders to pass into CompactionIterator 45 | */ 46 | public abstract Set openAll(final ReaderOpener readerOpener); 47 | } 48 | -------------------------------------------------------------------------------- /core/src/test/spark3/org/apache/cassandra/spark/cdc/RequireTwoReplicasLocalDataSource.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.cdc; 2 | 3 | import java.io.Serializable; 4 | 5 | import org.apache.cassandra.spark.data.DataLayer; 6 | import org.apache.cassandra.spark.data.LocalDataLayer; 7 | import org.apache.cassandra.spark.sparksql.CassandraTableProvider; 8 | import org.apache.spark.sql.util.CaseInsensitiveStringMap; 9 | 10 | /* 11 | * 12 | * Licensed to the Apache Software Foundation (ASF) under one 13 | * or more contributor license agreements. See the NOTICE file 14 | * distributed with this work for additional information 15 | * regarding copyright ownership. The ASF licenses this file 16 | * to you under the Apache License, Version 2.0 (the 17 | * "License"); you may not use this file except in compliance 18 | * with the License. You may obtain a copy of the License at 19 | * 20 | * http://www.apache.org/licenses/LICENSE-2.0 21 | * 22 | * Unless required by applicable law or agreed to in writing, 23 | * software distributed under the License is distributed on an 24 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 25 | * KIND, either express or implied. See the License for the 26 | * specific language governing permissions and limitations 27 | * under the License. 28 | * 29 | */ 30 | 31 | // Only used for testing 32 | public class RequireTwoReplicasLocalDataSource extends CassandraTableProvider implements Serializable 33 | { 34 | @Override 35 | public String shortName() 36 | { 37 | return "LocalDataSourceRequiresAtLeastTwoReplicasToCompact"; 38 | } 39 | 40 | @Override 41 | public DataLayer getDataLayer(final CaseInsensitiveStringMap options) 42 | { 43 | // overrides MinimumReplicasPerMutation for testing. 44 | return LocalDataLayer.from(options) 45 | .withMinimumReplicasPerMutation(2); 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/partitioner/Partitioner.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.partitioner; 2 | 3 | import java.math.BigInteger; 4 | 5 | /* 6 | * 7 | * Licensed to the Apache Software Foundation (ASF) under one 8 | * or more contributor license agreements. See the NOTICE file 9 | * distributed with this work for additional information 10 | * regarding copyright ownership. The ASF licenses this file 11 | * to you under the Apache License, Version 2.0 (the 12 | * "License"); you may not use this file except in compliance 13 | * with the License. You may obtain a copy of the License at 14 | * 15 | * http://www.apache.org/licenses/LICENSE-2.0 16 | * 17 | * Unless required by applicable law or agreed to in writing, 18 | * software distributed under the License is distributed on an 19 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 20 | * KIND, either express or implied. See the License for the 21 | * specific language governing permissions and limitations 22 | * under the License. 23 | * 24 | */ 25 | public enum Partitioner 26 | { 27 | RandomPartitioner(BigInteger.ZERO, BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE)), 28 | Murmur3Partitioner(BigInteger.valueOf(2).pow(63).negate(), 29 | BigInteger.valueOf(2).pow(63).subtract(BigInteger.ONE)); 30 | 31 | private final BigInteger minToken, maxToken; 32 | 33 | Partitioner(final BigInteger minToken, final BigInteger maxToken) 34 | { 35 | this.minToken = minToken; 36 | this.maxToken = maxToken; 37 | } 38 | 39 | public BigInteger minToken() 40 | { 41 | return minToken; 42 | } 43 | 44 | public BigInteger maxToken() 45 | { 46 | return maxToken; 47 | } 48 | 49 | @Override 50 | public String toString() 51 | { 52 | return "org.apache.cassandra.dht." + super.toString(); 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/data/fourzero/types/spark/Inet.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.data.fourzero.types.spark; 23 | 24 | import java.net.InetAddress; 25 | import java.net.UnknownHostException; 26 | 27 | import org.apache.cassandra.spark.data.SparkCqlField; 28 | import org.jetbrains.annotations.NotNull; 29 | 30 | public class Inet extends org.apache.cassandra.spark.data.fourzero.types.Inet implements SparkCqlField.BinaryTraits 31 | { 32 | public static Inet INSTANCE = new Inet(); 33 | 34 | @Override 35 | public Object toSparkSqlType(@NotNull Object o, boolean isFrozen) 36 | { 37 | return ((InetAddress) o).getAddress(); // byte[] 38 | } 39 | 40 | @Override 41 | public Object toTestRowType(Object value) 42 | { 43 | try 44 | { 45 | return InetAddress.getByAddress((byte[]) value); 46 | } 47 | catch (final UnknownHostException e) 48 | { 49 | throw new RuntimeException(e); 50 | } 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/IncompleteSSTableException.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data; 2 | 3 | import java.util.Arrays; 4 | 5 | import org.apache.cassandra.spark.reader.common.SSTableStreamException; 6 | import org.apache.cassandra.spark.utils.streaming.CassandraFile; 7 | import org.jetbrains.annotations.Nullable; 8 | 9 | /* 10 | * 11 | * Licensed to the Apache Software Foundation (ASF) under one 12 | * or more contributor license agreements. See the NOTICE file 13 | * distributed with this work for additional information 14 | * regarding copyright ownership. The ASF licenses this file 15 | * to you under the Apache License, Version 2.0 (the 16 | * "License"); you may not use this file except in compliance 17 | * with the License. You may obtain a copy of the License at 18 | * 19 | * http://www.apache.org/licenses/LICENSE-2.0 20 | * 21 | * Unless required by applicable law or agreed to in writing, 22 | * software distributed under the License is distributed on an 23 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 24 | * KIND, either express or implied. See the License for the 25 | * specific language governing permissions and limitations 26 | * under the License. 27 | * 28 | */ 29 | 30 | /** 31 | * Thrown when a snapshot of SSTables is incomplete so cannot be used 32 | */ 33 | @SuppressWarnings({ "unused", "WeakerAccess" }) 34 | public class IncompleteSSTableException extends SSTableStreamException 35 | { 36 | public IncompleteSSTableException(final CassandraFile.FileType... fileTypes) 37 | { 38 | super(String.format("SSTable file component '%s' is required but could not be found", Arrays.toString(fileTypes))); 39 | } 40 | 41 | public static boolean isIncompleteException(@Nullable final Throwable t) 42 | { 43 | return t != null && (t instanceof IncompleteSSTableException || isIncompleteException(t.getCause())); 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/utils/streaming/StreamConsumer.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.utils.streaming; 2 | 3 | /* 4 | * 5 | * Licensed to the Apache Software Foundation (ASF) under one 6 | * or more contributor license agreements. See the NOTICE file 7 | * distributed with this work for additional information 8 | * regarding copyright ownership. The ASF licenses this file 9 | * to you under the Apache License, Version 2.0 (the 10 | * "License"); you may not use this file except in compliance 11 | * with the License. You may obtain a copy of the License at 12 | * 13 | * http://www.apache.org/licenses/LICENSE-2.0 14 | * 15 | * Unless required by applicable law or agreed to in writing, 16 | * software distributed under the License is distributed on an 17 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 18 | * KIND, either express or implied. See the License for the 19 | * specific language governing permissions and limitations 20 | * under the License. 21 | * 22 | */ 23 | 24 | public interface StreamConsumer 25 | { 26 | /** 27 | * Called when {@link Source} completes a request and passes on the underlying bytes. 28 | * Note this can be called multiple times after a single {@link Source#request(long, long, StreamConsumer)} 29 | * 30 | * @param buffer StreamBuffer wrapping the bytes. 31 | */ 32 | void onRead(StreamBuffer buffer); 33 | 34 | /** 35 | * Called when {@link Source} has finished calling onRead for the last time after {@link Source#request(long, long, StreamConsumer)} was called. 36 | * Note {@link StreamConsumer#onRead(StreamBuffer)} may be called zero or more times before {@link StreamConsumer#onEnd()} is called. 37 | */ 38 | void onEnd(); 39 | 40 | /** 41 | * Called when {@link Source} fails for any reason to request the byte range. 42 | * 43 | * @param t throwable 44 | */ 45 | void onError(Throwable t); 46 | } -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/shaded/fourzero/cassandra/cql3/functions/types/UserTypeHelper.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types; 2 | 3 | import java.util.Collection; 4 | 5 | import org.apache.cassandra.spark.data.fourzero.FourZeroCqlType; 6 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.transport.ProtocolVersion; 7 | 8 | /* 9 | * 10 | * Licensed to the Apache Software Foundation (ASF) under one 11 | * or more contributor license agreements. See the NOTICE file 12 | * distributed with this work for additional information 13 | * regarding copyright ownership. The ASF licenses this file 14 | * to you under the Apache License, Version 2.0 (the 15 | * "License"); you may not use this file except in compliance 16 | * with the License. You may obtain a copy of the License at 17 | * 18 | * http://www.apache.org/licenses/LICENSE-2.0 19 | * 20 | * Unless required by applicable law or agreed to in writing, 21 | * software distributed under the License is distributed on an 22 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 23 | * KIND, either express or implied. See the License for the 24 | * specific language governing permissions and limitations 25 | * under the License. 26 | * 27 | */ 28 | public class UserTypeHelper 29 | { 30 | 31 | // helper methods to access package-private UDT methods 32 | 33 | public static UDTValue newUDTValue(final UserType userType) 34 | { 35 | return new UDTValue(userType); 36 | } 37 | 38 | public static UserType newUserType(String keyspace, String typeName, boolean frozen, Collection fields, ProtocolVersion protocolVersion) 39 | { 40 | return new UserType(keyspace, typeName, frozen, fields, protocolVersion, FourZeroCqlType.CODEC_REGISTRY); 41 | } 42 | 43 | public static UserType.Field newField(String name, DataType type) 44 | { 45 | return new UserType.Field(name, type); 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/reader/common/SSTableStreamException.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.reader.common; 2 | 3 | import java.io.IOException; 4 | 5 | import org.jetbrains.annotations.Nullable; 6 | 7 | /* 8 | * 9 | * Licensed to the Apache Software Foundation (ASF) under one 10 | * or more contributor license agreements. See the NOTICE file 11 | * distributed with this work for additional information 12 | * regarding copyright ownership. The ASF licenses this file 13 | * to you under the Apache License, Version 2.0 (the 14 | * "License"); you may not use this file except in compliance 15 | * with the License. You may obtain a copy of the License at 16 | * 17 | * http://www.apache.org/licenses/LICENSE-2.0 18 | * 19 | * Unless required by applicable law or agreed to in writing, 20 | * software distributed under the License is distributed on an 21 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 22 | * KIND, either express or implied. See the License for the 23 | * specific language governing permissions and limitations 24 | * under the License. 25 | * 26 | */ 27 | 28 | /** 29 | * An unchecked wrapper around IOException 30 | */ 31 | public class SSTableStreamException extends RuntimeException 32 | { 33 | public SSTableStreamException(final String msg) 34 | { 35 | this(new IOException(msg)); 36 | } 37 | 38 | public SSTableStreamException(final IOException e) 39 | { 40 | super(e); 41 | } 42 | 43 | public IOException getIOException() 44 | { 45 | return (IOException) getCause(); 46 | } 47 | 48 | @Nullable 49 | public static IOException getIOException(@Nullable final Throwable t) 50 | { 51 | if (t == null) 52 | { 53 | return null; 54 | } 55 | if (t instanceof SSTableStreamException) 56 | { 57 | return ((SSTableStreamException) t).getIOException(); 58 | } 59 | 60 | return getIOException(t.getCause()); 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /core/src/test/java/org/apache/cassandra/spark/data/VersionRunner.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data; 2 | 3 | import java.util.Arrays; 4 | import java.util.Collection; 5 | 6 | import org.junit.Before; 7 | import org.junit.runner.RunWith; 8 | import org.junit.runners.Parameterized; 9 | 10 | import org.apache.cassandra.spark.reader.CassandraBridge; 11 | import org.apache.cassandra.spark.reader.CassandraVersion; 12 | 13 | /* 14 | * 15 | * Licensed to the Apache Software Foundation (ASF) under one 16 | * or more contributor license agreements. See the NOTICE file 17 | * distributed with this work for additional information 18 | * regarding copyright ownership. The ASF licenses this file 19 | * to you under the Apache License, Version 2.0 (the 20 | * "License"); you may not use this file except in compliance 21 | * with the License. You may obtain a copy of the License at 22 | * 23 | * http://www.apache.org/licenses/LICENSE-2.0 24 | * 25 | * Unless required by applicable law or agreed to in writing, 26 | * software distributed under the License is distributed on an 27 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 28 | * KIND, either express or implied. See the License for the 29 | * specific language governing permissions and limitations 30 | * under the License. 31 | * 32 | */ 33 | 34 | // run tests Parameterized for multiple versions of Cassandra 35 | @RunWith(Parameterized.class) 36 | public abstract class VersionRunner 37 | { 38 | public final CassandraVersion version; 39 | public CassandraBridge bridge; 40 | 41 | @Parameterized.Parameters 42 | public static Collection versions() 43 | { 44 | return Arrays.asList(new Object[][]{ 45 | { CassandraVersion.FOURZERO } 46 | }); 47 | } 48 | 49 | public VersionRunner(CassandraVersion version) 50 | { 51 | this.version = version; 52 | } 53 | 54 | @Before 55 | public void initialize() 56 | { 57 | this.bridge = CassandraBridge.get(version); 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/data/fourzero/types/spark/VarInt.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types.spark; 2 | 3 | import java.math.BigDecimal; 4 | import java.math.BigInteger; 5 | 6 | import org.apache.cassandra.spark.data.SparkCqlField; 7 | import org.apache.cassandra.spark.reader.BigNumberConfig; 8 | import org.apache.spark.sql.Row; 9 | import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; 10 | import org.apache.spark.sql.types.DataType; 11 | import org.apache.spark.sql.types.DataTypes; 12 | import org.jetbrains.annotations.NotNull; 13 | 14 | public class VarInt extends org.apache.cassandra.spark.data.fourzero.types.VarInt implements SparkCqlField.DecimalTraits 15 | { 16 | public static final VarInt INSTANCE = new VarInt(); 17 | 18 | @Override 19 | public DataType sparkSqlType(BigNumberConfig bigNumberConfig) 20 | { 21 | return DataTypes.createDecimalType(bigNumberConfig.bigIntegerPrecision(), bigNumberConfig.bigIntegerScale()); 22 | } 23 | 24 | @Override 25 | public Object toTestRowType(Object value) 26 | { 27 | if (value instanceof BigInteger) 28 | { 29 | return value; 30 | } 31 | else if (value instanceof BigDecimal) 32 | { 33 | return ((BigDecimal) value).toBigInteger(); 34 | } 35 | return ((org.apache.spark.sql.types.Decimal) value).toJavaBigInteger(); 36 | } 37 | 38 | @Override 39 | public Object toSparkSqlType(@NotNull Object o, boolean isFrozen) 40 | { 41 | return org.apache.spark.sql.types.Decimal.apply((BigInteger) o); 42 | } 43 | 44 | @Override 45 | public Object nativeSparkSqlRowValue(final GenericInternalRow row, final int pos) 46 | { 47 | return row.getDecimal(pos, BigNumberConfig.DEFAULT.bigIntegerPrecision(), BigNumberConfig.DEFAULT.bigIntegerScale()); 48 | } 49 | 50 | @Override 51 | public Object nativeSparkSqlRowValue(Row row, int pos) 52 | { 53 | return row.getDecimal(pos).toBigInteger(); 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/utils/IOUtils.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.utils; 2 | 3 | import java.io.IOException; 4 | import java.nio.file.Files; 5 | import java.nio.file.Path; 6 | 7 | import org.slf4j.Logger; 8 | import org.slf4j.LoggerFactory; 9 | 10 | import org.jetbrains.annotations.Nullable; 11 | 12 | /* 13 | * 14 | * Licensed to the Apache Software Foundation (ASF) under one 15 | * or more contributor license agreements. See the NOTICE file 16 | * distributed with this work for additional information 17 | * regarding copyright ownership. The ASF licenses this file 18 | * to you under the Apache License, Version 2.0 (the 19 | * "License"); you may not use this file except in compliance 20 | * with the License. You may obtain a copy of the License at 21 | * 22 | * http://www.apache.org/licenses/LICENSE-2.0 23 | * 24 | * Unless required by applicable law or agreed to in writing, 25 | * software distributed under the License is distributed on an 26 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 27 | * KIND, either express or implied. See the License for the 28 | * specific language governing permissions and limitations 29 | * under the License. 30 | * 31 | */ 32 | 33 | public class IOUtils 34 | { 35 | private static final Logger LOGGER = LoggerFactory.getLogger(IOUtils.class); 36 | 37 | public static long fileLength(@Nullable final Path path) 38 | { 39 | try 40 | { 41 | return path == null ? 0 : Files.size(path); 42 | } 43 | catch (IOException e) 44 | { 45 | throw new RuntimeException(e); 46 | } 47 | } 48 | 49 | public static void closeQuietly(@Nullable final AutoCloseable closeable) 50 | { 51 | if (closeable == null) 52 | { 53 | return; 54 | } 55 | 56 | try 57 | { 58 | closeable.close(); 59 | } 60 | catch (Throwable t) 61 | { 62 | LOGGER.warn("Exception closing {}", closeable.getClass().getName(), t); 63 | } 64 | } 65 | } 66 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/TimeUUID.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType; 4 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.AbstractType; 5 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.TimeUUIDType; 6 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.utils.UUIDGen; 7 | 8 | /* 9 | * 10 | * Licensed to the Apache Software Foundation (ASF) under one 11 | * or more contributor license agreements. See the NOTICE file 12 | * distributed with this work for additional information 13 | * regarding copyright ownership. The ASF licenses this file 14 | * to you under the Apache License, Version 2.0 (the 15 | * "License"); you may not use this file except in compliance 16 | * with the License. You may obtain a copy of the License at 17 | * 18 | * http://www.apache.org/licenses/LICENSE-2.0 19 | * 20 | * Unless required by applicable law or agreed to in writing, 21 | * software distributed under the License is distributed on an 22 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 23 | * KIND, either express or implied. See the License for the 24 | * specific language governing permissions and limitations 25 | * under the License. 26 | * 27 | */ 28 | 29 | public class TimeUUID extends UUID 30 | { 31 | public static final TimeUUID INSTANCE = new TimeUUID(); 32 | 33 | @Override 34 | public String name() 35 | { 36 | return "timeuuid"; 37 | } 38 | 39 | @Override 40 | public AbstractType dataType() 41 | { 42 | return TimeUUIDType.instance; 43 | } 44 | 45 | @Override 46 | public Object randomValue(int minCollectionSize) 47 | { 48 | return UUIDGen.getTimeUUID(); 49 | } 50 | 51 | @Override 52 | public DataType driverDataType(boolean isFrozen) 53 | { 54 | return org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType.timeuuid(); 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /common/src/test/java/org/apache/cassandra/spark/utils/LoggerHelperTests.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.utils; 2 | 3 | import org.junit.Test; 4 | import org.slf4j.Logger; 5 | import org.slf4j.LoggerFactory; 6 | 7 | import static org.junit.Assert.assertArrayEquals; 8 | import static org.junit.Assert.assertEquals; 9 | 10 | /* 11 | * 12 | * Licensed to the Apache Software Foundation (ASF) under one 13 | * or more contributor license agreements. See the NOTICE file 14 | * distributed with this work for additional information 15 | * regarding copyright ownership. The ASF licenses this file 16 | * to you under the Apache License, Version 2.0 (the 17 | * "License"); you may not use this file except in compliance 18 | * with the License. You may obtain a copy of the License at 19 | * 20 | * http://www.apache.org/licenses/LICENSE-2.0 21 | * 22 | * Unless required by applicable law or agreed to in writing, 23 | * software distributed under the License is distributed on an 24 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 25 | * KIND, either express or implied. See the License for the 26 | * specific language governing permissions and limitations 27 | * under the License. 28 | * 29 | */ 30 | 31 | public class LoggerHelperTests 32 | { 33 | private static final Logger LOGGER = LoggerFactory.getLogger(LoggerHelperTests.class); 34 | 35 | @Test 36 | public void testBuildArgs() 37 | { 38 | final LoggerHelper loggerHelper = new LoggerHelper(LOGGER, "a", "1", "b", "2", "c", "3"); 39 | final Object[] additionalArgs = new Object[]{ "d", "4", "e", "5", "f", "6" }; 40 | assertEquals("hello this is the log message a={} b={} c={} d={} e={} f={}", 41 | loggerHelper.logMsg("hello this is the log message", additionalArgs)); 42 | assertArrayEquals(new Object[]{ "1", "2", "3", "4", "5", "6" }, loggerHelper.buildArgs(null, additionalArgs)); 43 | final Throwable t = new RuntimeException("Error"); 44 | assertArrayEquals(new Object[]{ "1", "2", "3", "4", "5", "6", t }, loggerHelper.buildArgs(t, additionalArgs)); 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /cdc/src/main/java/org/apache/cassandra/spark/cdc/ValueWithMetadata.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.cdc; 2 | 3 | import java.nio.ByteBuffer; 4 | 5 | import org.apache.cassandra.spark.utils.ByteBufUtils; 6 | 7 | /* 8 | * 9 | * Licensed to the Apache Software Foundation (ASF) under one 10 | * or more contributor license agreements. See the NOTICE file 11 | * distributed with this work for additional information 12 | * regarding copyright ownership. The ASF licenses this file 13 | * to you under the Apache License, Version 2.0 (the 14 | * "License"); you may not use this file except in compliance 15 | * with the License. You may obtain a copy of the License at 16 | * 17 | * http://www.apache.org/licenses/LICENSE-2.0 18 | * 19 | * Unless required by applicable law or agreed to in writing, 20 | * software distributed under the License is distributed on an 21 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 22 | * KIND, either express or implied. See the License for the 23 | * specific language governing permissions and limitations 24 | * under the License. 25 | * 26 | */ 27 | 28 | public abstract class ValueWithMetadata 29 | { 30 | public final String columnName; 31 | public final String columnType; 32 | private final ByteBuffer value; 33 | 34 | public ValueWithMetadata(String columnName, String columnType, ByteBuffer value) 35 | { 36 | this.columnName = columnName; 37 | this.columnType = columnType; 38 | this.value = value; 39 | } 40 | 41 | /** 42 | * @return the value as byte array 43 | */ 44 | public byte[] getBytes() 45 | { 46 | // if bb is null, we should return null; Null means deletion 47 | if (value == null) 48 | { 49 | return null; 50 | } 51 | return ByteBufUtils.getArray(value); 52 | } 53 | 54 | /** 55 | * @return the duplicated {@link ByteBuffer} of the value 56 | */ 57 | public ByteBuffer getValue() 58 | { 59 | if (value == null) 60 | { 61 | return null; 62 | } 63 | return value.duplicate(); 64 | } 65 | } -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/Time.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType; 4 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.AbstractType; 5 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.TimeType; 6 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.SettableByIndexData; 7 | 8 | /* 9 | * 10 | * Licensed to the Apache Software Foundation (ASF) under one 11 | * or more contributor license agreements. See the NOTICE file 12 | * distributed with this work for additional information 13 | * regarding copyright ownership. The ASF licenses this file 14 | * to you under the Apache License, Version 2.0 (the 15 | * "License"); you may not use this file except in compliance 16 | * with the License. You may obtain a copy of the License at 17 | * 18 | * http://www.apache.org/licenses/LICENSE-2.0 19 | * 20 | * Unless required by applicable law or agreed to in writing, 21 | * software distributed under the License is distributed on an 22 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 23 | * KIND, either express or implied. See the License for the 24 | * specific language governing permissions and limitations 25 | * under the License. 26 | * 27 | */ 28 | 29 | public class Time extends LongBased 30 | { 31 | public static final Time INSTANCE = new Time(); 32 | 33 | @Override 34 | public String name() 35 | { 36 | return "time"; 37 | } 38 | 39 | @Override 40 | public AbstractType dataType() 41 | { 42 | return TimeType.instance; 43 | } 44 | 45 | @Override 46 | public void setInnerValue(SettableByIndexData udtValue, int pos, Object value) 47 | { 48 | udtValue.setTime(pos, (long) value); 49 | } 50 | 51 | @Override 52 | public DataType driverDataType(boolean isFrozen) 53 | { 54 | return org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType.time(); 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/BigInt.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType; 4 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.AbstractType; 5 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.LongType; 6 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.SettableByIndexData; 7 | 8 | /* 9 | * 10 | * Licensed to the Apache Software Foundation (ASF) under one 11 | * or more contributor license agreements. See the NOTICE file 12 | * distributed with this work for additional information 13 | * regarding copyright ownership. The ASF licenses this file 14 | * to you under the Apache License, Version 2.0 (the 15 | * "License"); you may not use this file except in compliance 16 | * with the License. You may obtain a copy of the License at 17 | * 18 | * http://www.apache.org/licenses/LICENSE-2.0 19 | * 20 | * Unless required by applicable law or agreed to in writing, 21 | * software distributed under the License is distributed on an 22 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 23 | * KIND, either express or implied. See the License for the 24 | * specific language governing permissions and limitations 25 | * under the License. 26 | * 27 | */ 28 | 29 | public class BigInt extends LongBased 30 | { 31 | public static final BigInt INSTANCE = new BigInt(); 32 | 33 | @Override 34 | public String name() 35 | { 36 | return "bigint"; 37 | } 38 | 39 | @Override 40 | public AbstractType dataType() 41 | { 42 | return LongType.instance; 43 | } 44 | 45 | @Override 46 | public void setInnerValue(SettableByIndexData udtValue, int pos, Object value) 47 | { 48 | udtValue.setLong(pos, (long) value); 49 | } 50 | 51 | @Override 52 | public DataType driverDataType(boolean isFrozen) 53 | { 54 | return org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType.bigint(); 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/stats/IStats.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.stats; 23 | 24 | import org.apache.cassandra.spark.utils.streaming.Source; 25 | import org.apache.cassandra.spark.utils.streaming.CassandraFile; 26 | 27 | public interface IStats 28 | { 29 | IStats DO_NOTHING = new IStats() 30 | { 31 | }; 32 | 33 | default void inputStreamEnd(Source source, long runTimeNanos, long totalNanosBlocked) 34 | { 35 | 36 | } 37 | 38 | default void inputStreamEndBuffer(Source ssTable) 39 | { 40 | 41 | } 42 | 43 | default void inputStreamTimeBlocked(Source source, long nanos) 44 | { 45 | 46 | } 47 | 48 | default void inputStreamByteRead(Source source, int len, int queueSize, int percentComplete) 49 | { 50 | 51 | } 52 | 53 | default void inputStreamFailure(Source source, Throwable t) 54 | { 55 | 56 | } 57 | 58 | default void inputStreamBytesWritten(Source ssTable, int len) 59 | { 60 | 61 | } 62 | 63 | default void inputStreamBytesSkipped(Source source, long bufferedSkipped, long rangeSkipped) 64 | { 65 | 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/shaded/fourzero/cassandra/cql3/functions/types/TupleHelper.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types; 2 | 3 | import java.util.stream.Collectors; 4 | 5 | import org.apache.cassandra.spark.data.fourzero.FourZeroCqlType; 6 | import org.apache.cassandra.spark.data.fourzero.complex.CqlTuple; 7 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.transport.ProtocolVersion; 8 | 9 | /* 10 | * 11 | * Licensed to the Apache Software Foundation (ASF) under one 12 | * or more contributor license agreements. See the NOTICE file 13 | * distributed with this work for additional information 14 | * regarding copyright ownership. The ASF licenses this file 15 | * to you under the Apache License, Version 2.0 (the 16 | * "License"); you may not use this file except in compliance 17 | * with the License. You may obtain a copy of the License at 18 | * 19 | * http://www.apache.org/licenses/LICENSE-2.0 20 | * 21 | * Unless required by applicable law or agreed to in writing, 22 | * software distributed under the License is distributed on an 23 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 24 | * KIND, either express or implied. See the License for the 25 | * specific language governing permissions and limitations 26 | * under the License. 27 | * 28 | */ 29 | public class TupleHelper 30 | { 31 | // helper methods to access package-private Tuple methods 32 | 33 | public static TupleType buildTupleType(CqlTuple tuple, boolean isFrozen) 34 | { 35 | return new TupleType( 36 | tuple.types().stream() 37 | .map(type -> (FourZeroCqlType) type) 38 | .map(type -> type.driverDataType(isFrozen)).collect(Collectors.toList()), 39 | ProtocolVersion.V3, FourZeroCqlType.CODEC_REGISTRY 40 | ); 41 | } 42 | 43 | public static TupleValue buildTupleValue(final CqlTuple tuple) 44 | { 45 | return buildTupleValue(tuple, false); 46 | } 47 | 48 | public static TupleValue buildTupleValue(final CqlTuple tuple, boolean isFrozen) 49 | { 50 | return new TupleValue(buildTupleType(tuple, isFrozen)); 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /cdc/build.gradle: -------------------------------------------------------------------------------- 1 | project(':cdc') { 2 | apply plugin: 'jacoco' 3 | apply plugin: 'java' 4 | 5 | apply from: "../profiles/scala-${project.rootProject.ext.crossbuildVersion}${project.rootProject.ext.sparkVersionQualifier}.gradle" 6 | 7 | dependencies { 8 | configurations.all { 9 | resolutionStrategy.force "com.fasterxml.jackson.core:jackson-databind:${jacksonVersion}" 10 | } 11 | 12 | implementation project(path: ':common') 13 | implementation project(path: ':fourzero', configuration: 'shadow') 14 | implementation "org.slf4j:slf4j-api:${slf4jApiVersion}" 15 | implementation "com.fasterxml.jackson.core:jackson-annotations:${jacksonCoreVersion}" 16 | implementation "com.fasterxml.jackson.core:jackson-core:${jacksonCoreVersion}" 17 | implementation "com.fasterxml.jackson.core:jackson-databind:${jacksonVersion}" 18 | implementation "org.apache.commons:commons-lang3:${commonsLang3Version}" 19 | implementation "commons-lang:commons-lang:${commonsLangVersion}" 20 | implementation "com.esotericsoftware:kryo-shaded:${kryoVersion}" 21 | implementation "com.google.guava:guava:${guavaVersion}" 22 | 23 | testImplementation project(path: ':common', configuration: 'testArtifacts') 24 | testImplementation "junit:junit:${project.rootProject.junitVersion}" 25 | testImplementation "commons-codec:commons-codec:${project.rootProject.commonsCodecVersion}" 26 | testImplementation "org.quicktheories:quicktheories:${project.rootProject.quickTheoriesVersion}" 27 | testImplementation "org.slf4j:slf4j-simple:1.7.26" 28 | testImplementation "org.mockito:mockito-all:${project.rootProject.mockitoVersion}" 29 | } 30 | 31 | configurations { 32 | testArtifacts 33 | } 34 | task testJar(type: Jar) { 35 | baseName = "${project.name}-test" 36 | from sourceSets.test.output 37 | } 38 | artifacts { 39 | testArtifacts testJar 40 | } 41 | 42 | version = (System.getenv("CODE_VERSION") != null ? System.getenv("CODE_VERSION") : version) + "_${scalaMajorVersion}${sparkVersionQualifier}" 43 | println "Common Version: $version" 44 | } -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/utils/streaming/CassandraFile.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.utils.streaming; 2 | 3 | import java.nio.file.Files; 4 | import java.nio.file.Path; 5 | import java.util.HashMap; 6 | import java.util.Map; 7 | 8 | import com.google.common.base.Preconditions; 9 | 10 | import org.jetbrains.annotations.Nullable; 11 | 12 | public interface CassandraFile 13 | { 14 | enum FileType 15 | { 16 | DATA("Data.db"), 17 | INDEX("Index.db"), 18 | FILTER("Filter.db"), 19 | STATISTICS("Statistics.db"), 20 | SUMMARY("Summary.db"), 21 | COMPRESSION_INFO("CompressionInfo.db"), 22 | TOC("TOC.txt"), 23 | DIGEST("Digest.sha1"), 24 | CRC("CRC.db"), 25 | CRC32("Digest.crc32"), 26 | COMMITLOG(".log"); 27 | 28 | private final String fileSuffix; 29 | 30 | FileType(final String fileSuffix) 31 | { 32 | this.fileSuffix = fileSuffix; 33 | } 34 | 35 | private static final Map FILE_TYPE_HASH_MAP = new HashMap<>(); 36 | 37 | static 38 | { 39 | for (final CassandraFile.FileType fileType : FileType.values()) 40 | { 41 | FILE_TYPE_HASH_MAP.put(fileType.getFileSuffix(), fileType); 42 | } 43 | } 44 | 45 | public static CassandraFile.FileType fromExtension(final String extension) 46 | { 47 | Preconditions.checkArgument(FILE_TYPE_HASH_MAP.containsKey(extension), "Unknown sstable file type: " + extension); 48 | return FILE_TYPE_HASH_MAP.get(extension); 49 | } 50 | 51 | @Nullable 52 | public static Path resolveComponentFile(final FileType fileType, final Path dataFilePath) 53 | { 54 | final Path filePath = fileType == FileType.DATA ? dataFilePath : dataFilePath.resolveSibling(dataFilePath.getFileName().toString().replace(FileType.DATA.getFileSuffix(), fileType.getFileSuffix())); 55 | return Files.exists(filePath) ? filePath : null; 56 | } 57 | 58 | public String getFileSuffix() 59 | { 60 | return fileSuffix; 61 | } 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/Empty.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import org.apache.cassandra.spark.data.fourzero.NativeType; 4 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.SettableByIndexData; 5 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.AbstractType; 6 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.EmptyType; 7 | 8 | /* 9 | * 10 | * Licensed to the Apache Software Foundation (ASF) under one 11 | * or more contributor license agreements. See the NOTICE file 12 | * distributed with this work for additional information 13 | * regarding copyright ownership. The ASF licenses this file 14 | * to you under the Apache License, Version 2.0 (the 15 | * "License"); you may not use this file except in compliance 16 | * with the License. You may obtain a copy of the License at 17 | * 18 | * http://www.apache.org/licenses/LICENSE-2.0 19 | * 20 | * Unless required by applicable law or agreed to in writing, 21 | * software distributed under the License is distributed on an 22 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 23 | * KIND, either express or implied. See the License for the 24 | * specific language governing permissions and limitations 25 | * under the License. 26 | * 27 | */ 28 | 29 | public class Empty extends NativeType 30 | { 31 | public static final Empty INSTANCE = new Empty(); 32 | 33 | @Override 34 | public boolean isSupported() 35 | { 36 | return false; 37 | } 38 | 39 | @Override 40 | public String name() 41 | { 42 | return "empty"; 43 | } 44 | 45 | @Override 46 | public AbstractType dataType() 47 | { 48 | return EmptyType.instance; 49 | } 50 | 51 | @Override 52 | public int cardinality(int orElse) 53 | { 54 | return 1; 55 | } 56 | 57 | @Override 58 | public Object randomValue(int minCollectionSize) 59 | { 60 | return null; 61 | } 62 | 63 | @Override 64 | public void setInnerValue(SettableByIndexData udtValue, int pos, Object value) 65 | { 66 | udtValue.setToNull(pos); 67 | } 68 | } 69 | -------------------------------------------------------------------------------- /core/src/test/spark3/org/apache/cassandra/spark/cdc/SpyWatermarkerDataSource.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.cdc; 2 | 3 | import org.apache.cassandra.spark.cdc.watermarker.InMemoryWatermarker; 4 | import org.apache.cassandra.spark.cdc.watermarker.SparkInMemoryWatermarker; 5 | import org.apache.cassandra.spark.cdc.watermarker.Watermarker; 6 | import org.apache.cassandra.spark.data.DataLayer; 7 | import org.apache.cassandra.spark.data.LocalDataLayer; 8 | import org.apache.cassandra.spark.reader.CassandraVersion; 9 | import org.apache.cassandra.spark.sparksql.CassandraTableProvider; 10 | import org.apache.spark.sql.util.CaseInsensitiveStringMap; 11 | import org.jetbrains.annotations.NotNull; 12 | 13 | import java.io.Serializable; 14 | 15 | import static org.apache.cassandra.spark.data.LocalDataLayer.getOrThrow; 16 | import static org.apache.cassandra.spark.data.LocalDataLayer.lowerCaseKey; 17 | import static org.mockito.Mockito.spy; 18 | 19 | public class SpyWatermarkerDataSource extends CassandraTableProvider implements Serializable 20 | { 21 | @Override 22 | public DataLayer getDataLayer(CaseInsensitiveStringMap options) 23 | { 24 | return new SpyWaterMarkerDataLayer(CassandraVersion.valueOf(options.getOrDefault(lowerCaseKey("version"), CassandraVersion.THREEZERO.toString())), 25 | getOrThrow(options, lowerCaseKey("keyspace")), 26 | getOrThrow(options, lowerCaseKey("createStmt")), 27 | getOrThrow(options, lowerCaseKey("dirs")).split(",")); 28 | } 29 | 30 | @Override 31 | public String shortName() 32 | { 33 | return "SpyWaterMarkerDataLayer"; 34 | } 35 | 36 | public static class SpyWaterMarkerDataLayer extends LocalDataLayer 37 | { 38 | public static final InMemoryWatermarker inMemoryWatermarker = spy(SparkInMemoryWatermarker.INSTANCE); 39 | 40 | public SpyWaterMarkerDataLayer(@NotNull CassandraVersion version, @NotNull String keyspace, @NotNull String createStmt, String... paths) 41 | { 42 | super(version, keyspace, createStmt, paths); 43 | } 44 | 45 | @Override 46 | public Watermarker cdcWatermarker() 47 | { 48 | return inMemoryWatermarker; 49 | } 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /cdc/src/main/java/org/apache/cassandra/spark/stats/CdcStats.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.stats; 23 | 24 | import org.apache.cassandra.spark.cdc.CommitLog; 25 | import org.apache.cassandra.spark.utils.streaming.Source; 26 | 27 | public class CdcStats implements IStats, ICdcStats 28 | { 29 | public static class DoNothingCdcStats extends CdcStats 30 | { 31 | public static final DoNothingCdcStats INSTANCE = new DoNothingCdcStats(); 32 | } 33 | 34 | // SSTableInputStream 35 | 36 | public void inputStreamEnd(Source source, long runTimeNanos, long totalNanosBlocked) 37 | { 38 | 39 | } 40 | 41 | public void inputStreamEndBuffer(Source ssTable) 42 | { 43 | 44 | } 45 | 46 | public void inputStreamTimeBlocked(Source source, long nanos) 47 | { 48 | 49 | } 50 | 51 | public void inputStreamByteRead(Source source, int len, int queueSize, int percentComplete) 52 | { 53 | 54 | } 55 | 56 | public void inputStreamFailure(Source source, Throwable t) 57 | { 58 | 59 | } 60 | 61 | public void inputStreamBytesWritten(Source source, int len) 62 | { 63 | 64 | } 65 | 66 | public void inputStreamBytesSkipped(Source source, long bufferedSkipped, long rangeSkipped) 67 | { 68 | 69 | } 70 | } 71 | -------------------------------------------------------------------------------- /cdc/src/main/java/org/apache/cassandra/spark/cdc/jdk/msg/Column.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.cdc.jdk.msg; 23 | 24 | import java.nio.ByteBuffer; 25 | 26 | import org.apache.cassandra.spark.data.CqlField; 27 | import org.jetbrains.annotations.Nullable; 28 | 29 | public class Column 30 | { 31 | private final String name; 32 | private final CqlField.CqlType type; 33 | @Nullable 34 | private final Object value; 35 | 36 | public Column(String name, CqlField.CqlType type, ByteBuffer buf) 37 | { 38 | this(name, type, buf == null ? null : type.deserializeToJava(buf)); 39 | } 40 | 41 | public Column(String name, CqlField.CqlType type, @Nullable Object value) 42 | { 43 | this.name = name; 44 | this.type = type; 45 | this.value = value; 46 | } 47 | 48 | public String name() 49 | { 50 | return name; 51 | } 52 | 53 | public CqlField.CqlType type() 54 | { 55 | return type; 56 | } 57 | 58 | @Nullable 59 | public Object value() 60 | { 61 | return value; 62 | } 63 | 64 | public T getAs(Class tClass) 65 | { 66 | return value == null ? null : tClass.cast(value); 67 | } 68 | 69 | @Override 70 | public String toString() 71 | { 72 | return "\"" + name + "\": " + (value == null ? "null" : "\"" + value + "\""); 73 | } 74 | } 75 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/sparksql/filters/RangeFilter.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.sparksql.filters; 2 | 3 | import java.io.Serializable; 4 | import java.math.BigInteger; 5 | 6 | import com.google.common.base.Preconditions; 7 | import com.google.common.collect.Range; 8 | 9 | import org.jetbrains.annotations.NotNull; 10 | 11 | /* 12 | * 13 | * Licensed to the Apache Software Foundation (ASF) under one 14 | * or more contributor license agreements. See the NOTICE file 15 | * distributed with this work for additional information 16 | * regarding copyright ownership. The ASF licenses this file 17 | * to you under the Apache License, Version 2.0 (the 18 | * "License"); you may not use this file except in compliance 19 | * with the License. You may obtain a copy of the License at 20 | * 21 | * http://www.apache.org/licenses/LICENSE-2.0 22 | * 23 | * Unless required by applicable law or agreed to in writing, 24 | * software distributed under the License is distributed on an 25 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 26 | * KIND, either express or implied. See the License for the 27 | * specific language governing permissions and limitations 28 | * under the License. 29 | * 30 | */ 31 | 32 | public class RangeFilter implements Serializable 33 | { 34 | private final Range tokenRange; 35 | 36 | private RangeFilter(@NotNull final Range tokenRange) 37 | { 38 | this.tokenRange = tokenRange; 39 | } 40 | 41 | public Range tokenRange() 42 | { 43 | return tokenRange; 44 | } 45 | 46 | public boolean overlaps(final Range tokenRange) 47 | { 48 | return this.tokenRange.isConnected(tokenRange); 49 | } 50 | 51 | public boolean overlaps(final BigInteger token) 52 | { 53 | return this.tokenRange.contains(token); 54 | } 55 | 56 | public boolean skipPartition(final BigInteger token) 57 | { 58 | return !this.tokenRange.contains(token); 59 | } 60 | 61 | public static RangeFilter create(final Range tokenRange) 62 | { 63 | Preconditions.checkArgument(tokenRange.hasLowerBound() && tokenRange.hasUpperBound()); 64 | return new RangeFilter(tokenRange); 65 | } 66 | } 67 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/UUID.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType; 4 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.SettableByIndexData; 5 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.AbstractType; 6 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.UUIDType; 7 | 8 | /* 9 | * 10 | * Licensed to the Apache Software Foundation (ASF) under one 11 | * or more contributor license agreements. See the NOTICE file 12 | * distributed with this work for additional information 13 | * regarding copyright ownership. The ASF licenses this file 14 | * to you under the Apache License, Version 2.0 (the 15 | * "License"); you may not use this file except in compliance 16 | * with the License. You may obtain a copy of the License at 17 | * 18 | * http://www.apache.org/licenses/LICENSE-2.0 19 | * 20 | * Unless required by applicable law or agreed to in writing, 21 | * software distributed under the License is distributed on an 22 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 23 | * KIND, either express or implied. See the License for the 24 | * specific language governing permissions and limitations 25 | * under the License. 26 | * 27 | */ 28 | 29 | public class UUID extends StringBased 30 | { 31 | public static final UUID INSTANCE = new UUID(); 32 | 33 | @Override 34 | public String name() 35 | { 36 | return "uuid"; 37 | } 38 | 39 | @Override 40 | public AbstractType dataType() 41 | { 42 | return UUIDType.instance; 43 | } 44 | 45 | @Override 46 | public DataType driverDataType(boolean isFrozen) 47 | { 48 | return org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType.uuid(); 49 | } 50 | 51 | @Override 52 | public void setInnerValue(SettableByIndexData udtValue, int pos, Object value) 53 | { 54 | udtValue.setUUID(pos, (java.util.UUID) value); 55 | } 56 | 57 | @Override 58 | public Object randomValue(int minCollectionSize) 59 | { 60 | return java.util.UUID.randomUUID(); 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/data/fourzero/types/spark/SmallInt.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.data.fourzero.types.spark; 23 | 24 | import java.util.Comparator; 25 | 26 | import org.apache.cassandra.spark.data.SparkCqlField; 27 | import org.apache.cassandra.spark.reader.BigNumberConfig; 28 | import org.apache.spark.sql.Row; 29 | import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; 30 | import org.apache.spark.sql.types.DataType; 31 | import org.apache.spark.sql.types.DataTypes; 32 | 33 | public class SmallInt extends org.apache.cassandra.spark.data.fourzero.types.SmallInt implements SparkCqlField.SparkCqlType 34 | { 35 | private static final Comparator SHORT_COMPARATOR = Short::compare; 36 | public static final SmallInt INSTANCE = new SmallInt(); 37 | 38 | @Override 39 | public DataType sparkSqlType(BigNumberConfig bigNumberConfig) 40 | { 41 | return DataTypes.ShortType; 42 | } 43 | 44 | @Override 45 | public Object nativeSparkSqlRowValue(final GenericInternalRow row, final int pos) 46 | { 47 | return row.getShort(pos); 48 | } 49 | 50 | @Override 51 | public Object nativeSparkSqlRowValue(Row row, int pos) 52 | { 53 | return row.getShort(pos); 54 | } 55 | 56 | @Override 57 | public int compareTo(Object o1, Object o2) 58 | { 59 | return SHORT_COMPARATOR.compare((Short) o1, (Short) o2); 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/data/fourzero/types/spark/Timestamp.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.data.fourzero.types.spark; 23 | 24 | import org.apache.cassandra.spark.data.SparkCqlField; 25 | import org.apache.cassandra.spark.reader.BigNumberConfig; 26 | import org.apache.spark.sql.Row; 27 | import org.apache.spark.sql.types.DataType; 28 | import org.apache.spark.sql.types.DataTypes; 29 | import org.jetbrains.annotations.NotNull; 30 | 31 | public class Timestamp extends org.apache.cassandra.spark.data.fourzero.types.Timestamp implements SparkCqlField.LongTraits 32 | { 33 | public static final Timestamp INSTANCE = new Timestamp(); 34 | 35 | @Override 36 | public DataType sparkSqlType(BigNumberConfig bigNumberConfig) 37 | { 38 | return DataTypes.TimestampType; 39 | } 40 | 41 | @Override 42 | public Object toSparkSqlType(@NotNull Object o, boolean isFrozen) 43 | { 44 | return ((java.util.Date) o).getTime() * 1000L; // long 45 | } 46 | 47 | @Override 48 | public Object nativeSparkSqlRowValue(Row row, int pos) 49 | { 50 | return new java.util.Date(row.getTimestamp(pos).getTime()); 51 | } 52 | 53 | @Override 54 | public Object toTestRowType(Object value) 55 | { 56 | if (value instanceof java.util.Date) 57 | { 58 | return value; 59 | } 60 | return new java.util.Date((long) value / 1000L); 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/data/fourzero/types/spark/Double.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.data.fourzero.types.spark; 23 | 24 | import java.util.Comparator; 25 | 26 | import org.apache.cassandra.spark.data.SparkCqlField; 27 | import org.apache.cassandra.spark.reader.BigNumberConfig; 28 | import org.apache.spark.sql.Row; 29 | import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; 30 | import org.apache.spark.sql.types.DataType; 31 | import org.apache.spark.sql.types.DataTypes; 32 | 33 | public class Double extends org.apache.cassandra.spark.data.fourzero.types.Double implements SparkCqlField.SparkCqlType 34 | { 35 | public static Double INSTANCE = new Double(); 36 | public static final Comparator DOUBLE_COMPARATOR = java.lang.Double::compareTo; 37 | 38 | @Override 39 | public DataType sparkSqlType(BigNumberConfig bigNumberConfig) 40 | { 41 | return DataTypes.DoubleType; 42 | } 43 | 44 | @Override 45 | public Object nativeSparkSqlRowValue(final GenericInternalRow row, final int pos) 46 | { 47 | return row.getDouble(pos); 48 | } 49 | 50 | @Override 51 | public Object nativeSparkSqlRowValue(Row row, int pos) 52 | { 53 | return row.getDouble(pos); 54 | } 55 | 56 | public int compareTo(Object o1, Object o2) 57 | { 58 | return DOUBLE_COMPARATOR.compare((java.lang.Double) o1, (java.lang.Double) o2); 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/data/fourzero/types/spark/Empty.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.data.fourzero.types.spark; 23 | 24 | import java.util.Comparator; 25 | 26 | import org.apache.cassandra.spark.data.SparkCqlField; 27 | import org.apache.cassandra.spark.reader.BigNumberConfig; 28 | import org.apache.spark.sql.Row; 29 | import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; 30 | import org.apache.spark.sql.types.DataType; 31 | import org.apache.spark.sql.types.DataTypes; 32 | 33 | public class Empty extends org.apache.cassandra.spark.data.fourzero.types.Empty implements SparkCqlField.SparkCqlType 34 | { 35 | private static final Comparator VOID_COMPARATOR_COMPARATOR = (o1, o2) -> 0; 36 | public static Empty INSTANCE = new Empty(); 37 | 38 | @Override 39 | public DataType sparkSqlType(BigNumberConfig bigNumberConfig) 40 | { 41 | return DataTypes.NullType; 42 | } 43 | 44 | @Override 45 | public Object nativeSparkSqlRowValue(final GenericInternalRow row, final int pos) 46 | { 47 | return null; 48 | } 49 | 50 | @Override 51 | public Object nativeSparkSqlRowValue(Row row, int pos) 52 | { 53 | return null; 54 | } 55 | 56 | @SuppressWarnings("ConstantConditions") 57 | @Override 58 | public int compareTo(Object o1, Object o2) 59 | { 60 | return VOID_COMPARATOR_COMPARATOR.compare((Void) o1, (Void) o2); 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/Blob.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import java.nio.ByteBuffer; 4 | 5 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType; 6 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.SettableByIndexData; 7 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.AbstractType; 8 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.BytesType; 9 | import org.apache.cassandra.spark.utils.RandomUtils; 10 | 11 | /* 12 | * 13 | * Licensed to the Apache Software Foundation (ASF) under one 14 | * or more contributor license agreements. See the NOTICE file 15 | * distributed with this work for additional information 16 | * regarding copyright ownership. The ASF licenses this file 17 | * to you under the Apache License, Version 2.0 (the 18 | * "License"); you may not use this file except in compliance 19 | * with the License. You may obtain a copy of the License at 20 | * 21 | * http://www.apache.org/licenses/LICENSE-2.0 22 | * 23 | * Unless required by applicable law or agreed to in writing, 24 | * software distributed under the License is distributed on an 25 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 26 | * KIND, either express or implied. See the License for the 27 | * specific language governing permissions and limitations 28 | * under the License. 29 | * 30 | */ 31 | 32 | public class Blob extends BinaryBased 33 | { 34 | public static final Blob INSTANCE = new Blob(); 35 | 36 | @Override 37 | public String name() 38 | { 39 | return "blob"; 40 | } 41 | 42 | @Override 43 | public AbstractType dataType() 44 | { 45 | return BytesType.instance; 46 | } 47 | 48 | @Override 49 | public Object randomValue(int minCollectionSize) 50 | { 51 | return RandomUtils.randomByteBuffer(RandomUtils.randomPositiveInt(256)); 52 | } 53 | 54 | @Override 55 | public void setInnerValue(SettableByIndexData udtValue, int pos, Object value) 56 | { 57 | udtValue.setBytes(pos, (ByteBuffer) value); 58 | } 59 | 60 | @Override 61 | public DataType driverDataType(boolean isFrozen) 62 | { 63 | return DataType.blob(); 64 | } 65 | } 66 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/data/fourzero/types/spark/Float.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.data.fourzero.types.spark; 23 | 24 | import java.util.Comparator; 25 | 26 | import org.apache.cassandra.spark.data.SparkCqlField; 27 | import org.apache.cassandra.spark.reader.BigNumberConfig; 28 | import org.apache.spark.sql.Row; 29 | import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; 30 | import org.apache.spark.sql.types.DataType; 31 | import org.apache.spark.sql.types.DataTypes; 32 | 33 | public class Float extends org.apache.cassandra.spark.data.fourzero.types.Float implements SparkCqlField.SparkCqlType 34 | { 35 | public static final Comparator FLOAT_COMPARATOR = java.lang.Float::compareTo; 36 | public static final Float INSTANCE = new Float(); 37 | 38 | @Override 39 | public DataType sparkSqlType(BigNumberConfig bigNumberConfig) 40 | { 41 | return DataTypes.FloatType; 42 | } 43 | 44 | @Override 45 | public Object nativeSparkSqlRowValue(final GenericInternalRow row, final int pos) 46 | { 47 | return row.getFloat(pos); 48 | } 49 | 50 | @Override 51 | public Object nativeSparkSqlRowValue(Row row, int pos) 52 | { 53 | return row.getFloat(pos); 54 | } 55 | 56 | @Override 57 | public int compareTo(Object o1, Object o2) 58 | { 59 | return FLOAT_COMPARATOR.compare((java.lang.Float) o1, (java.lang.Float) o2); 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/reader/IndexEntry.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.reader; 23 | 24 | import java.math.BigInteger; 25 | import java.nio.ByteBuffer; 26 | 27 | import org.apache.cassandra.spark.utils.ByteBufUtils; 28 | 29 | public class IndexEntry 30 | { 31 | public final ByteBuffer partitionKey; 32 | public final BigInteger token; 33 | public final long uncompressed; 34 | public final long compressed; 35 | 36 | public IndexEntry(ByteBuffer partitionKey, 37 | BigInteger token, 38 | long uncompressed, 39 | long compressed) 40 | { 41 | this.partitionKey = partitionKey; 42 | this.token = token; 43 | this.uncompressed = uncompressed; 44 | this.compressed = compressed; 45 | } 46 | 47 | public ByteBuffer getPartitionKey() 48 | { 49 | return this.partitionKey; 50 | } 51 | 52 | public BigInteger getToken() 53 | { 54 | return this.token; 55 | } 56 | 57 | public long getCompressed() 58 | { 59 | return compressed; 60 | } 61 | 62 | public long getUncompressed() 63 | { 64 | return uncompressed; 65 | } 66 | 67 | @Override 68 | public String toString() 69 | { 70 | return ByteBufUtils.toHexString(this.partitionKey) + 71 | ": " + uncompressed + " uncompressed bytes, " + compressed + " compressed bytes"; 72 | } 73 | } 74 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/Timestamp.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import org.apache.cassandra.spark.data.fourzero.NativeType; 4 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.SettableByIndexData; 5 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.AbstractType; 6 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.TimestampType; 7 | 8 | /* 9 | * 10 | * Licensed to the Apache Software Foundation (ASF) under one 11 | * or more contributor license agreements. See the NOTICE file 12 | * distributed with this work for additional information 13 | * regarding copyright ownership. The ASF licenses this file 14 | * to you under the Apache License, Version 2.0 (the 15 | * "License"); you may not use this file except in compliance 16 | * with the License. You may obtain a copy of the License at 17 | * 18 | * http://www.apache.org/licenses/LICENSE-2.0 19 | * 20 | * Unless required by applicable law or agreed to in writing, 21 | * software distributed under the License is distributed on an 22 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 23 | * KIND, either express or implied. See the License for the 24 | * specific language governing permissions and limitations 25 | * under the License. 26 | * 27 | */ 28 | 29 | public class Timestamp extends NativeType 30 | { 31 | public static final Timestamp INSTANCE = new Timestamp(); 32 | 33 | @Override 34 | public String name() 35 | { 36 | return "timestamp"; 37 | } 38 | 39 | @Override 40 | public AbstractType dataType() 41 | { 42 | return TimestampType.instance; 43 | } 44 | 45 | @Override 46 | public void setInnerValue(SettableByIndexData udtValue, int pos, Object value) 47 | { 48 | udtValue.setTimestamp(pos, (java.util.Date) value); 49 | } 50 | 51 | @Override 52 | public Object randomValue(int minCollectionSize) 53 | { 54 | return new java.util.Date(); 55 | } 56 | 57 | @Override 58 | public org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen) 59 | { 60 | return org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType.timestamp(); 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/Int.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import org.apache.cassandra.spark.data.fourzero.NativeType; 4 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.SettableByIndexData; 5 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.AbstractType; 6 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.Int32Type; 7 | import org.apache.cassandra.spark.utils.RandomUtils; 8 | 9 | /* 10 | * 11 | * Licensed to the Apache Software Foundation (ASF) under one 12 | * or more contributor license agreements. See the NOTICE file 13 | * distributed with this work for additional information 14 | * regarding copyright ownership. The ASF licenses this file 15 | * to you under the Apache License, Version 2.0 (the 16 | * "License"); you may not use this file except in compliance 17 | * with the License. You may obtain a copy of the License at 18 | * 19 | * http://www.apache.org/licenses/LICENSE-2.0 20 | * 21 | * Unless required by applicable law or agreed to in writing, 22 | * software distributed under the License is distributed on an 23 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 24 | * KIND, either express or implied. See the License for the 25 | * specific language governing permissions and limitations 26 | * under the License. 27 | * 28 | */ 29 | 30 | public class Int extends NativeType 31 | { 32 | public static final Int INSTANCE = new Int(); 33 | 34 | @Override 35 | public String name() 36 | { 37 | return "int"; 38 | } 39 | 40 | @Override 41 | public AbstractType dataType() 42 | { 43 | return Int32Type.instance; 44 | } 45 | 46 | @Override 47 | public Object randomValue(int minCollectionSize) 48 | { 49 | return RandomUtils.RANDOM.nextInt(); 50 | } 51 | 52 | @Override 53 | public void setInnerValue(SettableByIndexData udtValue, int pos, Object value) 54 | { 55 | udtValue.setInt(pos, (int) value); 56 | } 57 | 58 | @Override 59 | public org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen) 60 | { 61 | return org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType.cint(); 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/data/fourzero/types/spark/Boolean.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.data.fourzero.types.spark; 23 | 24 | import java.util.Comparator; 25 | 26 | import org.apache.cassandra.spark.data.SparkCqlField; 27 | import org.apache.cassandra.spark.reader.BigNumberConfig; 28 | import org.apache.spark.sql.Row; 29 | import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; 30 | import org.apache.spark.sql.types.DataType; 31 | import org.apache.spark.sql.types.DataTypes; 32 | 33 | public class Boolean extends org.apache.cassandra.spark.data.fourzero.types.Boolean implements SparkCqlField.SparkCqlType 34 | { 35 | public static final Comparator BOOLEAN_COMPARATOR = java.lang.Boolean::compareTo; 36 | public static Boolean INSTANCE = new Boolean(); 37 | 38 | @Override 39 | public DataType sparkSqlType(BigNumberConfig bigNumberConfig) 40 | { 41 | return DataTypes.BooleanType; 42 | } 43 | 44 | @Override 45 | public Object nativeSparkSqlRowValue(final GenericInternalRow row, final int pos) 46 | { 47 | return row.getBoolean(pos); 48 | } 49 | 50 | @Override 51 | public Object nativeSparkSqlRowValue(Row row, int pos) 52 | { 53 | return row.getBoolean(pos); 54 | } 55 | 56 | @Override 57 | public int compareTo(Object o1, Object o2) 58 | { 59 | return BOOLEAN_COMPARATOR.compare((java.lang.Boolean) o1, (java.lang.Boolean) o2); 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /cdc/src/main/java/org/apache/cassandra/spark/cdc/jdk/msg/RangeTombstone.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.cdc.jdk.msg; 23 | 24 | import java.util.List; 25 | import java.util.stream.Collectors; 26 | 27 | import org.apache.cassandra.spark.cdc.jdk.JdkRangeTombstone; 28 | import org.apache.cassandra.spark.cdc.jdk.JdkValueMetadata; 29 | 30 | public class RangeTombstone 31 | { 32 | private final List startBound, endBound; 33 | public final boolean startInclusive, endInclusive; 34 | 35 | public RangeTombstone(JdkRangeTombstone tombstone) 36 | { 37 | this.startBound = tombstone.getStartBound().stream() 38 | .map(JdkValueMetadata::toRow) 39 | .collect(Collectors.toList()); 40 | this.endBound = tombstone.getEndBound().stream() 41 | .map(JdkValueMetadata::toRow) 42 | .collect(Collectors.toList()); 43 | this.startInclusive = tombstone.startInclusive; 44 | this.endInclusive = tombstone.endInclusive; 45 | } 46 | 47 | public List startBound() 48 | { 49 | return startBound; 50 | } 51 | 52 | public List endBound() 53 | { 54 | return endBound; 55 | } 56 | 57 | public boolean isStartInclusive() 58 | { 59 | return startInclusive; 60 | } 61 | 62 | public boolean isEndInclusive() 63 | { 64 | return endInclusive; 65 | } 66 | } 67 | -------------------------------------------------------------------------------- /common/src/test/java/org/apache/cassandra/spark/reader/fourzero/CompressionUtilTests.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.reader.fourzero; 2 | 3 | import java.io.IOException; 4 | import java.nio.ByteBuffer; 5 | import java.util.Arrays; 6 | 7 | import org.junit.Test; 8 | 9 | import org.apache.cassandra.spark.utils.RandomUtils; 10 | 11 | import static org.junit.Assert.assertArrayEquals; 12 | import static org.junit.Assert.assertEquals; 13 | 14 | /* 15 | * 16 | * Licensed to the Apache Software Foundation (ASF) under one 17 | * or more contributor license agreements. See the NOTICE file 18 | * distributed with this work for additional information 19 | * regarding copyright ownership. The ASF licenses this file 20 | * to you under the Apache License, Version 2.0 (the 21 | * "License"); you may not use this file except in compliance 22 | * with the License. You may obtain a copy of the License at 23 | * 24 | * http://www.apache.org/licenses/LICENSE-2.0 25 | * 26 | * Unless required by applicable law or agreed to in writing, 27 | * software distributed under the License is distributed on an 28 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 29 | * KIND, either express or implied. See the License for the 30 | * specific language governing permissions and limitations 31 | * under the License. 32 | * 33 | */ 34 | 35 | public class CompressionUtilTests 36 | { 37 | @Test 38 | public void testCompressRandom() throws IOException 39 | { 40 | // test with random data - not highly compressible 41 | testCompression(RandomUtils.randomBytes(4096)); 42 | } 43 | 44 | @Test 45 | public void testCompressUniform() throws IOException 46 | { 47 | // test with highly compressible data 48 | final byte[] ar = new byte[4096]; 49 | Arrays.fill(ar, (byte) 'a'); 50 | testCompression(ar); 51 | } 52 | 53 | private void testCompression(final byte[] ar) throws IOException 54 | { 55 | final CompressionUtil compress = new CompressionUtil(); 56 | final ByteBuffer compressed = compress.compress(ar); 57 | final ByteBuffer uncompressed = compress.uncompress(compressed); 58 | final byte[] result = new byte[uncompressed.remaining()]; 59 | uncompressed.get(result); 60 | assertEquals(ar.length, result.length); 61 | assertArrayEquals(ar, result); 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/Float.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import org.apache.cassandra.spark.data.fourzero.NativeType; 4 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.SettableByIndexData; 5 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.AbstractType; 6 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.FloatType; 7 | import org.apache.cassandra.spark.utils.RandomUtils; 8 | 9 | /* 10 | * 11 | * Licensed to the Apache Software Foundation (ASF) under one 12 | * or more contributor license agreements. See the NOTICE file 13 | * distributed with this work for additional information 14 | * regarding copyright ownership. The ASF licenses this file 15 | * to you under the Apache License, Version 2.0 (the 16 | * "License"); you may not use this file except in compliance 17 | * with the License. You may obtain a copy of the License at 18 | * 19 | * http://www.apache.org/licenses/LICENSE-2.0 20 | * 21 | * Unless required by applicable law or agreed to in writing, 22 | * software distributed under the License is distributed on an 23 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 24 | * KIND, either express or implied. See the License for the 25 | * specific language governing permissions and limitations 26 | * under the License. 27 | * 28 | */ 29 | 30 | public class Float extends NativeType 31 | { 32 | public static final Float INSTANCE = new Float(); 33 | 34 | @Override 35 | public String name() 36 | { 37 | return "float"; 38 | } 39 | 40 | @Override 41 | public AbstractType dataType() 42 | { 43 | return FloatType.instance; 44 | } 45 | 46 | @Override 47 | public Object randomValue(int minCollectionSize) 48 | { 49 | return RandomUtils.RANDOM.nextFloat(); 50 | } 51 | 52 | @Override 53 | public void setInnerValue(SettableByIndexData udtValue, int pos, Object value) 54 | { 55 | udtValue.setFloat(pos, (float) value); 56 | } 57 | 58 | @Override 59 | public org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen) 60 | { 61 | return org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType.cfloat(); 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/TinyInt.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import org.apache.cassandra.spark.data.fourzero.NativeType; 4 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.SettableByIndexData; 5 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.AbstractType; 6 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.ByteType; 7 | import org.apache.cassandra.spark.utils.RandomUtils; 8 | 9 | /* 10 | * 11 | * Licensed to the Apache Software Foundation (ASF) under one 12 | * or more contributor license agreements. See the NOTICE file 13 | * distributed with this work for additional information 14 | * regarding copyright ownership. The ASF licenses this file 15 | * to you under the Apache License, Version 2.0 (the 16 | * "License"); you may not use this file except in compliance 17 | * with the License. You may obtain a copy of the License at 18 | * 19 | * http://www.apache.org/licenses/LICENSE-2.0 20 | * 21 | * Unless required by applicable law or agreed to in writing, 22 | * software distributed under the License is distributed on an 23 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 24 | * KIND, either express or implied. See the License for the 25 | * specific language governing permissions and limitations 26 | * under the License. 27 | * 28 | */ 29 | 30 | public class TinyInt extends NativeType 31 | { 32 | public static final TinyInt INSTANCE = new TinyInt(); 33 | 34 | @Override 35 | public String name() 36 | { 37 | return "tinyint"; 38 | } 39 | 40 | @Override 41 | public AbstractType dataType() 42 | { 43 | return ByteType.instance; 44 | } 45 | 46 | @Override 47 | public Object randomValue(int minCollectionSize) 48 | { 49 | return RandomUtils.randomBytes(1)[0]; 50 | } 51 | 52 | @Override 53 | public void setInnerValue(SettableByIndexData udtValue, int pos, Object value) 54 | { 55 | udtValue.setByte(pos, (byte) value); 56 | } 57 | 58 | @Override 59 | public org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen) 60 | { 61 | return org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType.tinyint(); 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/Double.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import org.apache.cassandra.spark.data.fourzero.NativeType; 4 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.SettableByIndexData; 5 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.AbstractType; 6 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.DoubleType; 7 | import org.apache.cassandra.spark.utils.RandomUtils; 8 | 9 | /* 10 | * 11 | * Licensed to the Apache Software Foundation (ASF) under one 12 | * or more contributor license agreements. See the NOTICE file 13 | * distributed with this work for additional information 14 | * regarding copyright ownership. The ASF licenses this file 15 | * to you under the Apache License, Version 2.0 (the 16 | * "License"); you may not use this file except in compliance 17 | * with the License. You may obtain a copy of the License at 18 | * 19 | * http://www.apache.org/licenses/LICENSE-2.0 20 | * 21 | * Unless required by applicable law or agreed to in writing, 22 | * software distributed under the License is distributed on an 23 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 24 | * KIND, either express or implied. See the License for the 25 | * specific language governing permissions and limitations 26 | * under the License. 27 | * 28 | */ 29 | 30 | public class Double extends NativeType 31 | { 32 | public static final Double INSTANCE = new Double(); 33 | 34 | @Override 35 | public String name() 36 | { 37 | return "double"; 38 | } 39 | 40 | @Override 41 | public AbstractType dataType() 42 | { 43 | return DoubleType.instance; 44 | } 45 | 46 | @Override 47 | public Object randomValue(int minCollectionSize) 48 | { 49 | return RandomUtils.RANDOM.nextDouble(); 50 | } 51 | 52 | @Override 53 | public void setInnerValue(SettableByIndexData udtValue, int pos, Object value) 54 | { 55 | udtValue.setDouble(pos, (double) value); 56 | } 57 | 58 | @Override 59 | public org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen) 60 | { 61 | return org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType.cdouble(); 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/SmallInt.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import org.apache.cassandra.spark.data.fourzero.NativeType; 4 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.SettableByIndexData; 5 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.AbstractType; 6 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.ShortType; 7 | import org.apache.cassandra.spark.utils.RandomUtils; 8 | 9 | /* 10 | * 11 | * Licensed to the Apache Software Foundation (ASF) under one 12 | * or more contributor license agreements. See the NOTICE file 13 | * distributed with this work for additional information 14 | * regarding copyright ownership. The ASF licenses this file 15 | * to you under the Apache License, Version 2.0 (the 16 | * "License"); you may not use this file except in compliance 17 | * with the License. You may obtain a copy of the License at 18 | * 19 | * http://www.apache.org/licenses/LICENSE-2.0 20 | * 21 | * Unless required by applicable law or agreed to in writing, 22 | * software distributed under the License is distributed on an 23 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 24 | * KIND, either express or implied. See the License for the 25 | * specific language governing permissions and limitations 26 | * under the License. 27 | * 28 | */ 29 | 30 | public class SmallInt extends NativeType 31 | { 32 | public static final SmallInt INSTANCE = new SmallInt(); 33 | 34 | @Override 35 | public String name() 36 | { 37 | return "smallint"; 38 | } 39 | 40 | @Override 41 | public AbstractType dataType() 42 | { 43 | return ShortType.instance; 44 | } 45 | 46 | @Override 47 | public Object randomValue(int minCollectionSize) 48 | { 49 | return (short) RandomUtils.RANDOM.nextInt(Short.MAX_VALUE + 1); 50 | } 51 | 52 | @Override 53 | public void setInnerValue(SettableByIndexData udtValue, int pos, Object value) 54 | { 55 | udtValue.setShort(pos, (short) value); 56 | } 57 | 58 | @Override 59 | public org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen) 60 | { 61 | return org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType.smallint(); 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/utils/ThrowableUtils.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.utils; 2 | 3 | import org.jetbrains.annotations.NotNull; 4 | import org.jetbrains.annotations.Nullable; 5 | 6 | /* 7 | * 8 | * Licensed to the Apache Software Foundation (ASF) under one 9 | * or more contributor license agreements. See the NOTICE file 10 | * distributed with this work for additional information 11 | * regarding copyright ownership. The ASF licenses this file 12 | * to you under the Apache License, Version 2.0 (the 13 | * "License"); you may not use this file except in compliance 14 | * with the License. You may obtain a copy of the License at 15 | * 16 | * http://www.apache.org/licenses/LICENSE-2.0 17 | * 18 | * Unless required by applicable law or agreed to in writing, 19 | * software distributed under the License is distributed on an 20 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 21 | * KIND, either express or implied. See the License for the 22 | * specific language governing permissions and limitations 23 | * under the License. 24 | * 25 | */ 26 | 27 | public class ThrowableUtils 28 | { 29 | /** 30 | * Find root cause of throwable or this throwable if no prior cause. 31 | * 32 | * @param t throwable 33 | * @return initial cause throwable. 34 | */ 35 | @NotNull 36 | public static Throwable rootCause(@NotNull Throwable t) 37 | { 38 | while (t.getCause() != null) 39 | { 40 | t = t.getCause(); 41 | } 42 | return t; 43 | } 44 | 45 | /** 46 | * Find first throwable of type matching ofType parameter or null if not exists. 47 | * 48 | * @param t throwable 49 | * @param ofType type of class expected 50 | * @param generic type of expected return value 51 | * @return first throwable of type matching parameter ofType or null if cannot be found. 52 | */ 53 | @Nullable 54 | public static T rootCause(@NotNull Throwable t, 55 | @NotNull final Class ofType) 56 | { 57 | while (t.getCause() != null) 58 | { 59 | if (ofType.isInstance(t)) 60 | { 61 | return ofType.cast(t); 62 | } 63 | t = t.getCause(); 64 | } 65 | 66 | return ofType.isInstance(t) ? ofType.cast(t) : null; 67 | } 68 | } 69 | -------------------------------------------------------------------------------- /cdc/src/main/java/org/apache/cassandra/spark/cdc/watermarker/DoNothingWatermarker.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.cdc.watermarker; 2 | 3 | import org.apache.cassandra.spark.cdc.Marker; 4 | import org.apache.cassandra.spark.data.partitioner.CassandraInstance; 5 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.commitlog.PartitionUpdateWrapper; 6 | import org.jetbrains.annotations.Nullable; 7 | 8 | /* 9 | * 10 | * Licensed to the Apache Software Foundation (ASF) under one 11 | * or more contributor license agreements. See the NOTICE file 12 | * distributed with this work for additional information 13 | * regarding copyright ownership. The ASF licenses this file 14 | * to you under the Apache License, Version 2.0 (the 15 | * "License"); you may not use this file except in compliance 16 | * with the License. You may obtain a copy of the License at 17 | * 18 | * http://www.apache.org/licenses/LICENSE-2.0 19 | * 20 | * Unless required by applicable law or agreed to in writing, 21 | * software distributed under the License is distributed on an 22 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 23 | * KIND, either express or implied. See the License for the 24 | * specific language governing permissions and limitations 25 | * under the License. 26 | * 27 | */ 28 | 29 | /** 30 | * Watermarker that does nothing. 31 | */ 32 | public class DoNothingWatermarker implements Watermarker 33 | { 34 | public static final DoNothingWatermarker INSTANCE = new DoNothingWatermarker(); 35 | 36 | public Watermarker instance(String jobId) 37 | { 38 | return this; 39 | } 40 | 41 | public void recordReplicaCount(PartitionUpdateWrapper update, int numReplicas) 42 | { 43 | 44 | } 45 | 46 | public int replicaCount(PartitionUpdateWrapper update) 47 | { 48 | return 0; 49 | } 50 | 51 | public void untrackReplicaCount(PartitionUpdateWrapper update) 52 | { 53 | 54 | } 55 | 56 | public boolean seenBefore(PartitionUpdateWrapper update) 57 | { 58 | return false; 59 | } 60 | 61 | public void updateHighWaterMark(Marker marker) 62 | { 63 | 64 | } 65 | 66 | @Nullable 67 | public Marker highWaterMark(CassandraInstance instance) 68 | { 69 | return null; 70 | } 71 | 72 | public void persist(@Nullable Long maxAgeMicros) 73 | { 74 | 75 | } 76 | 77 | public void clear() 78 | { 79 | 80 | } 81 | } 82 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/reader/fourzero/BaseFourZeroUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.reader.fourzero; 23 | 24 | import java.math.BigInteger; 25 | 26 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.dht.Murmur3Partitioner; 27 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.dht.RandomPartitioner; 28 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.dht.Token; 29 | 30 | public class BaseFourZeroUtils 31 | { 32 | public static BigInteger tokenToBigInteger(final Token token) 33 | { 34 | if (token instanceof Murmur3Partitioner.LongToken) 35 | { 36 | return BigInteger.valueOf((long) token.getTokenValue()); 37 | } 38 | if (token instanceof RandomPartitioner.BigIntegerToken) 39 | { 40 | return ((RandomPartitioner.BigIntegerToken) token).getTokenValue(); 41 | } 42 | 43 | throw new UnsupportedOperationException("Unexpected token type: " + token.getClass().getName()); 44 | } 45 | 46 | public static long tokenToLong(final Token token) 47 | { 48 | if (token instanceof Murmur3Partitioner.LongToken) 49 | { 50 | return (long) token.getTokenValue(); 51 | } 52 | if (token instanceof RandomPartitioner.BigIntegerToken) 53 | { 54 | return ((RandomPartitioner.BigIntegerToken) token).getTokenValue().longValue(); 55 | } 56 | 57 | throw new UnsupportedOperationException("Unexpected token type: " + token.getClass().getName()); 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/data/fourzero/types/spark/TinyInt.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.data.fourzero.types.spark; 23 | 24 | import java.util.Comparator; 25 | 26 | import org.apache.cassandra.spark.data.SparkCqlField; 27 | import org.apache.cassandra.spark.reader.BigNumberConfig; 28 | import org.apache.spark.sql.Row; 29 | import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; 30 | import org.apache.spark.sql.types.DataType; 31 | import org.apache.spark.sql.types.DataTypes; 32 | 33 | public class TinyInt extends org.apache.cassandra.spark.data.fourzero.types.TinyInt implements SparkCqlField.SparkCqlType 34 | { 35 | public static TinyInt INSTANCE = new TinyInt(); 36 | private static final Comparator BYTE_COMPARATOR = TinyInt::compareBytes; 37 | 38 | @Override 39 | public DataType sparkSqlType(BigNumberConfig bigNumberConfig) 40 | { 41 | return DataTypes.ByteType; 42 | } 43 | 44 | @Override 45 | public Object nativeSparkSqlRowValue(final GenericInternalRow row, final int pos) 46 | { 47 | return row.getByte(pos); 48 | } 49 | 50 | @Override 51 | public Object nativeSparkSqlRowValue(Row row, int pos) 52 | { 53 | return row.getByte(pos); 54 | } 55 | 56 | @Override 57 | public int compareTo(Object o1, Object o2) 58 | { 59 | return BYTE_COMPARATOR.compare((Byte) o1, (Byte) o2); 60 | } 61 | 62 | private static int compareBytes(final byte a, final byte b) 63 | { 64 | return a - b; // safe due to restricted range 65 | } 66 | } 67 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/partitioner/NotEnoughReplicasException.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.partitioner; 2 | 3 | import java.math.BigInteger; 4 | 5 | import com.google.common.collect.Range; 6 | 7 | import org.jetbrains.annotations.NotNull; 8 | import org.jetbrains.annotations.Nullable; 9 | 10 | /* 11 | * 12 | * Licensed to the Apache Software Foundation (ASF) under one 13 | * or more contributor license agreements. See the NOTICE file 14 | * distributed with this work for additional information 15 | * regarding copyright ownership. The ASF licenses this file 16 | * to you under the Apache License, Version 2.0 (the 17 | * "License"); you may not use this file except in compliance 18 | * with the License. You may obtain a copy of the License at 19 | * 20 | * http://www.apache.org/licenses/LICENSE-2.0 21 | * 22 | * Unless required by applicable law or agreed to in writing, 23 | * software distributed under the License is distributed on an 24 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 25 | * KIND, either express or implied. See the License for the 26 | * specific language governing permissions and limitations 27 | * under the License. 28 | * 29 | */ 30 | 31 | 32 | public class NotEnoughReplicasException extends RuntimeException 33 | { 34 | public NotEnoughReplicasException(final String msg) 35 | { 36 | super(msg); 37 | } 38 | 39 | public NotEnoughReplicasException(@NotNull final ConsistencyLevel consistencyLevel, 40 | @NotNull final Range range, 41 | final int minRequired, 42 | final int numInstances, 43 | @Nullable final String dc) 44 | { 45 | super(String.format("Insufficient replicas found to achieve consistency level %s for token range %s - %s, required %d but only %d found, dc=%s", consistencyLevel.name(), range.lowerEndpoint(), range.upperEndpoint(), minRequired, numInstances, dc)); 46 | } 47 | 48 | static boolean isNotEnoughReplicasException(@Nullable final Throwable throwable) 49 | { 50 | Throwable t = throwable; 51 | while (t != null) 52 | { 53 | if (t instanceof NotEnoughReplicasException) 54 | { 55 | return true; 56 | } 57 | t = t.getCause(); 58 | } 59 | return false; 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/cdc/AbstractCdcEventWriter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.cdc; 23 | 24 | import org.slf4j.Logger; 25 | import org.slf4j.LoggerFactory; 26 | 27 | import org.apache.spark.sql.ForeachWriter; 28 | import org.apache.spark.sql.Row; 29 | 30 | public abstract class AbstractCdcEventWriter extends ForeachWriter 31 | { 32 | private transient volatile Logger logger = LoggerFactory.getLogger(getClass()); 33 | 34 | /** 35 | * Called to process the cdc events in the executor side. 36 | * Just like {@link ForeachWriter#process(Object)}, this method will be called only if open returns true. 37 | */ 38 | public abstract void processEvent(SparkCdcEvent event); 39 | 40 | @Override 41 | public boolean open(long partitionId, long epochId) 42 | { 43 | return true; 44 | } 45 | 46 | @Override 47 | public void close(Throwable errorOrNull) 48 | { 49 | if (errorOrNull != null) 50 | { 51 | logger().error("Closing CDC event writer on failure", errorOrNull); 52 | } 53 | else 54 | { 55 | logger().info("Closing CDC event writer"); 56 | } 57 | } 58 | 59 | protected Logger logger() 60 | { 61 | if (logger == null) 62 | { 63 | synchronized (this) 64 | { 65 | if (logger == null) 66 | { 67 | logger = LoggerFactory.getLogger(getClass()); 68 | } 69 | } 70 | } 71 | return logger; 72 | } 73 | } 74 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/VarInt.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import java.math.BigInteger; 4 | 5 | import org.apache.cassandra.spark.reader.BigNumberConfig; 6 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.SettableByIndexData; 7 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.AbstractType; 8 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.IntegerType; 9 | import org.apache.cassandra.spark.utils.RandomUtils; 10 | 11 | /* 12 | * 13 | * Licensed to the Apache Software Foundation (ASF) under one 14 | * or more contributor license agreements. See the NOTICE file 15 | * distributed with this work for additional information 16 | * regarding copyright ownership. The ASF licenses this file 17 | * to you under the Apache License, Version 2.0 (the 18 | * "License"); you may not use this file except in compliance 19 | * with the License. You may obtain a copy of the License at 20 | * 21 | * http://www.apache.org/licenses/LICENSE-2.0 22 | * 23 | * Unless required by applicable law or agreed to in writing, 24 | * software distributed under the License is distributed on an 25 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 26 | * KIND, either express or implied. See the License for the 27 | * specific language governing permissions and limitations 28 | * under the License. 29 | * 30 | */ 31 | 32 | public class VarInt extends Decimal 33 | { 34 | public static final VarInt INSTANCE = new VarInt(); 35 | 36 | @Override 37 | public String name() 38 | { 39 | return "varint"; 40 | } 41 | 42 | @Override 43 | public AbstractType dataType() 44 | { 45 | return IntegerType.instance; 46 | } 47 | 48 | @Override 49 | public Object randomValue(int minCollectionSize) 50 | { 51 | return new BigInteger(BigNumberConfig.DEFAULT.bigIntegerPrecision(), RandomUtils.RANDOM); 52 | } 53 | 54 | @Override 55 | public void setInnerValue(SettableByIndexData udtValue, int pos, Object value) 56 | { 57 | udtValue.setVarint(pos, (BigInteger) value); 58 | } 59 | 60 | @Override 61 | public org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen) 62 | { 63 | return org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType.varint(); 64 | } 65 | } 66 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/utils/streaming/StreamBuffer.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.utils.streaming; 2 | 3 | /* 4 | * 5 | * Licensed to the Apache Software Foundation (ASF) under one 6 | * or more contributor license agreements. See the NOTICE file 7 | * distributed with this work for additional information 8 | * regarding copyright ownership. The ASF licenses this file 9 | * to you under the Apache License, Version 2.0 (the 10 | * "License"); you may not use this file except in compliance 11 | * with the License. You may obtain a copy of the License at 12 | * 13 | * http://www.apache.org/licenses/LICENSE-2.0 14 | * 15 | * Unless required by applicable law or agreed to in writing, 16 | * software distributed under the License is distributed on an 17 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 18 | * KIND, either express or implied. See the License for the 19 | * specific language governing permissions and limitations 20 | * under the License. 21 | * 22 | */ 23 | 24 | import java.nio.ByteBuffer; 25 | 26 | /** 27 | * A generic wrapper around bytes to allow for on/off-heap byte arrays, 28 | * whichever the underlying {@link Source} implementation uses. 29 | */ 30 | public interface StreamBuffer 31 | { 32 | void getBytes(int index, ByteBuffer dst, int len); 33 | 34 | void getBytes(int index, byte[] dst, int dstIndex, int length); 35 | 36 | byte getByte(int index); 37 | 38 | int readableBytes(); 39 | 40 | void release(); 41 | 42 | static ByteArrayWrapper wrap(byte[] ar) 43 | { 44 | return new ByteArrayWrapper(ar); 45 | } 46 | 47 | class ByteArrayWrapper implements StreamBuffer 48 | { 49 | private final byte[] ar; 50 | 51 | private ByteArrayWrapper(final byte[] ar) 52 | { 53 | this.ar = ar; 54 | } 55 | 56 | public void getBytes(int index, ByteBuffer dst, int len) 57 | { 58 | dst.put(ar, index, len); 59 | } 60 | 61 | public void getBytes(int index, byte[] dst, int dstIndex, int length) 62 | { 63 | System.arraycopy(ar, index, dst, dstIndex, length); 64 | } 65 | 66 | public byte getByte(int index) 67 | { 68 | return ar[index]; 69 | } 70 | 71 | public int readableBytes() 72 | { 73 | return ar.length; 74 | } 75 | 76 | public void release() 77 | { 78 | 79 | } 80 | } 81 | } 82 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/cdc/fourzero/SparkRangeTombstoneBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.cdc.fourzero; 23 | 24 | import java.nio.ByteBuffer; 25 | import java.util.List; 26 | 27 | import org.apache.cassandra.spark.cdc.RangeTombstoneBuilder; 28 | import org.apache.cassandra.spark.cdc.SparkRangeTombstone; 29 | import org.apache.cassandra.spark.cdc.SparkValueWithMetadata; 30 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.schema.TableMetadata; 31 | 32 | /** 33 | * Keep track of the last range tombstone marker to build {@link SparkRangeTombstone} 34 | * The caller should check whether {@link #canBuild()} after adding marker, and it should build whenever possible. 35 | */ 36 | public class SparkRangeTombstoneBuilder extends RangeTombstoneBuilder 37 | { 38 | public SparkRangeTombstoneBuilder(TableMetadata tableMetadata) 39 | { 40 | super(tableMetadata); 41 | } 42 | 43 | @Override 44 | public SparkRangeTombstone buildTombstone(List start, 45 | boolean isStartInclusive, 46 | List end, 47 | boolean isEndInclusive) 48 | { 49 | return SparkRangeTombstone.of(start, isStartInclusive, end, isEndInclusive); 50 | } 51 | 52 | @Override 53 | public SparkValueWithMetadata buildValue(String name, String type, ByteBuffer buf) 54 | { 55 | return SparkValueWithMetadata.of(name, type, buf); 56 | } 57 | } -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/Boolean.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import org.apache.cassandra.spark.data.fourzero.NativeType; 4 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.SettableByIndexData; 5 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.AbstractType; 6 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.BooleanType; 7 | import org.apache.cassandra.spark.utils.RandomUtils; 8 | 9 | /* 10 | * 11 | * Licensed to the Apache Software Foundation (ASF) under one 12 | * or more contributor license agreements. See the NOTICE file 13 | * distributed with this work for additional information 14 | * regarding copyright ownership. The ASF licenses this file 15 | * to you under the Apache License, Version 2.0 (the 16 | * "License"); you may not use this file except in compliance 17 | * with the License. You may obtain a copy of the License at 18 | * 19 | * http://www.apache.org/licenses/LICENSE-2.0 20 | * 21 | * Unless required by applicable law or agreed to in writing, 22 | * software distributed under the License is distributed on an 23 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 24 | * KIND, either express or implied. See the License for the 25 | * specific language governing permissions and limitations 26 | * under the License. 27 | * 28 | */ 29 | 30 | public class Boolean extends NativeType 31 | { 32 | public static final Boolean INSTANCE = new Boolean(); 33 | 34 | @Override 35 | public String name() 36 | { 37 | return "boolean"; 38 | } 39 | 40 | @Override 41 | public AbstractType dataType() 42 | { 43 | return BooleanType.instance; 44 | } 45 | 46 | @Override 47 | public int cardinality(int orElse) 48 | { 49 | return 2; 50 | } 51 | 52 | @Override 53 | public Object randomValue(int minCollectionSize) 54 | { 55 | return RandomUtils.RANDOM.nextBoolean(); 56 | } 57 | 58 | @Override 59 | public void setInnerValue(SettableByIndexData udtValue, int pos, Object value) 60 | { 61 | udtValue.setBool(pos, (boolean) value); 62 | } 63 | 64 | @Override 65 | public org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen) 66 | { 67 | return org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType.cboolean(); 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/cassandra/spark/data/fourzero/types/Inet.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.data.fourzero.types; 2 | 3 | import java.net.InetAddress; 4 | 5 | import com.google.common.net.InetAddresses; 6 | 7 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType; 8 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.SettableByIndexData; 9 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.AbstractType; 10 | import org.apache.cassandra.spark.shaded.fourzero.cassandra.db.marshal.InetAddressType; 11 | import org.apache.cassandra.spark.utils.RandomUtils; 12 | 13 | /* 14 | * 15 | * Licensed to the Apache Software Foundation (ASF) under one 16 | * or more contributor license agreements. See the NOTICE file 17 | * distributed with this work for additional information 18 | * regarding copyright ownership. The ASF licenses this file 19 | * to you under the Apache License, Version 2.0 (the 20 | * "License"); you may not use this file except in compliance 21 | * with the License. You may obtain a copy of the License at 22 | * 23 | * http://www.apache.org/licenses/LICENSE-2.0 24 | * 25 | * Unless required by applicable law or agreed to in writing, 26 | * software distributed under the License is distributed on an 27 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 28 | * KIND, either express or implied. See the License for the 29 | * specific language governing permissions and limitations 30 | * under the License. 31 | * 32 | */ 33 | 34 | public class Inet extends BinaryBased 35 | { 36 | public static final Inet INSTANCE = new Inet(); 37 | 38 | @Override 39 | public String name() 40 | { 41 | return "inet"; 42 | } 43 | 44 | @Override 45 | public AbstractType dataType() 46 | { 47 | return InetAddressType.instance; 48 | } 49 | 50 | @SuppressWarnings("UnstableApiUsage") 51 | @Override 52 | public Object randomValue(int minCollectionSize) 53 | { 54 | return InetAddresses.fromInteger(RandomUtils.RANDOM.nextInt()); 55 | } 56 | 57 | @Override 58 | public void setInnerValue(SettableByIndexData udtValue, int pos, Object value) 59 | { 60 | udtValue.setInet(pos, (InetAddress) value); 61 | } 62 | 63 | @Override 64 | public DataType driverDataType(boolean isFrozen) 65 | { 66 | return org.apache.cassandra.spark.shaded.fourzero.cassandra.cql3.functions.types.DataType.inet(); 67 | } 68 | } 69 | -------------------------------------------------------------------------------- /core/src/test/spark2/org/apache/cassandra/spark/sparksql/RangeFilterTests.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.sparksql; 2 | 3 | import java.math.BigInteger; 4 | 5 | import com.google.common.collect.Range; 6 | import org.junit.Test; 7 | 8 | import org.apache.cassandra.spark.reader.SparkSSTableReader; 9 | import org.apache.cassandra.spark.sparksql.filters.RangeFilter; 10 | 11 | import static org.junit.Assert.assertFalse; 12 | import static org.junit.Assert.assertTrue; 13 | import static org.mockito.Mockito.mock; 14 | import static org.mockito.Mockito.when; 15 | 16 | /* 17 | * 18 | * Licensed to the Apache Software Foundation (ASF) under one 19 | * or more contributor license agreements. See the NOTICE file 20 | * distributed with this work for additional information 21 | * regarding copyright ownership. The ASF licenses this file 22 | * to you under the Apache License, Version 2.0 (the 23 | * "License"); you may not use this file except in compliance 24 | * with the License. You may obtain a copy of the License at 25 | * 26 | * http://www.apache.org/licenses/LICENSE-2.0 27 | * 28 | * Unless required by applicable law or agreed to in writing, 29 | * software distributed under the License is distributed on an 30 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 31 | * KIND, either express or implied. See the License for the 32 | * specific language governing permissions and limitations 33 | * under the License. 34 | * 35 | */ 36 | 37 | public class RangeFilterTests 38 | { 39 | @Test 40 | public void testValidFilter() 41 | { 42 | final Range connected = Range.closed(BigInteger.ONE, BigInteger.TWO); 43 | final Range notConnected = Range.greaterThan(BigInteger.TEN); 44 | 45 | final RangeFilter filter = RangeFilter.create(Range.closed(BigInteger.ZERO, BigInteger.ONE)); 46 | final SparkSSTableReader reader = mock(SparkSSTableReader.class); 47 | when(reader.range()).thenReturn(connected); 48 | 49 | assertTrue(filter.overlaps(connected)); 50 | assertFalse(filter.overlaps(notConnected)); 51 | assertTrue(filter.skipPartition(BigInteger.TEN)); 52 | assertFalse(filter.skipPartition(BigInteger.ONE)); 53 | assertTrue(SparkSSTableReader.overlaps(reader, filter.tokenRange())); 54 | } 55 | 56 | @Test(expected = IllegalArgumentException.class) 57 | public void testInvalidRange() 58 | { 59 | final RangeFilter filter = RangeFilter.create(Range.atLeast(BigInteger.TEN)); 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /common/src/test/java/org/apache/cassandra/spark/utils/ArrayUtilsTest.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark.utils; 2 | 3 | import java.util.function.Consumer; 4 | 5 | import org.junit.Assert; 6 | import org.junit.Test; 7 | 8 | import static org.apache.cassandra.spark.utils.ArrayUtils.retain; 9 | 10 | /* 11 | * 12 | * Licensed to the Apache Software Foundation (ASF) under one 13 | * or more contributor license agreements. See the NOTICE file 14 | * distributed with this work for additional information 15 | * regarding copyright ownership. The ASF licenses this file 16 | * to you under the Apache License, Version 2.0 (the 17 | * "License"); you may not use this file except in compliance 18 | * with the License. You may obtain a copy of the License at 19 | * 20 | * http://www.apache.org/licenses/LICENSE-2.0 21 | * 22 | * Unless required by applicable law or agreed to in writing, 23 | * software distributed under the License is distributed on an 24 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 25 | * KIND, either express or implied. See the License for the 26 | * specific language governing permissions and limitations 27 | * under the License. 28 | * 29 | */ 30 | 31 | public class ArrayUtilsTest 32 | { 33 | @Test 34 | public void testRetain() 35 | { 36 | Object[] source = new Object[] {1, 2, 3, 4, 5}; 37 | Assert.assertArrayEquals(new Object[]{ 1, 2, 3 }, retain(source, 0, 3)); 38 | } 39 | 40 | @Test 41 | public void testRetainThrows() 42 | { 43 | // not using JUnit rule ExpectedException in order to assert multiple throwables. 44 | expectedThrows(() -> retain(null, 0, 1), 45 | t -> Assert.assertSame(IllegalArgumentException.class, t.getClass())); 46 | 47 | expectedThrows(() -> retain(new Object[] {1, 2, 3}, -1, 1), 48 | t -> Assert.assertSame(IllegalArgumentException.class, t.getClass())); 49 | 50 | expectedThrows(() -> retain(new Object[] {1, 2, 3}, 0, -1), 51 | t -> Assert.assertSame(IllegalArgumentException.class, t.getClass())); 52 | 53 | expectedThrows(() -> retain(new Object[] {1, 2, 3}, 0, 5), 54 | t -> Assert.assertSame(IllegalArgumentException.class, t.getClass())); 55 | } 56 | 57 | private void expectedThrows(Runnable test, Consumer throwableVerifier) 58 | { 59 | try 60 | { 61 | test.run(); 62 | } 63 | catch (Throwable t) 64 | { 65 | throwableVerifier.accept(t); 66 | } 67 | } 68 | } 69 | -------------------------------------------------------------------------------- /cdc/src/test/java/org/apache/cassandra/spark/CdcKryoSerializationTests.java: -------------------------------------------------------------------------------- 1 | package org.apache.cassandra.spark; 2 | 3 | import com.google.common.collect.ImmutableList; 4 | import com.google.common.collect.ImmutableMap; 5 | import org.junit.Test; 6 | 7 | import com.esotericsoftware.kryo.Kryo; 8 | import org.apache.cassandra.spark.cdc.Marker; 9 | import org.apache.cassandra.spark.data.partitioner.CassandraInstance; 10 | import org.apache.cassandra.spark.sparksql.filters.CdcOffset; 11 | import org.apache.cassandra.spark.sparksql.filters.InstanceLogs; 12 | import org.apache.cassandra.spark.sparksql.filters.SerializableCommitLog; 13 | import org.apache.cassandra.spark.utils.TimeUtils; 14 | 15 | import static org.apache.cassandra.spark.utils.KryoUtils.deserialize; 16 | import static org.apache.cassandra.spark.utils.KryoUtils.serializeToBytes; 17 | import static org.junit.Assert.assertEquals; 18 | import static org.junit.Assert.assertNotNull; 19 | 20 | public class CdcKryoSerializationTests 21 | { 22 | public static Kryo kryo() 23 | { 24 | return CdcKryoRegister.kryo(); 25 | } 26 | 27 | @Test 28 | public void testCdcOffset() 29 | { 30 | final CassandraInstance inst1 = new CassandraInstance("0", "local1-i1", "DC1"); 31 | final Marker marker1 = new Marker(inst1, 500L, 200); 32 | final CassandraInstance inst2 = new CassandraInstance("1", "local2-i1", "DC1"); 33 | final Marker marker2 = new Marker(inst2, 1000L, 350); 34 | final CassandraInstance inst3 = new CassandraInstance("2", "local3-i1", "DC1"); 35 | final Marker marker3 = new Marker(inst3, 1500L, 500); 36 | 37 | final CdcOffset offset = new CdcOffset(TimeUtils.nowMicros(), 38 | ImmutableMap.of( 39 | inst1, new InstanceLogs(marker1, ImmutableList.of(new SerializableCommitLog("CommitLog-6-12345.log", "/cassandra/d1", 500L, 10000L))), 40 | inst2, new InstanceLogs(marker2, ImmutableList.of(new SerializableCommitLog("CommitLog-6-98765.log", "/cassandra/d2", 1200L, 15000L))), 41 | inst3, new InstanceLogs(marker3, ImmutableList.of(new SerializableCommitLog("CommitLog-6-123987.log", "/cassandra/d3", 1500L, 20000L))) 42 | )); 43 | final byte[] ar = serializeToBytes(kryo(), offset); 44 | final CdcOffset deserialized = deserialize(kryo(), ar, CdcOffset.class); 45 | assertNotNull(deserialized); 46 | assertEquals(offset, deserialized); 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/cassandra/spark/data/fourzero/types/spark/complex/CqlList.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, 14 | * software distributed under the License is distributed on an 15 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | * KIND, either express or implied. See the License for the 17 | * specific language governing permissions and limitations 18 | * under the License. 19 | * 20 | */ 21 | 22 | package org.apache.cassandra.spark.data.fourzero.types.spark.complex; 23 | 24 | import java.util.List; 25 | import java.util.stream.Collectors; 26 | 27 | import org.apache.cassandra.spark.data.CqlField; 28 | import org.apache.cassandra.spark.data.SparkCqlField; 29 | import org.apache.cassandra.spark.data.fourzero.types.spark.FourZeroSparkCqlField; 30 | 31 | public class CqlList extends org.apache.cassandra.spark.data.fourzero.complex.CqlList implements SparkCqlField.ComplexTrait, SparkCqlField.SparkList, SparkCqlField.ListTrait 32 | { 33 | public CqlList(CqlField.CqlList list) 34 | { 35 | super(FourZeroSparkCqlField.decorate(list.type())); 36 | } 37 | 38 | @Override 39 | public SparkCqlField.SparkCqlType type(int i) 40 | { 41 | return (SparkCqlField.SparkCqlType) super.type(i); 42 | } 43 | 44 | @Override 45 | public SparkCqlField.SparkCqlType type() 46 | { 47 | return (SparkCqlField.SparkCqlType) super.type(); 48 | } 49 | 50 | public SparkCqlField.SparkFrozen freeze() 51 | { 52 | return new CqlFrozen(this); 53 | } 54 | 55 | public List sparkTypes() 56 | { 57 | return super.types().stream() 58 | .map(f -> (SparkCqlField.SparkCqlType) f) 59 | .collect(Collectors.toList()); 60 | } 61 | 62 | public SparkCqlField.SparkCqlType sparkType() 63 | { 64 | return (SparkCqlField.SparkCqlType) super.type(); 65 | } 66 | 67 | public SparkCqlField.SparkCqlType sparkType(int i) 68 | { 69 | return (SparkCqlField.SparkCqlType) super.type(i); 70 | } 71 | } 72 | --------------------------------------------------------------------------------