├── project ├── build.properties ├── plugins.sbt └── BuildUtil.scala ├── rootdoc.txt ├── sbt ├── sbt-launch-0.13.8.jar └── sbt ├── spark-cassandra-connector ├── src │ ├── it │ │ ├── resources │ │ │ ├── triggers │ │ │ │ └── README.txt │ │ │ ├── keystore │ │ │ ├── truststore │ │ │ ├── metrics.properties │ │ │ └── log4j.properties │ │ └── scala │ │ │ └── com │ │ │ └── datastax │ │ │ └── spark │ │ │ └── connector │ │ │ ├── sql │ │ │ └── CassandraPrunedScanSpec.scala │ │ │ ├── SparkCassandraITFlatSpecBase.scala │ │ │ ├── cql │ │ │ ├── CassandraSSLConnectorSpec.scala │ │ │ └── CassandraAuthenticatedConnectorSpec.scala │ │ │ ├── streaming │ │ │ └── StreamingSpec.scala │ │ │ └── util │ │ │ └── MultiThreadedSpec.scala │ ├── main │ │ └── scala │ │ │ ├── org │ │ │ └── apache │ │ │ │ └── spark │ │ │ │ ├── sql │ │ │ │ └── cassandra │ │ │ │ │ ├── package-info.java │ │ │ │ │ ├── package.scala │ │ │ │ │ ├── TableRef.scala │ │ │ │ │ └── types │ │ │ │ │ ├── UUIDType.scala │ │ │ │ │ └── InetAddressType.scala │ │ │ │ └── metrics │ │ │ │ └── MetricsUpdater.scala │ │ │ └── com │ │ │ └── datastax │ │ │ └── spark │ │ │ └── connector │ │ │ ├── util │ │ │ ├── package.scala │ │ │ ├── Quote.scala │ │ │ ├── CountingIterator.scala │ │ │ ├── ByteBufferUtil.scala │ │ │ ├── SpanningIterator.scala │ │ │ ├── MagicalTypeTricks.scala │ │ │ ├── BufferedIterator2.scala │ │ │ └── Symbols.scala │ │ │ ├── writer │ │ │ ├── package.scala │ │ │ ├── NullKeyColumnException.scala │ │ │ ├── QueryExecutor.scala │ │ │ ├── RowWriter.scala │ │ │ ├── BatchGroupingKey.scala │ │ │ ├── BatchStatementBuilder.scala │ │ │ ├── CassandraRowWriter.scala │ │ │ ├── PropertyExtractor.scala │ │ │ ├── DefaultRowWriter.scala │ │ │ ├── SqlRowWriter.scala │ │ │ ├── RichStatement.scala │ │ │ ├── RowWriterFactory.scala │ │ │ ├── ObjectSizeEstimator.scala │ │ │ ├── RoutingKeyGenerator.scala │ │ │ ├── WriteOption.scala │ │ │ ├── WritableToCassandra.scala │ │ │ ├── AsyncExecutor.scala │ │ │ └── RateLimiter.scala │ │ │ ├── rdd │ │ │ ├── package.scala │ │ │ ├── partitioner │ │ │ │ ├── package.scala │ │ │ │ ├── TokenRangeSplitter.scala │ │ │ │ ├── dht │ │ │ │ │ ├── Token.scala │ │ │ │ │ └── TokenRange.scala │ │ │ │ ├── CassandraRDDPartition.scala │ │ │ │ ├── Murmur3PartitionerTokenRangeSplitter.scala │ │ │ │ ├── RandomPartitionerTokenRangeSplitter.scala │ │ │ │ ├── CassandraPartitionedRDD.scala │ │ │ │ └── NodeAddresses.scala │ │ │ ├── reader │ │ │ │ ├── package.scala │ │ │ │ ├── RowReader.scala │ │ │ │ ├── KeyValueRowReader.scala │ │ │ │ ├── ValueRowReader.scala │ │ │ │ ├── PrefetchingResultSetIterator.scala │ │ │ │ └── ClassBasedRowReader.scala │ │ │ ├── ValidRDDType.scala │ │ │ ├── CqlWhereClause.scala │ │ │ ├── ClusteringOrder.scala │ │ │ ├── SpannedByKeyRDD.scala │ │ │ ├── SpannedRDD.scala │ │ │ └── ReadConf.scala │ │ │ ├── mapper │ │ │ ├── package.scala │ │ │ ├── ColumnMapperConvention.scala │ │ │ ├── ColumnMap.scala │ │ │ └── JavaBeanColumnMapper.scala │ │ │ ├── types │ │ │ ├── package.scala │ │ │ ├── TimestampFormatter.scala │ │ │ ├── TimestampParser.scala │ │ │ └── CollectionColumnType.scala │ │ │ ├── cql │ │ │ ├── package.scala │ │ │ ├── MultipleRetryPolicy.scala │ │ │ ├── PreparedStatementCache.scala │ │ │ ├── RefCountMap.scala │ │ │ └── SessionProxy.scala │ │ │ ├── streaming │ │ │ ├── package.scala │ │ │ └── CassandraStreamingRDD.scala │ │ │ ├── BatchSize.scala │ │ │ ├── PairRDDFunctions.scala │ │ │ ├── TupleValue.scala │ │ │ ├── UDTValue.scala │ │ │ ├── GettableByIndexData.scala │ │ │ └── ColumnSelector.scala │ ├── test │ │ ├── scala │ │ │ ├── com │ │ │ │ └── datastax │ │ │ │ │ ├── spark │ │ │ │ │ └── connector │ │ │ │ │ │ ├── testkit │ │ │ │ │ │ ├── package.scala │ │ │ │ │ │ └── SparkCassandraFixture.scala │ │ │ │ │ │ ├── rdd │ │ │ │ │ │ ├── reader │ │ │ │ │ │ │ └── ClassBasedRowReaderTest.scala │ │ │ │ │ │ └── partitioner │ │ │ │ │ │ │ └── dht │ │ │ │ │ │ │ ├── Murmur3TokenFactorySpec.scala │ │ │ │ │ │ │ └── RandomPartitionerTokenFactorySpec.scala │ │ │ │ │ │ ├── samples.scala │ │ │ │ │ │ ├── streaming │ │ │ │ │ │ └── TestProducer.scala │ │ │ │ │ │ ├── types │ │ │ │ │ │ ├── CanBuildFromTest.scala │ │ │ │ │ │ └── TypeSerializationTest.scala │ │ │ │ │ │ ├── writer │ │ │ │ │ │ ├── PropertyExtractorTest.scala │ │ │ │ │ │ ├── AsyncExecutorTest.scala │ │ │ │ │ │ └── RateLimiterSpec.scala │ │ │ │ │ │ ├── cql │ │ │ │ │ │ └── RetryDelayConfSpec.scala │ │ │ │ │ │ ├── ColumnSelectorSpec.scala │ │ │ │ │ │ └── util │ │ │ │ │ │ ├── SpanningIteratorSpec.scala │ │ │ │ │ │ └── BufferedIterator2Spec.scala │ │ │ │ │ └── driver │ │ │ │ │ └── core │ │ │ │ │ └── RowMock.scala │ │ │ └── org │ │ │ │ └── apache │ │ │ │ └── spark │ │ │ │ └── sql │ │ │ │ └── cassandra │ │ │ │ └── ConsolidateSettingsSpec.scala │ │ └── java │ │ │ └── com │ │ │ └── datastax │ │ │ └── spark │ │ │ └── connector │ │ │ ├── SampleJavaBeanWithoutNoArgsCtor.java │ │ │ ├── SampleJavaBeanSubClass.java │ │ │ ├── SampleJavaBean.java │ │ │ ├── SampleWeirdJavaBean.java │ │ │ ├── SampleJavaBeanWithMultipleCtors.java │ │ │ ├── SampleWithNestedJavaBean.java │ │ │ ├── SampleJavaBeanWithTransientFields.java │ │ │ └── SampleWithDeeplyNestedJavaBean.java │ └── perf │ │ └── scala │ │ └── com │ │ └── datastax │ │ └── spark │ │ └── connector │ │ ├── util │ │ ├── SpanningIteratorBenchmark.scala │ │ ├── PriorityHashMapBenchmark.scala │ │ └── BenchmarkUtil.scala │ │ └── writer │ │ └── BasicWriteBenchmark.scala ├── scala-2.10 │ └── src │ │ └── main │ │ └── scala │ │ └── com │ │ └── datastax │ │ └── spark │ │ └── connector │ │ └── util │ │ └── Reflect.scala └── scala-2.11 │ └── src │ └── main │ └── scala │ └── com │ └── datastax │ └── spark │ └── connector │ └── util │ └── Reflect.scala ├── .travis.yml ├── .gitignore ├── spark-cassandra-connector-demos ├── kafka-streaming │ └── src │ │ └── main │ │ ├── scala │ │ └── com │ │ │ └── datastax │ │ │ └── spark │ │ │ └── connector │ │ │ └── demo │ │ │ └── KafkaStreamingScala211App.scala │ │ └── resources │ │ ├── log4j.properties │ │ └── data │ │ └── words ├── simple-demos │ └── src │ │ └── main │ │ ├── resources │ │ ├── application.conf │ │ ├── log4j.properties │ │ └── data │ │ │ └── words │ │ └── scala │ │ └── com │ │ └── datastax │ │ └── spark │ │ └── connector │ │ └── demo │ │ ├── DemoApp.scala │ │ ├── SparkCassandraSettings.scala │ │ ├── WordCountDemo.scala │ │ ├── TableCopyDemo.scala │ │ ├── BasicReadWriteDemo.scala │ │ └── SQLDemo.scala └── twitter-streaming │ └── src │ └── main │ ├── resources │ ├── application.conf │ └── log4j.properties │ └── scala │ └── com │ └── datastax │ └── spark │ └── connector │ └── demo │ └── TwitterStreamingTopicsByInterval.scala ├── dev └── run-tests.sh ├── spark-cassandra-connector-embedded ├── src │ └── main │ │ └── scala │ │ └── com │ │ └── datastax │ │ └── spark │ │ └── connector │ │ └── embedded │ │ ├── Event.scala │ │ ├── package.scala │ │ └── Assertions.scala ├── scala-2.10 │ └── src │ │ └── main │ │ └── scala │ │ └── com │ │ └── datastax │ │ └── spark │ │ └── connector │ │ └── embedded │ │ └── SparkRepl.scala └── scala-2.11 │ └── src │ └── main │ └── scala │ └── com │ └── datastax │ └── spark │ └── connector │ └── embedded │ └── SparkRepl.scala ├── spark-cassandra-connector-java └── src │ ├── main │ ├── scala │ │ └── com │ │ │ └── datastax │ │ │ └── spark │ │ │ └── connector │ │ │ └── japi │ │ │ ├── types │ │ │ └── JavaTypeConverter.scala │ │ │ ├── UDTValue.scala │ │ │ ├── TupleValue.scala │ │ │ └── CassandraRow.scala │ └── java │ │ └── com │ │ └── datastax │ │ └── spark │ │ └── connector │ │ └── japi │ │ ├── StreamingContextJavaFunctions.java │ │ ├── rdd │ │ └── CassandraJoinJavaRDD.java │ │ ├── DStreamJavaFunctions.java │ │ ├── PairRDDJavaFunctions.java │ │ ├── GenericJavaRowReaderFactory.java │ │ └── CassandraStreamingJavaUtil.java │ ├── test │ └── java │ │ └── com │ │ └── datastax │ │ └── spark │ │ └── connector │ │ └── japi │ │ ├── rdd │ │ └── CassandraJoinJavaRDDTest.java │ │ └── SparkContextJavaFunctionsTest.java │ └── it │ └── java │ └── com │ └── datastax │ └── spark │ └── connector │ └── CassandraStreamingJavaUtilTest.java ├── doc ├── 13_1_setup_spark_shell.md ├── 15_python.md └── 10_embedded.md └── scripts └── submit-demos /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version=0.13.8 2 | -------------------------------------------------------------------------------- /rootdoc.txt: -------------------------------------------------------------------------------- 1 | Cassandra connector for Apache Spark. 2 | See documentation of package [[com.datastax.spark.connector]]. -------------------------------------------------------------------------------- /sbt/sbt-launch-0.13.8.jar: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/Stratio/spark-cassandra-connector/HEAD/sbt/sbt-launch-0.13.8.jar -------------------------------------------------------------------------------- /spark-cassandra-connector/src/it/resources/triggers/README.txt: -------------------------------------------------------------------------------- 1 | Place triggers to be loaded in this directory, as jar files. 2 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/org/apache/spark/sql/cassandra/package-info.java: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.cassandra; -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/org/apache/spark/sql/cassandra/package.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql 2 | 3 | package object cassandra { 4 | 5 | } 6 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/it/resources/keystore: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/Stratio/spark-cassandra-connector/HEAD/spark-cassandra-connector/src/it/resources/keystore -------------------------------------------------------------------------------- /spark-cassandra-connector/src/it/resources/truststore: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/Stratio/spark-cassandra-connector/HEAD/spark-cassandra-connector/src/it/resources/truststore -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/util/package.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector 2 | 3 | /** Useful stuff that didn't fit elsewhere. */ 4 | package object util { 5 | 6 | } 7 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/util/Quote.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.util 2 | 3 | object Quote { 4 | 5 | def quote(name: String): String = "\"" + name + "\"" 6 | 7 | } 8 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/writer/package.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector 2 | 3 | /** Contains components for writing RDDs to Cassandra */ 4 | package object writer { 5 | 6 | } 7 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/it/scala/com/datastax/spark/connector/sql/CassandraPrunedScanSpec.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.sql 2 | 3 | class CassandraPrunedScanSpec extends CassandraDataSourceSpec { 4 | override def pushDown = false 5 | } -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: scala 2 | jdk: oraclejdk7 3 | scala: 4 | - 2.10.5 5 | - 2.11.6 6 | 7 | script: 8 | - "sbt ++$TRAVIS_SCALA_VERSION test:compile" 9 | - "sbt ++$TRAVIS_SCALA_VERSION it:compile" 10 | - "sbt ++$TRAVIS_SCALA_VERSION test" 11 | 12 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/writer/NullKeyColumnException.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.writer 2 | 3 | case class NullKeyColumnException(columnName: String) 4 | extends NullPointerException(s"Invalid null value for key column $columnName") 5 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/it/resources/metrics.properties: -------------------------------------------------------------------------------- 1 | #*.sink.csv.class=org.apache.spark.metrics.sink.CsvSink 2 | # 3 | ## Polling period for CsvSink 4 | #*.sink.csv.period=1 5 | # 6 | #*.sink.csv.unit=seconds 7 | # 8 | ## Polling directory for CsvSink 9 | #*.sink.csv.directory=/tmp/spark/sink 10 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/rdd/package.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector 2 | 3 | 4 | /** Contains [[com.datastax.spark.connector.rdd.CassandraTableScanRDD]] class that is the main entry point for 5 | * analyzing Cassandra data from Spark. */ 6 | package object rdd { 7 | 8 | } 9 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/org/apache/spark/sql/cassandra/TableRef.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.cassandra 2 | 3 | 4 | /** Store table name, keyspace name and option cluster name, keyspace is equivalent to database */ 5 | case class TableRef(table: String, keyspace: String, cluster: Option[String] = None) 6 | 7 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | *.class 2 | *.log 3 | .DS_Store 4 | # sbt specific 5 | .cache/ 6 | .history/ 7 | .lib/ 8 | dist/* 9 | target/ 10 | lib_managed/ 11 | src_managed/ 12 | project/boot/ 13 | project/plugins/project/ 14 | 15 | # Scala-IDE specific 16 | .scala_dependencies 17 | .worksheet 18 | .idea 19 | .idea_modules 20 | 21 | checkpoint 22 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/package.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd 2 | 3 | /** Provides components for partitioning a Cassandra table into smaller parts of appropriate size. 4 | * Each partition can be processed locally on at least one cluster node. */ 5 | package object partitioner { 6 | 7 | } 8 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/mapper/package.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector 2 | 3 | /** Provides machinery for mapping Cassandra tables to user defined Scala classes or tuples. 4 | * The main class in this package is [[mapper.ColumnMapper]] responsible for matching Scala object's 5 | * properties with Cassandra column names.*/ 6 | package object mapper { 7 | 8 | } 9 | -------------------------------------------------------------------------------- /spark-cassandra-connector-demos/kafka-streaming/src/main/scala/com/datastax/spark/connector/demo/KafkaStreamingScala211App.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.demo 2 | 3 | import org.apache.spark.Logging 4 | 5 | object KafkaStreamingScala211App extends App with Logging { 6 | 7 | log.info("Spark is not yet supporting Kafka with Scala 2.11, or publishing the spark-streaming-kafka artifact.You can run the demo against Scala 2.10 only so far.") 8 | } -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/types/package.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector 2 | 3 | /** Offers type conversion magic, so you can receive Cassandra column values in a form you like the most. 4 | * Simply specify the type you want to use on the Scala side, and the column value will be converted automatically. 5 | * Works also with complex objects like collections. */ 6 | package object types { 7 | 8 | } 9 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/rdd/reader/package.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd 2 | 3 | import com.datastax.spark.connector.CassandraRow 4 | 5 | /** Provides components for reading data rows from Cassandra and converting them to objects of desired type. 6 | * Additionally provides a generic [[CassandraRow CassandraRow]] class which can represent any row.*/ 7 | package object reader { 8 | 9 | } 10 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/cql/package.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector 2 | 3 | 4 | /** Contains a [[cql.CassandraConnector]] object which is used to connect 5 | * to a Cassandra cluster and to send CQL statements to it. `CassandraConnector` 6 | * provides a Scala-idiomatic way of working with `Cluster` and `Session` object 7 | * and takes care of connection pooling and proper resource disposal.*/ 8 | package object cql { 9 | 10 | } 11 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/scala/com/datastax/spark/connector/testkit/package.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector 2 | 3 | import scala.collection.immutable 4 | import scala.concurrent.duration._ 5 | import akka.util.Timeout 6 | 7 | package object testkit { 8 | 9 | final val DefaultHost = "127.0.0.1" 10 | 11 | implicit val DefaultTimeout = Timeout(5.seconds) 12 | 13 | val data = immutable.Set("words ", "may ", "count ") 14 | 15 | val actorName = "my-actor" 16 | 17 | } 18 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/TokenRangeSplitter.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd.partitioner 2 | 3 | import com.datastax.spark.connector.rdd.partitioner.dht.{Token, TokenRange} 4 | 5 | /** Splits a token range into smaller sub-ranges, 6 | * each with the desired approximate number of rows. */ 7 | trait TokenRangeSplitter[V, T <: Token[V]] { 8 | 9 | /** Splits given token range into n equal sub-ranges. */ 10 | def split(range: TokenRange[V, T], splitSize: Long): Seq[TokenRange[V, T]] 11 | } 12 | 13 | 14 | 15 | 16 | 17 | -------------------------------------------------------------------------------- /dev/run-tests.sh: -------------------------------------------------------------------------------- 1 | #!/bin/sh 2 | 3 | echo "Running tests for Scala 2.10" 4 | sbt/sbt clean package test it:test 5 | s210r="$?" 6 | 7 | echo "Running tests for Scala 2.11" 8 | sbt/sbt -Dscala-2.11=true clean package test it:test 9 | s211r="$?" 10 | 11 | retval=0 12 | 13 | if [ "$s210r" = "0" ]; then 14 | echo "Tests for Scala 2.10 succeeded" 15 | else 16 | echo "Tests for Scala 2.10 failed" 17 | retval=1 18 | fi 19 | 20 | if [ "$s211r" = "0" ]; then 21 | echo "Tests for Scala 2.11 succeeded" 22 | else 23 | echo "Tests for Scala 2.11 failed" 24 | retval=1 25 | fi 26 | 27 | 28 | exit $retval 29 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/streaming/package.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector 2 | 3 | import org.apache.spark.streaming.StreamingContext 4 | import org.apache.spark.streaming.dstream.DStream 5 | 6 | import scala.reflect.ClassTag 7 | 8 | package object streaming { 9 | 10 | implicit def toStreamingContextFunctions(ssc: StreamingContext): SparkContextFunctions = 11 | new StreamingContextFunctions(ssc) 12 | 13 | implicit def toDStreamFunctions[T: ClassTag](ds: DStream[T]): DStreamFunctions[T] = 14 | new DStreamFunctions[T](ds) 15 | 16 | } 17 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/writer/QueryExecutor.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.writer 2 | 3 | import com.datastax.driver.core.{ResultSet, Statement, Session} 4 | 5 | import AsyncExecutor.Handler 6 | 7 | class QueryExecutor(session: Session, maxConcurrentQueries: Int, 8 | successHandler: Option[Handler[RichStatement]], failureHandler: Option[Handler[RichStatement]]) 9 | 10 | extends AsyncExecutor[RichStatement, ResultSet]( 11 | stmt => session.executeAsync(stmt.asInstanceOf[Statement]), maxConcurrentQueries, successHandler, failureHandler) 12 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/BatchSize.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector 2 | 3 | import com.datastax.spark.connector.writer.WriteConf 4 | 5 | sealed trait BatchSize 6 | 7 | case class RowsInBatch(batchSize: Int) extends BatchSize 8 | case class BytesInBatch(batchSize: Int) extends BatchSize 9 | 10 | object BatchSize { 11 | @deprecated("Use com.datastax.spark.connector.FixedBatchSize instead of a number", "1.1") 12 | implicit def intToFixedBatchSize(batchSize: Int): RowsInBatch = RowsInBatch(batchSize) 13 | 14 | val Automatic = BytesInBatch(WriteConf.DefaultBatchSizeInBytes) 15 | } 16 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/dht/Token.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd.partitioner.dht 2 | 3 | trait Token[T] extends Ordered[Token[T]] { 4 | def value: T 5 | } 6 | 7 | case class LongToken(value: Long) extends Token[Long] { 8 | override def compare(that: Token[Long]) = value.compareTo(that.value) 9 | override def toString = value.toString 10 | } 11 | 12 | case class BigIntToken(value: BigInt) extends Token[BigInt] { 13 | override def compare(that: Token[BigInt]) = value.compare(that.value) 14 | override def toString = value.toString() 15 | } 16 | 17 | 18 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/types/TimestampFormatter.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.types 2 | 3 | import java.util.Date 4 | 5 | import org.apache.cassandra.serializers.TimestampSerializer 6 | import org.joda.time.DateTime 7 | import org.joda.time.format.DateTimeFormat 8 | 9 | /** Formats timestamps and dates using CQL timestamp format `yyyy-MM-dd HH:mm:ssZ` */ 10 | object TimestampFormatter { 11 | 12 | private val TimestampPattern = "yyyy-MM-dd HH:mm:ssZ" 13 | 14 | def format(date: Date): String = 15 | DateTimeFormat.forPattern(TimestampPattern).print(new DateTime(date.getTime)) 16 | } 17 | -------------------------------------------------------------------------------- /spark-cassandra-connector-embedded/src/main/scala/com/datastax/spark/connector/embedded/Event.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.embedded 2 | 3 | import akka.actor.ActorRef 4 | 5 | object Event { 6 | 7 | sealed trait Status extends Serializable 8 | 9 | case class ReceiverStarted(ref: ActorRef) extends Status 10 | 11 | case class Pushed(data: AnyRef) extends Status 12 | 13 | case object Completed extends Status 14 | 15 | case object Report extends Status 16 | 17 | sealed trait Task extends Serializable 18 | case object QueryTask extends Task 19 | 20 | case class WordCount(word: String, count: Int) extends Serializable 21 | 22 | } 23 | -------------------------------------------------------------------------------- /spark-cassandra-connector-embedded/src/main/scala/com/datastax/spark/connector/embedded/package.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector 2 | 3 | import java.net.InetAddress 4 | 5 | import scala.concurrent.duration.FiniteDuration 6 | 7 | package object embedded { 8 | 9 | implicit val ZookeeperConnectionString = s"${InetAddress.getLocalHost.getHostAddress}:2181" 10 | 11 | /* Factor by which to scale timeouts during tests, e.g. to account for shared build system load. */ 12 | implicit class SparkTestDuration(val duration: FiniteDuration) extends AnyVal { 13 | def dilated: FiniteDuration = (duration * 1.0).asInstanceOf[FiniteDuration] 14 | } 15 | } 16 | -------------------------------------------------------------------------------- /spark-cassandra-connector-java/src/main/scala/com/datastax/spark/connector/japi/types/JavaTypeConverter.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.japi.types 2 | 3 | import akka.japi.JavaPartialFunction 4 | import com.datastax.spark.connector.types.NullableTypeConverter 5 | 6 | import scala.reflect.runtime.universe._ 7 | 8 | class JavaTypeConverter[T <: AnyRef](typeTag: TypeTag[T], convertFunction: JavaPartialFunction[Any, T]) 9 | extends NullableTypeConverter[T] { 10 | 11 | override def targetTypeTag: TypeTag[T] = typeTag 12 | 13 | override def convertPF: PartialFunction[Any, T] = convertFunction 14 | 15 | def noMatch() = JavaPartialFunction.noMatch() 16 | } 17 | -------------------------------------------------------------------------------- /spark-cassandra-connector-demos/simple-demos/src/main/resources/application.conf: -------------------------------------------------------------------------------- 1 | #################################### 2 | # Streaming Demo Reference Config File # 3 | #################################### 4 | 5 | streaming-demo { 6 | 7 | # spark://127.0.0.1@7077,127.0.0.2@7077,127.0.0.3@7077 8 | # or a local spark://host@7077 9 | # This defaults to local 10 | spark.master = "local[12]" 11 | # Would normally be `ms` in config but Spark just wants the Long 12 | spark.streaming.batch.duration = 300 13 | spark.cleaner.ttl = 3600 14 | spark.cassandra.connection.host = "127.0.0.1" 15 | 16 | spark.cassandra.keyspace = "streaming_demo" 17 | spark.cassandra.table = "words" 18 | data = ["words ", "may ", "count "] 19 | } -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/rdd/ValidRDDType.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd 2 | 3 | import java.io.{Serializable => JavaSerializable} 4 | 5 | import com.datastax.spark.connector.types.TypeConverter 6 | 7 | import scala.annotation.implicitNotFound 8 | 9 | @implicitNotFound("Not a valid RDD type. There should exists either a type converter for the type or the type should implement Serializable") 10 | trait ValidRDDType[T] 11 | 12 | object ValidRDDType { 13 | implicit def withTypeConverterAsValidRDDType[T](implicit tc: TypeConverter[T]): ValidRDDType[T] = null 14 | 15 | implicit def javaSerializableAsValidRDDType[T <: JavaSerializable]: ValidRDDType[T] = null 16 | } 17 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/util/CountingIterator.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.util 2 | 3 | /** Counts elements fetched form the underlying iterator. Limit causes iterator to terminate early */ 4 | class CountingIterator[T](iterator: Iterator[T], limit: Option[Long] = None) extends Iterator[T] { 5 | private var _count = 0 6 | 7 | /** Returns the number of successful invocations of `next` */ 8 | def count = _count 9 | 10 | def hasNext = limit match { 11 | case Some(l) => _count < l && iterator.hasNext 12 | case _ => iterator.hasNext 13 | } 14 | 15 | def next() = { 16 | val item = iterator.next() 17 | _count += 1 18 | item 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/dht/TokenRange.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd.partitioner.dht 2 | 3 | import java.net.InetAddress 4 | 5 | 6 | case class TokenRange[V, T <: Token[V]] ( 7 | start: T, end: T, replicas: Set[InetAddress], dataSize: Long) { 8 | 9 | def isWrapAround: Boolean = 10 | start >= end 11 | 12 | def unwrap(implicit tokenFactory: TokenFactory[V, T]): Seq[TokenRange[V, T]] = { 13 | val minToken = tokenFactory.minToken 14 | if (isWrapAround) 15 | Seq( 16 | TokenRange(start, minToken, replicas, dataSize / 2), 17 | TokenRange(minToken, end, replicas, dataSize / 2)) 18 | else 19 | Seq(this) 20 | } 21 | } -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | // pulls in: sbt-pgp, sbt-release, sbt-mima-plugin, sbt-dependency-graph, sbt-buildinfo, sbt-sonatype 2 | // TODO use sbt-release plugin 3 | addSbtPlugin("org.typelevel" % "sbt-typelevel" % "0.3.1") 4 | 5 | addSbtPlugin("com.typesafe.sbt" % "sbt-git" % "0.6.4") 6 | 7 | addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.11.2") 8 | 9 | addSbtPlugin("com.scalapenos" % "sbt-prompt" % "0.2.1") 10 | 11 | addSbtPlugin("com.typesafe.sbt" % "sbt-scalariform" % "1.3.0") 12 | 13 | addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.6.0") 14 | 15 | resolvers += "sonatype-releases" at "https://oss.sonatype.org/content/repositories/releases/" 16 | 17 | resolvers += Classpaths.sbtPluginReleases 18 | 19 | addSbtPlugin("org.scoverage" % "sbt-scoverage" % "1.0.4") 20 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/it/scala/com/datastax/spark/connector/SparkCassandraITFlatSpecBase.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector 2 | 3 | import com.datastax.spark.connector.embedded.SparkTemplate 4 | import com.datastax.spark.connector.testkit.{AbstractSpec, SharedEmbeddedCassandra} 5 | import org.apache.spark.SparkContext 6 | import org.scalatest._ 7 | 8 | 9 | trait SparkCassandraITFlatSpecBase extends FlatSpec with SparkCassandraITSpecBase 10 | 11 | trait SparkCassandraITWordSpecBase extends WordSpec with SparkCassandraITSpecBase 12 | 13 | trait SparkCassandraITAbstractSpecBase extends AbstractSpec with SparkCassandraITSpecBase 14 | 15 | trait SparkCassandraITSpecBase extends Suite with Matchers with SharedEmbeddedCassandra with SparkTemplate with BeforeAndAfterAll { 16 | } 17 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/scala/com/datastax/spark/connector/testkit/SparkCassandraFixture.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.testkit 2 | 3 | import org.scalatest.{BeforeAndAfter, Matchers, WordSpecLike} 4 | import com.datastax.spark.connector.cql.CassandraConnector 5 | import com.datastax.spark.connector.embedded.EmbeddedCassandra 6 | 7 | /** Basic unit test abstraction. */ 8 | trait AbstractSpec extends WordSpecLike with Matchers with BeforeAndAfter 9 | 10 | /** Used for IT tests. */ 11 | trait SharedEmbeddedCassandra extends EmbeddedCassandra { 12 | 13 | def clearCache(): Unit = CassandraConnector.evictCache() 14 | 15 | } 16 | 17 | private[connector] object TestEvent { 18 | 19 | case object Stop 20 | 21 | case object Completed 22 | 23 | case class WordCount(word: String, count: Int) 24 | 25 | } 26 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/rdd/CqlWhereClause.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd 2 | 3 | /** Represents a logical conjunction of CQL predicates. 4 | * Each predicate can have placeholders denoted by '?' which get substituted by values from the `values` array. 5 | * The number of placeholders must match the size of the `values` array. */ 6 | case class CqlWhereClause(predicates: Seq[String], values: Seq[Any]) { 7 | 8 | /** Returns a conjunction of this clause and the given predicate. */ 9 | def and(other: CqlWhereClause) = 10 | CqlWhereClause(predicates ++ other.predicates, values ++ other.values) 11 | 12 | } 13 | 14 | object CqlWhereClause { 15 | 16 | /** Empty CQL WHERE clause selects all rows */ 17 | val empty = new CqlWhereClause(Nil, Nil) 18 | } 19 | 20 | 21 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/PairRDDFunctions.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector 2 | 3 | import com.datastax.spark.connector.rdd.SpannedByKeyRDD 4 | import org.apache.spark.rdd.RDD 5 | 6 | class PairRDDFunctions[K, V](rdd: RDD[(K, V)]) extends Serializable { 7 | 8 | /** 9 | * Groups items with the same key, assuming the items with the same key are next to each other 10 | * in the collection. It does not perform shuffle, therefore it is much faster than using 11 | * much more universal Spark RDD `groupByKey`. For this method to be useful with Cassandra tables, 12 | * the key must represent a prefix of the primary key, containing at least the partition key of the 13 | * Cassandra table. */ 14 | def spanByKey: RDD[(K, Seq[V])] = 15 | new SpannedByKeyRDD[K, V](rdd) 16 | 17 | } 18 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/mapper/ColumnMapperConvention.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.mapper 2 | 3 | import com.datastax.spark.connector.ColumnRef 4 | import org.apache.commons.lang.StringUtils 5 | 6 | object ColumnMapperConvention { 7 | 8 | def camelCaseToUnderscore(str: String): String = 9 | StringUtils.splitByCharacterTypeCamelCase(str).mkString("_").replaceAll("_+", "_").toLowerCase 10 | 11 | def columnForProperty(propertyName: String, columnByName: Map[String, ColumnRef]): Option[ColumnRef] = { 12 | val underscoreName = camelCaseToUnderscore(propertyName) 13 | val candidateColumnNames = Seq(propertyName, underscoreName) 14 | candidateColumnNames.iterator 15 | .map(name => columnByName.get(name)) 16 | .find(_.isDefined) 17 | .flatten 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/writer/RowWriter.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.writer 2 | 3 | 4 | /** `RowWriter` knows how to extract column names and values from custom row objects 5 | * and how to convert them to values that can be written to Cassandra. 6 | * `RowWriter` is required to apply any user-defined data type conversion. */ 7 | trait RowWriter[T] extends Serializable { 8 | 9 | /** List of columns this `RowWriter` is going to write. 10 | * Used to construct appropriate INSERT or UPDATE statement. */ 11 | def columnNames: Seq[String] 12 | 13 | /** Extracts column values from `data` object and writes them into the given buffer 14 | * in the same order as they are listed in the columnNames sequence. */ 15 | def readColumnValues(data: T, buffer: Array[Any]) 16 | 17 | } 18 | -------------------------------------------------------------------------------- /spark-cassandra-connector-java/src/main/java/com/datastax/spark/connector/japi/StreamingContextJavaFunctions.java: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.japi; 2 | 3 | import org.apache.spark.streaming.StreamingContext; 4 | 5 | /** 6 | * Java API wrapper over {@link org.apache.spark.streaming.StreamingContext} to provide Spark Cassandra Connector 7 | * functionality. 8 | * 9 | *

To obtain an instance of this wrapper, use one of the factory methods in {@link 10 | * com.datastax.spark.connector.japi.CassandraJavaUtil} class.

11 | */ 12 | @SuppressWarnings("UnusedDeclaration") 13 | public class StreamingContextJavaFunctions extends SparkContextJavaFunctions { 14 | public final StreamingContext ssc; 15 | 16 | StreamingContextJavaFunctions(StreamingContext ssc) { 17 | super(ssc.sparkContext()); 18 | this.ssc = ssc; 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/perf/scala/com/datastax/spark/connector/util/SpanningIteratorBenchmark.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.util 2 | 3 | object SpanningIteratorBenchmark extends App { 4 | 5 | val iterator = Iterator.from(0) 6 | val groupsOf1 = new SpanningIterator(iterator, (i: Int) => i) 7 | val groupsOf10 = new SpanningIterator(iterator, (i: Int) => i / 10) 8 | val groupsOf1000 = new SpanningIterator(iterator, (i: Int) => i / 1000) 9 | 10 | println("1,000,000 groups of size 1 per each iteration:") 11 | BenchmarkUtil.timeIt { 12 | groupsOf10.drop(1000000) 13 | } 14 | 15 | println("100,000 groups of size 10 per each iteration:") 16 | BenchmarkUtil.timeIt { 17 | groupsOf10.drop(100000) 18 | } 19 | 20 | println("1000 groups of size 1000, per each iteration:") 21 | BenchmarkUtil.timeIt { 22 | groupsOf1000.drop(1000) 23 | } 24 | 25 | } 26 | -------------------------------------------------------------------------------- /spark-cassandra-connector/scala-2.10/src/main/scala/com/datastax/spark/connector/util/Reflect.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.util 2 | 3 | import scala.reflect.runtime.universe._ 4 | 5 | private[connector] object Reflect { 6 | 7 | def constructor(tpe: Type): Symbol = tpe.declaration(nme.CONSTRUCTOR) 8 | 9 | def member(tpe: Type, name: String): Symbol = tpe.member(newTermName(name)) 10 | 11 | def methodSymbol(tpe: Type): MethodSymbol = { 12 | val constructors = constructor(tpe).asTerm.alternatives.map(_.asMethod) 13 | val paramCount = constructors.map(_.paramss.flatten.size).max 14 | constructors.filter(_.paramss.flatten.size == paramCount) match { 15 | case List(onlyOne) => onlyOne 16 | case _ => throw new IllegalArgumentException( 17 | "Multiple constructors with the same number of parameters not allowed.") 18 | } 19 | } 20 | } 21 | 22 | -------------------------------------------------------------------------------- /spark-cassandra-connector-demos/simple-demos/src/main/scala/com/datastax/spark/connector/demo/DemoApp.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.demo 2 | 3 | import org.apache.spark.{Logging, SparkContext, SparkConf} 4 | 5 | trait DemoApp extends App with Logging { 6 | 7 | val words = "./spark-cassandra-connector-demos/simple-demos/src/main/resources/data/words" 8 | 9 | val SparkMasterHost = "127.0.0.1" 10 | 11 | val CassandraHost = "127.0.0.1" 12 | 13 | // Tell Spark the address of one Cassandra node: 14 | val conf = new SparkConf(true) 15 | .set("spark.cassandra.connection.host", CassandraHost) 16 | .set("spark.cleaner.ttl", "3600") 17 | .setMaster("local[12]") 18 | .setAppName(getClass.getSimpleName) 19 | 20 | // Connect to the Spark cluster: 21 | lazy val sc = new SparkContext(conf) 22 | } 23 | 24 | object DemoApp { 25 | def apply(): DemoApp = new DemoApp {} 26 | } 27 | -------------------------------------------------------------------------------- /spark-cassandra-connector/scala-2.11/src/main/scala/com/datastax/spark/connector/util/Reflect.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.util 2 | 3 | import scala.reflect.runtime.universe._ 4 | 5 | private[connector] object Reflect { 6 | 7 | def constructor(tpe: Type): Symbol = tpe.decl(termNames.CONSTRUCTOR) 8 | 9 | def member(tpe: Type, name: String): Symbol = tpe.member(TermName(name)) 10 | 11 | def methodSymbol(tpe: Type): MethodSymbol = { 12 | val constructors = constructor(tpe).asTerm.alternatives.map(_.asMethod) 13 | val paramCount = constructors.map(_.paramLists.flatten.size).max 14 | constructors.filter(_.paramLists.flatten.size == paramCount) match { 15 | case List(onlyOne) => onlyOne 16 | case _ => throw new IllegalArgumentException( 17 | "Multiple constructors with the same number of parameters not allowed.") 18 | } 19 | } 20 | } 21 | 22 | -------------------------------------------------------------------------------- /spark-cassandra-connector-java/src/main/scala/com/datastax/spark/connector/japi/UDTValue.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.japi 2 | 3 | import scala.reflect.runtime.universe._ 4 | 5 | import com.datastax.spark.connector.types.{TypeConverter, NullableTypeConverter} 6 | import com.datastax.spark.connector.{UDTValue => ConnectorUDTValue} 7 | 8 | final class UDTValue(val columnNames: IndexedSeq[String], val columnValues: IndexedSeq[AnyRef]) 9 | extends JavaGettableData with Serializable 10 | 11 | object UDTValue { 12 | 13 | val UDTValueTypeTag = implicitly[TypeTag[UDTValue]] 14 | 15 | implicit object UDTValueConverter extends NullableTypeConverter[UDTValue] { 16 | def targetTypeTag = UDTValueTypeTag 17 | 18 | def convertPF = { 19 | case x: UDTValue => x 20 | case x: ConnectorUDTValue => 21 | new UDTValue(x.columnNames, x.columnValues) 22 | } 23 | } 24 | 25 | TypeConverter.registerConverter(UDTValueConverter) 26 | 27 | } -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/scala/com/datastax/spark/connector/rdd/reader/ClassBasedRowReaderTest.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd.reader 2 | 3 | import com.datastax.spark.connector.cql.{RegularColumn, ColumnDef, PartitionKeyColumn, TableDef} 4 | import org.apache.commons.lang3.SerializationUtils 5 | import org.junit.Test 6 | 7 | import com.datastax.spark.connector.types.{BigIntType, IntType, VarCharType} 8 | 9 | case class TestClass(a: String, b: Int, c: Option[Long]) 10 | 11 | class ClassBasedRowReaderTest { 12 | 13 | private val a = ColumnDef("a", PartitionKeyColumn, VarCharType) 14 | private val b = ColumnDef("b", RegularColumn, IntType) 15 | private val c = ColumnDef("c", RegularColumn, BigIntType) 16 | private val table = TableDef("test", "table", Seq(a), Nil, Seq(b, c)) 17 | 18 | @Test 19 | def testSerialize() { 20 | val reader = new ClassBasedRowReader[TestClass](table, table.columnRefs) 21 | SerializationUtils.roundtrip(reader) 22 | } 23 | 24 | } 25 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/rdd/ClusteringOrder.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd 2 | 3 | import com.datastax.spark.connector.cql.TableDef 4 | 5 | sealed trait ClusteringOrder extends Serializable { 6 | private[connector] def toCql(tableDef: TableDef): String 7 | } 8 | 9 | object ClusteringOrder { 10 | private[connector] def cqlClause(tableDef: TableDef, order: String) = 11 | tableDef.clusteringColumns.headOption.map(cc => s"""ORDER BY "${cc.columnName}" $order""") 12 | .getOrElse(throw new IllegalArgumentException("Order by can be specified only if there are some clustering columns")) 13 | 14 | case object Ascending extends ClusteringOrder { 15 | override private[connector] def toCql(tableDef: TableDef): String = cqlClause(tableDef, "ASC") 16 | } 17 | 18 | case object Descending extends ClusteringOrder { 19 | override private[connector] def toCql(tableDef: TableDef): String = cqlClause(tableDef, "DESC") 20 | } 21 | 22 | } 23 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/java/com/datastax/spark/connector/SampleJavaBeanWithoutNoArgsCtor.java: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector; 2 | 3 | import java.io.Serializable; 4 | 5 | /** 6 | * This is a sample JavaBean style class. In order to test JavaAPI correctly, we cannot implement this in Scala because 7 | * Scala adds some additional accessors and mutators. 8 | */ 9 | public class SampleJavaBeanWithoutNoArgsCtor implements Serializable { 10 | private Integer key; 11 | private String value; 12 | 13 | private SampleJavaBeanWithoutNoArgsCtor(Integer key, String value) { 14 | this.key = key; 15 | this.value = value; 16 | } 17 | 18 | public Integer getKey() { 19 | return key; 20 | } 21 | 22 | public void setKey(Integer key) { 23 | this.key = key; 24 | } 25 | 26 | public String getValue() { 27 | return value; 28 | } 29 | 30 | public void setValue(String value) { 31 | this.value = value; 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/java/com/datastax/spark/connector/SampleJavaBeanSubClass.java: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector; 2 | 3 | /** 4 | * This is a sample JavaBean style class/subclass. In order to test JavaAPI correctly, we cannot 5 | * implement this in Scala because Scala adds some additional accessors and mutators. 6 | */ 7 | public class SampleJavaBeanSubClass extends SampleJavaBean 8 | { 9 | private String subClassField; 10 | 11 | public static SampleJavaBeanSubClass newInstance(Integer key, String value, String subClassField) { 12 | SampleJavaBeanSubClass bean = new SampleJavaBeanSubClass(); 13 | bean.setKey(key); 14 | bean.setValue(value); 15 | bean.setSubClassField(subClassField); 16 | return bean; 17 | } 18 | 19 | public String getSubClassField() 20 | { 21 | return subClassField; 22 | } 23 | 24 | public void setSubClassField(String subClassField) 25 | { 26 | this.subClassField = subClassField; 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/scala/com/datastax/spark/connector/samples.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector { 2 | 3 | case class SampleScalaCaseClass(key: Int, value: String) 4 | 5 | class SampleScalaClass(val key: Int, val value: String) extends Serializable 6 | 7 | class SampleScalaClassWithNoFields(key: Int, value: String) extends Serializable 8 | 9 | class SampleScalaClassWithMultipleCtors(var key: Int, var value: String) extends Serializable { 10 | def this(key: Int) = this(key, null) 11 | 12 | def this() = this(0, null) 13 | } 14 | 15 | class SampleWithNestedScalaCaseClass extends Serializable { 16 | 17 | case class InnerClass(key: Int, value: String) 18 | 19 | } 20 | 21 | class SampleWithDeeplyNestedScalaCaseClass extends Serializable { 22 | 23 | class IntermediateClass extends Serializable { 24 | 25 | case class InnerClass(key: Int, value: String) 26 | 27 | } 28 | 29 | } 30 | 31 | object SampleObject { 32 | 33 | case class ClassInObject(key: Int, value: String) 34 | 35 | } 36 | 37 | } -------------------------------------------------------------------------------- /spark-cassandra-connector-demos/simple-demos/src/main/scala/com/datastax/spark/connector/demo/SparkCassandraSettings.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.demo 2 | 3 | import com.typesafe.config.{Config, ConfigFactory} 4 | 5 | /* Initializes Akka, Cassandra and Spark settings. */ 6 | final class SparkCassandraSettings(rootConfig: Config) { 7 | def this() = this(ConfigFactory.load) 8 | 9 | protected val config = rootConfig.getConfig("streaming-demo") 10 | 11 | val SparkMaster: String = config.getString("spark.master") 12 | 13 | val SparkCleanerTtl: Int = config.getInt("spark.cleaner.ttl") 14 | 15 | val SparkStreamingBatchDuration: Long = config.getLong("spark.streaming.batch.duration") 16 | 17 | val Data = akka.japi.Util.immutableSeq(config.getStringList("data")).toSet 18 | 19 | val CassandraSeed: String = config.getString("spark.cassandra.connection.host") 20 | 21 | val CassandraKeyspace = config.getString("spark.cassandra.keyspace") 22 | 23 | val CassandraTable = config.getString("spark.cassandra.table") 24 | } -------------------------------------------------------------------------------- /spark-cassandra-connector-demos/simple-demos/src/main/scala/com/datastax/spark/connector/demo/WordCountDemo.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.demo 2 | 3 | import org.apache.spark.SparkContext._ 4 | import com.datastax.spark.connector.cql.CassandraConnector 5 | import com.datastax.spark.connector._ 6 | 7 | object WordCountDemo extends DemoApp { 8 | 9 | CassandraConnector(conf).withSessionDo { session => 10 | session.execute(s"CREATE KEYSPACE IF NOT EXISTS demo WITH REPLICATION = {'class': 'SimpleStrategy', 'replication_factor': 1 }") 11 | session.execute(s"CREATE TABLE IF NOT EXISTS demo.wordcount (word TEXT PRIMARY KEY, count COUNTER)") 12 | session.execute(s"TRUNCATE demo.wordcount") 13 | } 14 | 15 | sc.textFile(words) 16 | .flatMap(_.split("\\s+")) 17 | .map(word => (word.toLowerCase, 1)) 18 | .reduceByKey(_ + _) 19 | .saveToCassandra("demo", "wordcount") 20 | 21 | // print out the data saved from Spark to Cassandra 22 | sc.cassandraTable("demo", "wordcount").collect.foreach(println) 23 | sc.stop() 24 | } 25 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/java/com/datastax/spark/connector/SampleJavaBean.java: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector; 2 | 3 | import java.io.Serializable; 4 | 5 | /** 6 | * This is a sample JavaBean style class. In order to test JavaAPI correctly, we cannot implement this in Scala because 7 | * Scala adds some additional accessors and mutators. 8 | */ 9 | public class SampleJavaBean implements Serializable { 10 | private Integer key; 11 | private String value; 12 | 13 | public static SampleJavaBean newInstance(Integer key, String value) { 14 | SampleJavaBean bean = new SampleJavaBean(); 15 | bean.setKey(key); 16 | bean.setValue(value); 17 | return bean; 18 | } 19 | 20 | public Integer getKey() { 21 | return key; 22 | } 23 | 24 | public void setKey(Integer key) { 25 | this.key = key; 26 | } 27 | 28 | public String getValue() { 29 | return value; 30 | } 31 | 32 | public void setValue(String value) { 33 | this.value = value; 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/rdd/SpannedByKeyRDD.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd 2 | 3 | import org.apache.spark.{TaskContext, Partition} 4 | import org.apache.spark.annotation.DeveloperApi 5 | import org.apache.spark.rdd.RDD 6 | 7 | import com.datastax.spark.connector.util.SpanningIterator 8 | 9 | /** 10 | * Similar to [[SpannedRDD]] but, instead of extracting the key by the given function, 11 | * it groups binary tuples by the first element of each tuple. 12 | */ 13 | private[connector] class SpannedByKeyRDD[K, V](parent: RDD[(K, V)]) extends RDD[(K, Seq[V])](parent) { 14 | 15 | override protected def getPartitions = parent.partitions 16 | 17 | @DeveloperApi 18 | override def compute(split: Partition, context: TaskContext) = { 19 | val parentIterator = parent.iterator(split, context) 20 | def keyFunction(item: (K, V)) = item._1 21 | def extractValues(group: (K, Seq[(K, V)])) = (group._1, group._2.map(_._2)) 22 | new SpanningIterator(parentIterator, keyFunction).map(extractValues) 23 | } 24 | 25 | } 26 | -------------------------------------------------------------------------------- /spark-cassandra-connector-java/src/main/scala/com/datastax/spark/connector/japi/TupleValue.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.japi 2 | 3 | import scala.annotation.varargs 4 | import scala.reflect.runtime.universe._ 5 | 6 | import com.datastax.spark.connector.{TupleValue => ConnectorTupleValue} 7 | import com.datastax.spark.connector.types.{TypeConverter, NullableTypeConverter} 8 | 9 | final class TupleValue private (val columnValues: IndexedSeq[AnyRef]) 10 | extends JavaGettableByIndexData with Serializable 11 | 12 | 13 | object TupleValue { 14 | 15 | val TypeTag = typeTag[TupleValue] 16 | 17 | implicit object UDTValueConverter extends NullableTypeConverter[TupleValue] { 18 | def targetTypeTag = TypeTag 19 | 20 | def convertPF = { 21 | case x: TupleValue => x 22 | case x: ConnectorTupleValue => 23 | new TupleValue(x.columnValues) 24 | } 25 | } 26 | 27 | TypeConverter.registerConverter(UDTValueConverter) 28 | 29 | @varargs 30 | def newTuple(values: Object*): TupleValue = 31 | new TupleValue(values.toIndexedSeq) 32 | } 33 | 34 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/java/com/datastax/spark/connector/SampleWeirdJavaBean.java: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector; 2 | 3 | import java.io.Serializable; 4 | 5 | /** 6 | * This is a sample JavaBean style class. In order to test JavaAPI correctly, we cannot implement this in Scala because 7 | * Scala adds some additional accessors and mutators. 8 | */ 9 | public class SampleWeirdJavaBean implements Serializable { 10 | private Integer devil; 11 | private String cat; 12 | 13 | public static SampleWeirdJavaBean newInstance(Integer key, String value) { 14 | SampleWeirdJavaBean bean = new SampleWeirdJavaBean(); 15 | bean.setDevil(key); 16 | bean.setCat(value); 17 | return bean; 18 | } 19 | 20 | public Integer getDevil() { 21 | return devil; 22 | } 23 | 24 | public void setDevil(Integer devil) { 25 | this.devil = devil; 26 | } 27 | 28 | public String getCat() { 29 | return cat; 30 | } 31 | 32 | public void setCat(String cat) { 33 | this.cat = cat; 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/rdd/reader/RowReader.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd.reader 2 | 3 | import com.datastax.driver.core.{ProtocolVersion, Row} 4 | import com.datastax.spark.connector.ColumnRef 5 | 6 | /** Transforms a Cassandra Java driver `Row` into high-level row representation, e.g. a tuple 7 | * or a user-defined case class object. The target type `T` must be serializable. */ 8 | trait RowReader[T] extends Serializable { 9 | 10 | /** Reads column values from low-level `Row` and turns them into higher level representation. 11 | * @param row row fetched from Cassandra 12 | * @param columnNames column names available in the `row` 13 | * @param protocolVersion java driver protocol version to be used for deserialization */ 14 | def read(row: Row, columnNames: Array[String])(implicit protocolVersion: ProtocolVersion): T 15 | 16 | /** List of columns this `RowReader` is going to read. 17 | * Useful to avoid fetching the columns that are not needed. */ 18 | def neededColumns: Option[Seq[ColumnRef]] 19 | 20 | } 21 | -------------------------------------------------------------------------------- /spark-cassandra-connector-java/src/test/java/com/datastax/spark/connector/japi/rdd/CassandraJoinJavaRDDTest.java: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.japi.rdd; 2 | 3 | import static com.datastax.spark.connector.japi.CassandraJavaUtil.someColumns; 4 | import static org.hamcrest.CoreMatchers.is; 5 | import static org.hamcrest.MatcherAssert.assertThat; 6 | import org.junit.Test; 7 | import static org.mockito.Mockito.mock; 8 | import static org.mockito.Mockito.when; 9 | 10 | import com.datastax.spark.connector.rdd.CassandraJoinRDD; 11 | 12 | @SuppressWarnings({"unchecked", "RedundantTypeArguments"}) 13 | public class CassandraJoinJavaRDDTest { 14 | 15 | @Test 16 | public void testOn() { 17 | CassandraJoinRDD rdd = mock(CassandraJoinRDD.class); 18 | CassandraJoinRDD rdd2 = mock(CassandraJoinRDD.class); 19 | when(rdd.on(someColumns("a", "b"))).thenReturn(rdd2); 20 | CassandraJoinJavaRDD jrdd = new CassandraJoinJavaRDD<>(rdd, String.class, Integer.class); 21 | assertThat(jrdd.on(someColumns("a", "b")).rdd(), is(rdd2)); 22 | } 23 | 24 | } 25 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/writer/BatchGroupingKey.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.writer 2 | 3 | sealed trait BatchGroupingKey 4 | 5 | object BatchGroupingKey { 6 | 7 | /** Any row can be added to any batch. This works the same as previous batching implementation. */ 8 | case object None extends BatchGroupingKey 9 | 10 | /** Each batch is associated with a set of replicas. If a set of replicas for the inserted row is 11 | * the same as it is for a batch, the row can be added to the batch. */ 12 | case object ReplicaSet extends BatchGroupingKey 13 | 14 | /** Each batch is associated with a partition key. If the partition key of the inserted row is the 15 | * same as it is for a batch, the row can be added to the batch. */ 16 | case object Partition extends BatchGroupingKey 17 | 18 | def apply(name: String): BatchGroupingKey = name.toLowerCase match { 19 | case "none" => None 20 | case "replica_set" => ReplicaSet 21 | case "partition" => Partition 22 | case _ => throw new IllegalArgumentException(s"Invalid batch level: $name") 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/perf/scala/com/datastax/spark/connector/util/PriorityHashMapBenchmark.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.util 2 | 3 | import scala.util.Random 4 | 5 | /** Benchmarks a simulation of a heap-based stream-sort. 6 | * In each iteration, a random item is added to the priority queue 7 | * and the head item is removed. */ 8 | object PriorityHashMapBenchmark extends App { 9 | 10 | val capacity = 1024 * 8 11 | val count = 1024 * 8 12 | val random = new Random 13 | 14 | val m = new PriorityHashMap[Int, Int](capacity) 15 | 16 | // we need to measure how much the loop and random alone are taking 17 | println("Benchmarking caller...") 18 | BenchmarkUtil.timeIt { 19 | for (i <- 1 to 1000000) { // 1 million 20 | random.nextInt(count).asInstanceOf[AnyRef] 21 | random.nextInt().asInstanceOf[AnyRef] 22 | } 23 | } 24 | 25 | println("Benchmarking real code...") 26 | BenchmarkUtil.timeIt { 27 | for (i <- 1 to 1000000) { // 1 million 28 | if (m.size >= count) 29 | m.remove(m.headKey) 30 | m.put(random.nextInt(count), random.nextInt(count)) 31 | } 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/TupleValue.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector 2 | 3 | import scala.reflect.runtime.universe._ 4 | 5 | import com.datastax.driver.core.{ProtocolVersion, TupleValue => DriverTupleValue} 6 | import com.datastax.spark.connector.types.NullableTypeConverter 7 | 8 | final case class TupleValue(values: Any*) extends ScalaGettableByIndexData { 9 | override def columnValues = values.toIndexedSeq.map(_.asInstanceOf[AnyRef]) 10 | } 11 | 12 | object TupleValue { 13 | 14 | def fromJavaDriverTupleValue 15 | (value: DriverTupleValue) 16 | (implicit protocolVersion: ProtocolVersion): TupleValue = { 17 | val values = 18 | for (i <- 0 until value.getType.getComponentTypes.size()) yield 19 | GettableData.get(value, i) 20 | new TupleValue(values: _*) 21 | } 22 | 23 | val TypeTag = typeTag[TupleValue] 24 | val Symbol = typeOf[TupleValue].asInstanceOf[TypeRef].sym 25 | 26 | implicit object TupleValueConverter extends NullableTypeConverter[TupleValue] { 27 | def targetTypeTag = TypeTag 28 | def convertPF = { 29 | case x: TupleValue => x 30 | } 31 | } 32 | } -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/java/com/datastax/spark/connector/SampleJavaBeanWithMultipleCtors.java: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector; 2 | 3 | import java.io.Serializable; 4 | 5 | /** 6 | * This is a sample JavaBean style class. In order to test JavaAPI correctly, we cannot implement this in Scala because 7 | * Scala adds some additional accessors and mutators. 8 | */ 9 | public class SampleJavaBeanWithMultipleCtors implements Serializable { 10 | private Integer key; 11 | private String value; 12 | 13 | public SampleJavaBeanWithMultipleCtors(Integer key) { 14 | this.key = key; 15 | } 16 | 17 | public SampleJavaBeanWithMultipleCtors() { 18 | } 19 | 20 | public SampleJavaBeanWithMultipleCtors(Integer key, String value) { 21 | this.key = key; 22 | this.value = value; 23 | } 24 | 25 | public Integer getKey() { 26 | return key; 27 | } 28 | 29 | public void setKey(Integer key) { 30 | this.key = key; 31 | } 32 | 33 | public String getValue() { 34 | return value; 35 | } 36 | 37 | public void setValue(String value) { 38 | this.value = value; 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /spark-cassandra-connector-demos/twitter-streaming/src/main/resources/application.conf: -------------------------------------------------------------------------------- 1 | #################################### 2 | # Spark Cassandra Connector (Twitter Demo App) Config File # 3 | #################################### 4 | 5 | # This is the reference config file that contains all the default settings. 6 | streaming-app { 7 | 8 | # These can be search terms to filter for, or hashtags 9 | filters = ["cat", "dog"] 10 | 11 | spark { 12 | # The fallback Spark master, it auto-detection fails. 13 | # Can change to spark://127.0.0.1:7077 for example. 14 | master = "local[*]" 15 | 16 | # In seconds: Not using hcon 5s format until Spark 17 | # Upgrades their akka and thus config versions (to avoid a deprecation issue). 18 | streaming.batch.interval = 5 19 | 20 | # The default 21 | executor.memory = 2g 22 | cores.max = 2 23 | 24 | jars = [ 25 | "./spark-cassandra-connector-demos/twitter-streaming/target/scala-2.10/twitter-streaming-assembly-1.1.1-SNAPSHOT.jar" 26 | ] 27 | 28 | cassandra { 29 | connection.host = ["127.0.0.1"] 30 | keyspace = "twitter_stream" 31 | table = "topics_by_interval" 32 | } 33 | } 34 | } -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/scala/com/datastax/spark/connector/streaming/TestProducer.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.streaming 2 | 3 | import scala.concurrent.duration._ 4 | import akka.actor.{PoisonPill, Actor, ActorRef} 5 | 6 | class TestProducer(data: Array[String], to: ActorRef) extends Counter { 7 | import scala.util.Random 8 | import context.dispatcher 9 | 10 | val rand = new Random() 11 | 12 | val task = context.system.scheduler.schedule(2.second, 1.millis) { 13 | if (count < scale) { // we need this test to avoid generating more than 'scale' messages 14 | to ! makeMessage() 15 | increment() 16 | } 17 | } 18 | 19 | def receive: Actor.Receive = { 20 | case _ => 21 | } 22 | 23 | def makeMessage(): String = { 24 | val x = rand.nextInt(3) 25 | data(x) + data(2 - x) 26 | } 27 | } 28 | 29 | trait CounterFixture { 30 | val scale = 30 31 | } 32 | 33 | // CountDownLatch is not Serializable, can't use in stream so we do this. 34 | trait Counter extends Actor with CounterFixture { 35 | 36 | var count = 0 37 | 38 | def increment(): Unit = { 39 | count += 1 40 | if (count == scale) self ! PoisonPill 41 | } 42 | } -------------------------------------------------------------------------------- /project/BuildUtil.scala: -------------------------------------------------------------------------------- 1 | import sbt._ 2 | 3 | object BuildUtil { 4 | 5 | case class DocumentationMapping(url: URL, jarFileMatcher: Attributed[File] ⇒ Boolean) 6 | 7 | object DocumentationMapping { 8 | def apply(url: String, pattern: String): DocumentationMapping = { 9 | val regex = pattern.r 10 | new DocumentationMapping(new URL(url), file ⇒ regex.findPrefixOf(file.data.getName).isDefined) 11 | } 12 | 13 | def apply(url: URL, moduleIds: ModuleID*): DocumentationMapping = { 14 | def matches(fileName: String)(moduleID: ModuleID): Boolean = { 15 | fileName.matches(s"${moduleID.name}.*\\.jar") 16 | } 17 | 18 | val matcher = { file: Attributed[File] ⇒ moduleIds.exists(matches(file.data.getName)) } 19 | new DocumentationMapping(url, matcher) 20 | } 21 | 22 | def mapJarToDocURL( 23 | files: Seq[Attributed[File]], 24 | mappings: Seq[DocumentationMapping] 25 | ): Map[File, URL] = { 26 | 27 | val foundMappings = 28 | for (file ← files; mapping ← mappings if mapping.jarFileMatcher(file)) 29 | yield file.data → mapping.url 30 | 31 | foundMappings.toMap 32 | } 33 | 34 | } 35 | 36 | } 37 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/writer/BatchStatementBuilder.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.writer 2 | 3 | import com.datastax.driver.core._ 4 | import org.apache.spark.Logging 5 | 6 | private[connector] class BatchStatementBuilder( 7 | val batchType: BatchStatement.Type, 8 | val routingKeyGenerator: RoutingKeyGenerator, 9 | val consistencyLevel: ConsistencyLevel) extends Logging { 10 | 11 | /** Converts a sequence of statements into a batch if its size is greater than 1. 12 | * Sets the routing key and consistency level. */ 13 | def maybeCreateBatch(stmts: Seq[RichBoundStatement]): RichStatement = { 14 | require(stmts.size > 0, "Statements list cannot be empty") 15 | val stmt = stmts.head 16 | // for batch statements, it is enough to set routing key for the first statement 17 | stmt.setRoutingKey(routingKeyGenerator.apply(stmt)) 18 | 19 | if (stmts.size == 1) { 20 | stmt.setConsistencyLevel(consistencyLevel) 21 | stmt 22 | } else { 23 | val batch = new RichBatchStatement(batchType, stmts) 24 | batch.setConsistencyLevel(consistencyLevel) 25 | batch 26 | } 27 | } 28 | 29 | } 30 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/scala/com/datastax/spark/connector/types/CanBuildFromTest.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.types 2 | 3 | import org.apache.commons.lang3.SerializationUtils 4 | import org.junit.Assert._ 5 | import org.junit.Test 6 | 7 | class CanBuildFromTest { 8 | 9 | @Test 10 | def testBuild() { 11 | val bf = CanBuildFrom.setCanBuildFrom[Int] 12 | val builder = bf.apply() 13 | builder += 1 14 | builder += 2 15 | builder += 3 16 | assertEquals(Set(1,2,3), builder.result()) 17 | } 18 | 19 | @Test 20 | def testSerializeAndBuild() { 21 | val bf = CanBuildFrom.setCanBuildFrom[Int] 22 | val bf2 = SerializationUtils.roundtrip(bf) 23 | val builder = bf2.apply() 24 | builder += 1 25 | builder += 2 26 | builder += 3 27 | assertEquals(Set(1,2,3), builder.result()) 28 | } 29 | 30 | @Test 31 | def testSerializeAndBuildWithOrdering() { 32 | val bf = CanBuildFrom.treeSetCanBuildFrom[Int] 33 | val bf2 = SerializationUtils.roundtrip(bf) 34 | val builder = bf2.apply() 35 | builder += 1 36 | builder += 2 37 | builder += 3 38 | assertEquals(Set(1,2,3), builder.result()) 39 | } 40 | 41 | 42 | } 43 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/util/ByteBufferUtil.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.util 2 | 3 | import java.nio.ByteBuffer 4 | 5 | object ByteBufferUtil { 6 | 7 | /** Copies the remaining bytes of the buffer into the given array, starting from offset zero. 8 | * The array must have capacity to store all of the remaining bytes of the buffer. 9 | * The buffer's position remains untouched. */ 10 | def copyBuffer(src: ByteBuffer, dest: Array[Byte]): Array[Byte] = { 11 | if (src.hasArray) { 12 | val length = src.remaining 13 | val offset = src.arrayOffset + src.position 14 | System.arraycopy(src.array, offset, dest, 0, length) 15 | } else { 16 | src.duplicate.get(dest) 17 | } 18 | dest 19 | } 20 | 21 | /** Converts a byte buffer into an array. 22 | * The buffer's position remains untouched. */ 23 | def toArray(buffer: ByteBuffer): Array[Byte] = { 24 | if (buffer.hasArray && 25 | buffer.arrayOffset + buffer.position == 0 && 26 | buffer.remaining == buffer.array.length) { 27 | buffer.array 28 | } else { 29 | val dest = new Array[Byte](buffer.remaining) 30 | copyBuffer(buffer, dest) 31 | } 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/writer/CassandraRowWriter.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.writer 2 | 3 | import com.datastax.spark.connector.{ColumnRef, CassandraRow} 4 | import com.datastax.spark.connector.cql.TableDef 5 | 6 | /** A [[RowWriter]] that can write [[CassandraRow]] objects.*/ 7 | class CassandraRowWriter(table: TableDef, selectedColumns: IndexedSeq[ColumnRef]) extends RowWriter[CassandraRow] { 8 | 9 | override val columnNames = selectedColumns.map(_.columnName) 10 | 11 | private val columns = columnNames.map(table.columnByName).toIndexedSeq 12 | private val converters = columns.map(_.columnType.converterToCassandra) 13 | 14 | override def readColumnValues(data: CassandraRow, buffer: Array[Any]) = { 15 | for ((c, i) <- columnNames.zipWithIndex) { 16 | val value = data.getRaw(c) 17 | val convertedValue = converters(i).convert(value) 18 | buffer(i) = convertedValue 19 | } 20 | } 21 | } 22 | 23 | 24 | object CassandraRowWriter { 25 | 26 | object Factory extends RowWriterFactory[CassandraRow] { 27 | override def rowWriter(table: TableDef, selectedColumns: IndexedSeq[ColumnRef]) = 28 | new CassandraRowWriter(table, selectedColumns) 29 | } 30 | 31 | } 32 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/java/com/datastax/spark/connector/SampleWithNestedJavaBean.java: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector; 2 | 3 | import java.io.Serializable; 4 | 5 | /** 6 | * This is a sample JavaBean style class. In order to test JavaAPI correctly, we cannot implement this in Scala because 7 | * Scala adds some additional accessors and mutators. 8 | */ 9 | public class SampleWithNestedJavaBean implements Serializable { 10 | public class InnerClass implements Serializable { 11 | private Integer key; 12 | private String value; 13 | 14 | public InnerClass(Integer key) { 15 | this.key = key; 16 | } 17 | 18 | public InnerClass() { 19 | } 20 | 21 | public InnerClass(Integer key, String value) { 22 | this.key = key; 23 | this.value = value; 24 | } 25 | 26 | public Integer getKey() { 27 | return key; 28 | } 29 | 30 | public void setKey(Integer key) { 31 | this.key = key; 32 | } 33 | 34 | public String getValue() { 35 | return value; 36 | } 37 | 38 | public void setValue(String value) { 39 | this.value = value; 40 | } 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/scala/com/datastax/spark/connector/writer/PropertyExtractorTest.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.writer 2 | 3 | import org.junit.Assert._ 4 | import org.junit.Test 5 | 6 | class PropertyExtractorTest { 7 | 8 | class TestClass(val field1: String, val field2: Int) 9 | 10 | @Test 11 | def testSimpleExtraction() { 12 | val testObject = new TestClass("a", 1) 13 | val propertyExtractor = new PropertyExtractor(classOf[TestClass], Seq("field1", "field2")) 14 | val result = propertyExtractor.extract(testObject) 15 | assertEquals(2, result.size) 16 | assertEquals("a", result(0)) 17 | assertEquals(1, result(1)) 18 | } 19 | 20 | @Test 21 | def testAvailableProperties() { 22 | val triedProperties = Seq("field1", "foo", "bar") 23 | val availableProperties = PropertyExtractor.availablePropertyNames(classOf[TestClass], triedProperties) 24 | assertEquals(Seq("field1"), availableProperties) 25 | } 26 | 27 | @Test(expected = classOf[NoSuchMethodException]) 28 | def testWrongPropertyName() { 29 | val testObject = new TestClass("a", 1) 30 | val propertyExtractor = new PropertyExtractor(classOf[TestClass], Seq("foo")) 31 | propertyExtractor.extract(testObject) 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/util/SpanningIterator.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.util 2 | 3 | import scala.collection.mutable.ArrayBuffer 4 | 5 | /** An iterator that groups items having the same value of the given function (key). 6 | * To be included in the same group, items with the same key must be next to each other 7 | * in the original collection. 8 | * 9 | * `SpanningIterator` buffers internally one group at a time and the wrapped iterator 10 | * is consumed in a lazy way. 11 | * 12 | * Example: 13 | * {{{ 14 | * val collection = Seq(1 -> "a", 1 -> "b", 1 -> "c", 2 -> "d", 2 -> "e") 15 | * val iterator = new SpanningIterator(collection.iterator, (x: (Int, String)) => x._1) 16 | * val result = iterator.toSeq // Seq(1 -> Seq("a", "b", "c"), 2 -> Seq("d", "e")) 17 | * }}} 18 | */ 19 | class SpanningIterator[K, T](iterator: Iterator[T], f: T => K) extends Iterator[(K, Seq[T])] { 20 | 21 | private[this] val items = new BufferedIterator2(iterator) 22 | 23 | override def hasNext = items.hasNext 24 | 25 | override def next(): (K, Seq[T]) = { 26 | val key = f(items.head) 27 | val buffer = new ArrayBuffer[T] 28 | items.appendWhile(r => f(r) == key, buffer) 29 | (key, buffer) 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/it/scala/com/datastax/spark/connector/cql/CassandraSSLConnectorSpec.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.cql 2 | 3 | import com.datastax.spark.connector.SparkCassandraITFlatSpecBase 4 | import com.datastax.spark.connector.cql.CassandraConnectorConf.CassandraSSLConf 5 | import com.datastax.spark.connector.embedded.EmbeddedCassandra 6 | 7 | class CassandraSSLConnectorSpec extends SparkCassandraITFlatSpecBase { 8 | 9 | useCassandraConfig(Seq("cassandra-ssl.yaml.template")) 10 | 11 | val conn = CassandraConnector( 12 | hosts = Set(EmbeddedCassandra.getHost(0)), 13 | cassandraSSLConf = CassandraSSLConf( 14 | enabled = true, 15 | trustStorePath = Some(ClassLoader.getSystemResource("truststore").getPath), 16 | trustStorePassword = Some("connector"), 17 | enabledAlgorithms = Array("TLS_RSA_WITH_AES_128_CBC_SHA"))) 18 | 19 | // Wait for the default user to be created in Cassandra. 20 | Thread.sleep(1000) 21 | 22 | "A CassandraConnector" should "be able to use a secure connection when using native protocol" in { 23 | conn.withSessionDo { session => 24 | assert(session !== null) 25 | assert(session.isClosed === false) 26 | assert(session.getCluster.getMetadata.getClusterName === "Test Cluster0") 27 | } 28 | } 29 | 30 | } 31 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/scala/com/datastax/spark/connector/types/TypeSerializationTest.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.types 2 | 3 | import org.apache.commons.lang3.SerializationUtils 4 | import org.junit.Assert._ 5 | import org.junit.Test 6 | 7 | class TypeSerializationTest { 8 | 9 | private def testSerialization(t: ColumnType[_]) { 10 | assertEquals(t, SerializationUtils.roundtrip(t)) 11 | } 12 | 13 | @Test 14 | def testSerializationOfPrimitiveTypes() { 15 | testSerialization(AsciiType) 16 | testSerialization(TextType) 17 | testSerialization(IntType) 18 | testSerialization(BigIntType) 19 | testSerialization(DoubleType) 20 | testSerialization(FloatType) 21 | testSerialization(BooleanType) 22 | testSerialization(UUIDType) 23 | testSerialization(TimeUUIDType) 24 | testSerialization(TimestampType) 25 | testSerialization(DecimalType) 26 | testSerialization(BigIntType) 27 | testSerialization(InetType) 28 | testSerialization(CounterType) 29 | } 30 | 31 | @Test 32 | def testSerializationOfCollectionTypes() { 33 | testSerialization(ListType(IntType)) 34 | testSerialization(ListType(ListType(IntType))) 35 | testSerialization(SetType(TextType)) 36 | testSerialization(MapType(BigIntType, TimestampType)) 37 | } 38 | 39 | 40 | } 41 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/UDTValue.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector 2 | 3 | import scala.collection.JavaConversions._ 4 | import scala.reflect.runtime.universe._ 5 | 6 | import org.apache.commons.lang3.tuple 7 | 8 | import com.datastax.driver.core.{ProtocolVersion, UDTValue => DriverUDTValue} 9 | import com.datastax.spark.connector.types.NullableTypeConverter 10 | 11 | final case class UDTValue(columnNames: IndexedSeq[String], columnValues: IndexedSeq[AnyRef]) 12 | extends ScalaGettableData 13 | 14 | object UDTValue { 15 | 16 | def fromJavaDriverUDTValue(value: DriverUDTValue)(implicit protocolVersion: ProtocolVersion): UDTValue = { 17 | val fields = value.getType.getFieldNames.toIndexedSeq 18 | val values = fields.map(GettableData.get(value, _)) 19 | new UDTValue(fields, values) 20 | } 21 | 22 | def fromMap(map: Map[String, Any]): UDTValue = 23 | new UDTValue(map.keys.toIndexedSeq, map.values.map(_.asInstanceOf[AnyRef]).toIndexedSeq) 24 | 25 | val TypeTag = implicitly[TypeTag[UDTValue]] 26 | val Symbol = typeOf[UDTValue].asInstanceOf[TypeRef].sym 27 | 28 | implicit object UDTValueConverter extends NullableTypeConverter[UDTValue] { 29 | def targetTypeTag = TypeTag 30 | def convertPF = { 31 | case x: UDTValue => x 32 | } 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /spark-cassandra-connector-java/src/test/java/com/datastax/spark/connector/japi/SparkContextJavaFunctionsTest.java: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.japi; 2 | 3 | import org.apache.spark.SparkConf; 4 | import org.apache.spark.SparkContext; 5 | import org.junit.Test; 6 | import org.mockito.Mockito; 7 | 8 | import com.datastax.driver.core.ConsistencyLevel; 9 | import com.datastax.spark.connector.rdd.ReadConf; 10 | 11 | import static org.junit.Assert.assertEquals; 12 | import static org.mockito.Mockito.when; 13 | 14 | public class SparkContextJavaFunctionsTest 15 | { 16 | @Test 17 | public void testReadConfPopulating() { 18 | SparkConf conf = new SparkConf(); 19 | conf.set("spark.cassandra.input.fetch.size_in_rows", "1234"); 20 | conf.set("spark.cassandra.input.split.size_in_mb", "4321"); 21 | conf.set("spark.cassandra.input.consistency.level", "THREE"); 22 | 23 | SparkContext sc = Mockito.mock(SparkContext.class); 24 | when(sc.getConf()).thenReturn(conf); 25 | 26 | ReadConf readConf = CassandraJavaUtil.javaFunctions(sc).cassandraTable("a", "b").rdd().readConf(); 27 | 28 | assertEquals(readConf.fetchSizeInRows(), 1234); 29 | assertEquals(readConf.splitSizeInMB(), 4321); 30 | assertEquals(readConf.consistencyLevel(), ConsistencyLevel.THREE); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/writer/PropertyExtractor.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.writer 2 | 3 | import java.lang.reflect.Method 4 | 5 | import scala.util.Try 6 | 7 | /** Extracts values from fields of an object. */ 8 | class PropertyExtractor[T](val cls: Class[T], val propertyNames: Seq[String]) extends Serializable { 9 | 10 | private def getter(name: String) = 11 | cls.getMethod(name) 12 | 13 | @transient 14 | private lazy val methods: Array[Method] = 15 | propertyNames.map(getter).toArray 16 | 17 | @transient 18 | private lazy val methodByName = 19 | methods.map(m => (m.getName, m)).toMap 20 | 21 | def extract(obj: T): Array[AnyRef] = 22 | extract(obj, Array.ofDim(methods.length)) 23 | 24 | def extract(obj: T, target: Array[AnyRef]): Array[AnyRef] = { 25 | for (i <- 0 until methods.length) 26 | target(i) = methods(i).invoke(obj) 27 | target 28 | } 29 | 30 | def extractProperty(obj: T, propertyName: String): AnyRef = { 31 | val m = methodByName(propertyName) 32 | m.invoke(obj) 33 | } 34 | } 35 | 36 | object PropertyExtractor { 37 | 38 | def availablePropertyNames(cls: Class[_], requestedPropertyNames: Seq[String]): Seq[String] = 39 | requestedPropertyNames.filter(name => Try(cls.getMethod(name)).isSuccess) 40 | 41 | } 42 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/scala/com/datastax/driver/core/RowMock.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.driver.core 2 | 3 | import java.nio.ByteBuffer 4 | 5 | class RowMock(columnSizes: Option[Int]*) 6 | extends AbstractGettableData(ProtocolVersion.NEWEST_SUPPORTED) with Row { 7 | 8 | val bufs = columnSizes.map { 9 | case Some(size) => ByteBuffer.allocate(size) 10 | case _ => null 11 | }.toArray 12 | 13 | val defs = new ColumnDefinitions( 14 | columnSizes.map(i => new ColumnDefinitions.Definition("ks", "tab", s"c$i", DataType.text())).toArray) 15 | 16 | override def getColumnDefinitions: ColumnDefinitions = defs 17 | 18 | override def getBytesUnsafe(i: Int): ByteBuffer = bufs(i) 19 | 20 | override def getBytesUnsafe(s: String): ByteBuffer = getBytesUnsafe(defs.getIndexOf(s)) 21 | 22 | override def isNull(i: Int): Boolean = bufs(i) == null 23 | 24 | override def isNull(s: String): Boolean = isNull(defs.getIndexOf(s)) 25 | 26 | override def getIndexOf(name: String): Int = ??? 27 | 28 | override def getToken(i: Int): Token = ??? 29 | 30 | override def getToken(name: String): Token = ??? 31 | 32 | override def getPartitionKeyToken: Token = ??? 33 | 34 | override def getType(i: Int): DataType = ??? 35 | 36 | override def getValue(i: Int): ByteBuffer = ??? 37 | 38 | override def getName(i: Int): String = ??? 39 | } 40 | -------------------------------------------------------------------------------- /doc/13_1_setup_spark_shell.md: -------------------------------------------------------------------------------- 1 | # Documentation 2 | 3 | ## Setting up Cassandra 4 | 5 | The easiest way to get quickly started with Cassandra is to follow the instructions provided by 6 | [Datastax](http://docs.datastax.com/en/cassandra/2.1/cassandra/install/install_cassandraTOC.html) 7 | 8 | ## Setting up spark 9 | 10 | ### Download Spark 11 | 12 | Download a pre-built Spark from https://spark.apache.org/downloads.html 13 | Untar the tar.gz downloaded with 14 | 15 | tar -xvf spark-*-.tgz 16 | 17 | ### Start Spark in Stand Alone Mode (Optional) 18 | 19 | [Official Spark Instructions](https://spark.apache.org/docs/latest/spark-standalone.html) 20 | 21 | If you would like to run against a separate executor JVM then you need a running Spark Master and Worker. 22 | By default the spark-shell will run in local mode (driver/master/executor share a jvm.) 23 | 24 | Go to the newly created directory and start up Spark in stand-alone mode bound to localhost 25 | 26 | cd spark* 27 | ./sbin/start-all.sh 28 | 29 | At this point you should be able to access the Spark UI at localhost:8080. In the display you 30 | should see a single worker. At the top of this website you should see a URL set for the spark master. Save 31 | the master address (the entire spark://something:7077) if you would like to connect the shell to 32 | this stand alone spark master (use as sparkMasterAddress below). -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/rdd/SpannedRDD.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd 2 | 3 | import org.apache.spark.{TaskContext, Partition} 4 | import org.apache.spark.annotation.DeveloperApi 5 | import org.apache.spark.rdd.RDD 6 | 7 | import com.datastax.spark.connector.util.SpanningIterator 8 | 9 | /** 10 | * Groups items with the same key, assuming items with the same key are next to each other in 11 | * the parent collection. Contrary to Spark GroupedRDD, it does not perform shuffle, therefore it 12 | * is much faster. A key for each item is obtained by calling a given function. 13 | * 14 | * This RDD is very useful for grouping data coming out from Cassandra, because they are already 15 | * coming in order of partitioning key i.e. it is not possible for two rows 16 | * with the same partition key to be in different Spark partitions. 17 | * 18 | * @param parent parent RDD 19 | * @tparam K type of keys 20 | * @tparam T type of elements to be grouped together 21 | */ 22 | private[connector] class SpannedRDD[K, T](parent: RDD[T], f: T => K) 23 | extends RDD[(K, Iterable[T])](parent) { 24 | 25 | override protected def getPartitions = parent.partitions 26 | 27 | @DeveloperApi 28 | override def compute(split: Partition, context: TaskContext) = 29 | new SpanningIterator(parent.iterator(split, context), f) 30 | 31 | } 32 | 33 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/writer/DefaultRowWriter.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.writer 2 | 3 | import scala.reflect.runtime.universe._ 4 | import scala.collection.Seq 5 | 6 | import com.datastax.spark.connector.ColumnRef 7 | import com.datastax.spark.connector.cql.TableDef 8 | import com.datastax.spark.connector.mapper.ColumnMapper 9 | 10 | /** A `RowWriter` suitable for saving objects mappable by a [[com.datastax.spark.connector.mapper.ColumnMapper ColumnMapper]]. 11 | * Can save case class objects, java beans and tuples. */ 12 | class DefaultRowWriter[T : TypeTag : ColumnMapper]( 13 | table: TableDef, 14 | selectedColumns: IndexedSeq[ColumnRef]) 15 | extends RowWriter[T] { 16 | 17 | private val converter = MappedToGettableDataConverter[T](table, selectedColumns) 18 | override val columnNames = selectedColumns.map(_.columnName) 19 | 20 | override def readColumnValues(data: T, buffer: Array[Any]) = { 21 | val row = converter.convert(data) 22 | for (i <- columnNames.indices) 23 | buffer(i) = row.getRaw(i) 24 | } 25 | } 26 | 27 | object DefaultRowWriter { 28 | 29 | def factory[T : ColumnMapper : TypeTag] = new RowWriterFactory[T] { 30 | override def rowWriter(tableDef: TableDef, selectedColumns: IndexedSeq[ColumnRef]) = { 31 | new DefaultRowWriter[T](tableDef, selectedColumns) 32 | } 33 | } 34 | } 35 | 36 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/scala/com/datastax/spark/connector/cql/RetryDelayConfSpec.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.cql 2 | 3 | import scala.language.postfixOps 4 | 5 | import org.scalatest.{FlatSpec, Matchers} 6 | 7 | import scala.concurrent.duration._ 8 | import com.datastax.spark.connector.cql.CassandraConnectorConf.RetryDelayConf.{ExponentialDelay, LinearDelay, ConstantDelay} 9 | 10 | class RetryDelayConfSpec extends FlatSpec with Matchers { 11 | 12 | "ConstantDelay" should "return the same delay regardless of the retry number" in { 13 | val d = ConstantDelay(1234 milliseconds) 14 | d.forRetry(1) shouldBe (1234 milliseconds) 15 | d.forRetry(2) shouldBe (1234 milliseconds) 16 | d.forRetry(3) shouldBe (1234 milliseconds) 17 | } 18 | 19 | "LinearDelay" should "return the calculated delay for different retry numbers" in { 20 | val d = LinearDelay(1234 milliseconds, 200 milliseconds) 21 | d.forRetry(1) shouldBe (1234 milliseconds) 22 | d.forRetry(2) shouldBe (1434 milliseconds) 23 | d.forRetry(3) shouldBe (1634 milliseconds) 24 | } 25 | 26 | "ExponentialDelay" should "return the calculated delay for different retry numbers" in { 27 | val d = ExponentialDelay(1200 milliseconds, 2.5d) 28 | d.forRetry(1) shouldBe (1200 milliseconds) 29 | d.forRetry(2) shouldBe (3000 milliseconds) 30 | d.forRetry(3) shouldBe (7500 milliseconds) 31 | } 32 | 33 | } 34 | -------------------------------------------------------------------------------- /scripts/submit-demos: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | # This very basic script that submits the demos jar to a local spark master. 3 | # TODO add input validation and error handling. 4 | 5 | # ## 1. Build the demos assembly jar: 6 | # sbt -Dspark.cassandra.connector.demos.assembly=true assembly 7 | 8 | # ## 2. Run this script ## 9 | # Pass in 3 parameters: 10 | # 1. Path to /bin/spark-submit 11 | # 2. Spark master 12 | # 3. The FQCN of the demo class to run, e.g: com.datastax.spark.connector.demo.BasicReadWriteDemo 13 | # For further customization options see https://spark.apache.org/docs/latest/submitting-applications.html 14 | # Example: 15 | # sudo ./scripts/submit-demos /path/to/spark/bin spark://master:7077 com.datastax.spark.connector.demo.BasicReadWriteDemo 16 | # ## 17 | 18 | 19 | PATH_TO_SPARK_BIN_SCRIPTS=$1 20 | SPARK_MASTER=$2 21 | APP_TO_RUN=$3 22 | 23 | # TODO read from Settings.scala scalaVersion and version in ThisBuild: 24 | VERSION="1.0.0-SNAPSHOT" 25 | SCALA_VERSION="scala-2.10" 26 | DEMOS_ASSEMBLY_JAR="spark-cassandra-connector-demos-assembly-$VERSION.jar" 27 | PATH_TO_JAR="spark-cassandra-connector-demos/target/$SCALA_VERSION/$DEMOS_ASSEMBLY_JAR" 28 | SPARK_SUBMIT="$PATH_TO_SPARK_BIN_SCRIPTS/spark-submit" 29 | 30 | # Run on a Spark standalone cluster 31 | echo "Attempting to submit demo $SPARK_SUBMIT on $SPARK_MASTER with $PATH_TO_JAR" 32 | $SPARK_SUBMIT --class $APP_TO_RUN --master $SPARK_MASTER $PATH_TO_JAR 100 33 | 34 | 35 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/writer/SqlRowWriter.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.writer 2 | 3 | import com.datastax.spark.connector.ColumnRef 4 | import com.datastax.spark.connector.cql.TableDef 5 | import org.apache.spark.sql.catalyst.expressions.Row 6 | 7 | /** A [[RowWriter]] that can write SparkSQL `Row` objects. */ 8 | class SqlRowWriter(val table: TableDef, val selectedColumns: IndexedSeq[ColumnRef]) 9 | extends RowWriter[Row] { 10 | 11 | override val columnNames = selectedColumns.map(_.columnName) 12 | 13 | private val columns = columnNames.map(table.columnByName) 14 | private val converters = columns.map(_.columnType.converterToCassandra) 15 | 16 | /** Extracts column values from `data` object and writes them into the given buffer 17 | * in the same order as they are listed in the columnNames sequence. */ 18 | override def readColumnValues(row: Row, buffer: Array[Any]) = { 19 | require(row.size == columnNames.size, s"Invalid row size: ${row.size} instead of ${columnNames.size}.") 20 | for (i <- 0 until row.size) 21 | buffer(i) = converters(i).convert(row(i)) 22 | 23 | } 24 | 25 | } 26 | 27 | 28 | object SqlRowWriter { 29 | 30 | object Factory extends RowWriterFactory[Row] { 31 | override def rowWriter(table: TableDef, selectedColumns: IndexedSeq[ColumnRef]) = 32 | new SqlRowWriter(table, selectedColumns) 33 | } 34 | 35 | } -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/java/com/datastax/spark/connector/SampleJavaBeanWithTransientFields.java: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector; 2 | 3 | import java.io.Serializable; 4 | 5 | /** 6 | * This is a sample JavaBean style class. In order to test JavaAPI correctly, we cannot implement this in Scala because 7 | * Scala adds some additional accessors and mutators. 8 | */ 9 | public class SampleJavaBeanWithTransientFields implements Serializable { 10 | private Integer key; 11 | private String value; 12 | 13 | transient private String transientField; 14 | 15 | public static SampleJavaBeanWithTransientFields newInstance(Integer key, String value) { 16 | SampleJavaBeanWithTransientFields bean = new SampleJavaBeanWithTransientFields(); 17 | bean.setKey(key); 18 | bean.setValue(value); 19 | return bean; 20 | } 21 | 22 | public Integer getKey() { 23 | return key; 24 | } 25 | 26 | public void setKey(Integer key) { 27 | this.key = key; 28 | } 29 | 30 | public String getValue() { 31 | return value; 32 | } 33 | 34 | public void setValue(String value) { 35 | this.value = value; 36 | } 37 | 38 | public String getTransientField() 39 | { 40 | return transientField; 41 | } 42 | 43 | public void setTransientField(String transientField) 44 | { 45 | this.transientField = transientField; 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/java/com/datastax/spark/connector/SampleWithDeeplyNestedJavaBean.java: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector; 2 | 3 | import java.io.Serializable; 4 | 5 | /** 6 | * This is a sample JavaBean style class. In order to test JavaAPI correctly, we cannot implement this in Scala because 7 | * Scala adds some additional accessors and mutators. 8 | */ 9 | public class SampleWithDeeplyNestedJavaBean implements Serializable { 10 | public class IntermediateClass implements Serializable { 11 | public class InnerClass implements Serializable { 12 | private Integer key; 13 | private String value; 14 | 15 | public InnerClass(Integer key) { 16 | this.key = key; 17 | } 18 | 19 | public InnerClass() { 20 | } 21 | 22 | public InnerClass(Integer key, String value) { 23 | this.key = key; 24 | this.value = value; 25 | } 26 | 27 | public Integer getKey() { 28 | return key; 29 | } 30 | 31 | public void setKey(Integer key) { 32 | this.key = key; 33 | } 34 | 35 | public String getValue() { 36 | return value; 37 | } 38 | 39 | public void setValue(String value) { 40 | this.value = value; 41 | } 42 | } 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /doc/15_python.md: -------------------------------------------------------------------------------- 1 | # Documentation 2 | 3 | ## PySpark with Data Frames - Experimental 4 | 5 | With the inclusion of the Cassandra Data Source, PySpark can now be used with the Connector to 6 | access Cassandra data. This does not require DataStax Enterprise but you are limited to DataFrame 7 | only operations. 8 | 9 | ### Setup 10 | 11 | To enable Cassandra access the Spark Cassandra Connector assembly jar must be included on both the 12 | driver and executor classpath for the PySpark Java Gateway. This can be done by starting the PySpark 13 | shell similarlly to how the spark shell is started. 14 | 15 | ```bash 16 | ./bin/pyspark \ 17 | --driver-class-path spark-cassandra-connector-assembly-1.4.0-M1-SNAPSHOT.jar \ 18 | --jars spark-cassandra-connector-assembly-1.4.0-M1-SNAPSHOT.jar 19 | ``` 20 | 21 | ### Loading a DataFrame in Python 22 | 23 | A DataFrame can be created which links to cassandra by using the the `org.apache.spark.sql.cassandra` 24 | source and by specifying keyword arguements for `keyspace` and `table`. 25 | 26 | ```python 27 | sqlContext.read\ 28 | .format("org.apache.spark.sql.cassandra")\ 29 | .options(table="kv", keyspace="test")\ 30 | .load().show() 31 | ``` 32 | 33 | ``` 34 | +-+-+ 35 | |k|v| 36 | +-+-+ 37 | |5|5| 38 | |1|1| 39 | |2|2| 40 | |4|4| 41 | |3|3| 42 | +-+-+ 43 | ``` 44 | 45 | The options and parameters are identical to the Scala Data Frames Api so 46 | please see [Data Frames](14_data_frames.md) for more information. -------------------------------------------------------------------------------- /spark-cassandra-connector-java/src/main/java/com/datastax/spark/connector/japi/rdd/CassandraJoinJavaRDD.java: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.japi.rdd; 2 | 3 | import scala.reflect.ClassTag; 4 | 5 | import static com.datastax.spark.connector.japi.CassandraJavaUtil.safeClassTag; 6 | 7 | import com.datastax.spark.connector.ColumnSelector; 8 | import com.datastax.spark.connector.rdd.CassandraJoinRDD; 9 | 10 | public class CassandraJoinJavaRDD extends CassandraJavaPairRDD { 11 | 12 | public CassandraJoinJavaRDD(CassandraJoinRDD rdd, 13 | ClassTag keyClassTag, ClassTag valueClassTag) { 14 | 15 | super(rdd, keyClassTag, valueClassTag); 16 | } 17 | 18 | public CassandraJoinJavaRDD(CassandraJoinRDD rdd, 19 | Class keyClass, Class valueClass) { 20 | 21 | super(rdd, safeClassTag(keyClass), safeClassTag(valueClass)); 22 | } 23 | 24 | private CassandraJoinJavaRDD wrap(CassandraJoinRDD newRDD) { 25 | return new CassandraJoinJavaRDD<>(newRDD, kClassTag(), vClassTag()); 26 | } 27 | 28 | 29 | public CassandraJoinJavaRDD on(ColumnSelector joinColumns) { 30 | CassandraJoinRDD newRDD = rdd().on(joinColumns); 31 | return wrap(newRDD); 32 | } 33 | 34 | @Override 35 | public CassandraJoinRDD rdd() { 36 | return (CassandraJoinRDD) super.rdd(); 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/org/apache/spark/metrics/MetricsUpdater.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.metrics 2 | 3 | import com.codahale.metrics.Timer.Context 4 | import org.apache.spark.SparkEnv 5 | 6 | /** The base trait for metrics updaters implementations. The metrics updater is an object which provides 7 | * a unified way to update all the relevant metrics which are collected for the particular type of 8 | * activity. The connector provides `InputMetricsUpdater` and `OutputMetricsUpdater` which are aimed 9 | * to update all the read and write metrics respectively. */ 10 | trait MetricsUpdater { 11 | /** A method to be called when the task is finished. It stops the task timer and flushes data. */ 12 | def finish(): Long 13 | } 14 | 15 | /** Timer mixin allows to measure the time of a task - or, in other words - the time from creating an 16 | * instance to calling `com.codahale.metrics.Timer.Context.stop` method. */ 17 | trait Timer { 18 | def stopTimer(): Long 19 | } 20 | 21 | trait SimpleTimer extends Timer { 22 | private val startTime = System.nanoTime() 23 | 24 | override def stopTimer(): Long = System.nanoTime() - startTime 25 | } 26 | 27 | trait CCSTimer extends Timer { 28 | def source: CassandraConnectorSource 29 | 30 | val timer: Context 31 | 32 | override def stopTimer(): Long = { 33 | val t = timer.stop() 34 | Option(SparkEnv.get).flatMap(env => Option(env.metricsSystem)).foreach(_.report()) 35 | t 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/CassandraRDDPartition.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd.partitioner 2 | 3 | import java.net.InetAddress 4 | 5 | import org.apache.spark.Partition 6 | 7 | /** Stores a CQL `WHERE` predicate matching a range of tokens. */ 8 | case class CqlTokenRange(cql: String, values: Any*) 9 | 10 | trait EndpointPartition extends Partition { 11 | def endpoints: Iterable[InetAddress] 12 | } 13 | 14 | /** Metadata describing Cassandra table partition processed by a single Spark task. 15 | * Beware the term "partition" is overloaded. Here, in the context of Spark, 16 | * it means an arbitrary collection of rows that can be processed locally on a single Cassandra cluster node. 17 | * A `CassandraPartition` typically contains multiple CQL partitions, i.e. rows identified by different values of 18 | * the CQL partitioning key. 19 | * 20 | * @param index identifier of the partition, used internally by Spark 21 | * @param endpoints which nodes the data partition is located on 22 | * @param tokenRanges token ranges determining the row set to be fetched 23 | * @param rowCount estimated total row count in a partition 24 | */ 25 | case class CassandraPartition(index: Int, 26 | endpoints: Iterable[InetAddress], 27 | tokenRanges: Iterable[CqlTokenRange], 28 | rowCount: Long) extends EndpointPartition 29 | 30 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/writer/RichStatement.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.writer 2 | 3 | import java.util 4 | 5 | import com.datastax.driver.core.{BatchStatement, BoundStatement, PreparedStatement, Statement} 6 | 7 | trait RichStatement { 8 | def bytesCount: Int 9 | def rowsCount: Int 10 | } 11 | 12 | private[connector] class RichBoundStatement(stmt: PreparedStatement) extends BoundStatement(stmt) with RichStatement { 13 | var bytesCount = 0 14 | val rowsCount = 1 15 | } 16 | 17 | private[connector] class RichBatchStatement(batchType: BatchStatement.Type, stmts: Seq[RichBoundStatement]) 18 | extends BatchStatement(batchType) with RichStatement { 19 | 20 | // a small optimisation 21 | RichBatchStatement.ensureCapacity(this, stmts.size) 22 | var bytesCount = 0 23 | for (stmt <- stmts) { 24 | add(stmt) 25 | bytesCount += stmt.bytesCount 26 | } 27 | 28 | def rowsCount = size() 29 | } 30 | 31 | private[connector] object RichBatchStatement { 32 | private val statementsField = classOf[BatchStatement].getDeclaredField("statements") 33 | def ensureCapacity(batchStatement: BatchStatement, expectedCapacity: Int): Unit = { 34 | // TODO remove this workaround when https://datastax-oss.atlassian.net/browse/JAVA-649 is fixed 35 | statementsField.setAccessible(true) 36 | statementsField.get(batchStatement) 37 | .asInstanceOf[util.ArrayList[Statement]] 38 | .ensureCapacity(expectedCapacity) 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /spark-cassandra-connector-demos/simple-demos/src/main/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | 17 | # for production, you should probably set pattern to %c instead of %l. 18 | # (%l is slower.) 19 | 20 | # output messages into a rolling log file as well as stdout 21 | log4j.rootLogger=WARN,stdout 22 | 23 | # stdout 24 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 25 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 26 | log4j.appender.stdout.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n 27 | 28 | # Avoid "no host ID found" when starting a fresh node 29 | log4j.logger.org.apache.cassandra.db.SystemKeyspace=ERROR 30 | 31 | log4j.logger.com.datastax.spark.connector=INFO 32 | -------------------------------------------------------------------------------- /spark-cassandra-connector-demos/simple-demos/src/main/scala/com/datastax/spark/connector/demo/TableCopyDemo.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.demo 2 | 3 | import com.datastax.spark.connector.cql.CassandraConnector 4 | 5 | object TableCopyDemo extends DemoApp { 6 | 7 | CassandraConnector(conf).withSessionDo { session => 8 | session.execute("CREATE KEYSPACE IF NOT EXISTS test WITH REPLICATION = {'class': 'SimpleStrategy', 'replication_factor': 1 }") 9 | session.execute("CREATE TABLE IF NOT EXISTS test.source (key INT PRIMARY KEY, data VARCHAR)") 10 | session.execute("CREATE TABLE IF NOT EXISTS test.destination (key INT PRIMARY KEY, data VARCHAR)") 11 | session.execute("TRUNCATE test.source") 12 | session.execute("TRUNCATE test.destination") 13 | session.execute("INSERT INTO test.source(key, data) VALUES (1, 'first row')") 14 | session.execute("INSERT INTO test.source(key, data) VALUES (2, 'second row')") 15 | session.execute("INSERT INTO test.source(key, data) VALUES (3, 'third row')") 16 | } 17 | 18 | import com.datastax.spark.connector._ 19 | 20 | val src = sc.cassandraTable("test", "source") 21 | src.saveToCassandra("test", "destination") 22 | 23 | val dest = sc.cassandraTable("test", "destination") 24 | dest.collect().foreach(row => log.info(s"$row")) 25 | 26 | // Assert the rows were copied from test.source to test.destination table: 27 | assert(dest.collect().length == 3) 28 | 29 | log.info(s"Work completed, stopping the Spark context.") 30 | sc.stop() 31 | } 32 | -------------------------------------------------------------------------------- /spark-cassandra-connector-demos/kafka-streaming/src/main/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | 17 | # for production, you should probably set pattern to %c instead of %l. 18 | # (%l is slower.) 19 | 20 | # output messages into a rolling log file as well as stdout 21 | log4j.rootLogger=WARN,stdout 22 | 23 | # stdout 24 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 25 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 26 | log4j.appender.stdout.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n 27 | 28 | # Avoid "no host ID found" when starting a fresh node 29 | log4j.logger.org.apache.cassandra.db.SystemKeyspace=ERROR 30 | 31 | log4j.logger.com.datastax.spark.connector=INFO 32 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/streaming/CassandraStreamingRDD.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.streaming 2 | 3 | import com.datastax.spark.connector.cql.CassandraConnector 4 | import com.datastax.spark.connector.rdd.reader._ 5 | import com.datastax.spark.connector.rdd.{CassandraTableScanRDD, ClusteringOrder, CqlWhereClause, ReadConf} 6 | import com.datastax.spark.connector.{AllColumns, ColumnSelector} 7 | import org.apache.spark.streaming.StreamingContext 8 | 9 | import scala.reflect.ClassTag 10 | 11 | /** RDD representing a Cassandra table for Spark Streaming. 12 | * @see [[com.datastax.spark.connector.rdd.CassandraTableScanRDD]]*/ 13 | class CassandraStreamingRDD[R] private[connector] ( 14 | sctx: StreamingContext, 15 | connector: CassandraConnector, 16 | keyspace: String, 17 | table: String, 18 | columns: ColumnSelector = AllColumns, 19 | where: CqlWhereClause = CqlWhereClause.empty, 20 | empty: Boolean = false, 21 | limit: Option[Long] = None, 22 | clusteringOrder: Option[ClusteringOrder] = None, 23 | readConf: ReadConf = ReadConf())( 24 | implicit 25 | ct : ClassTag[R], 26 | @transient val rrf: RowReaderFactory[R]) 27 | extends CassandraTableScanRDD[R]( 28 | sc = sctx.sparkContext, 29 | connector = connector, 30 | keyspaceName = keyspace, 31 | tableName = table, 32 | columnNames = columns, 33 | where = where, 34 | limit = limit, 35 | clusteringOrder = clusteringOrder, 36 | readConf = readConf) 37 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/Murmur3PartitionerTokenRangeSplitter.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd.partitioner 2 | 3 | import com.datastax.spark.connector.rdd.partitioner.dht.{LongToken, TokenFactory, TokenRange} 4 | 5 | /** Fast token range splitter assuming that data are spread out evenly in the whole range. 6 | * @param dataSize estimate of the size of the data in the whole ring */ 7 | class Murmur3PartitionerTokenRangeSplitter(dataSize: Long) 8 | extends TokenRangeSplitter[Long, LongToken] { 9 | 10 | private val tokenFactory = 11 | TokenFactory.Murmur3TokenFactory 12 | 13 | private type TR = TokenRange[Long, LongToken] 14 | 15 | /** Splits the token range uniformly into sub-ranges. 16 | * @param splitSize requested sub-split size, given in the same units as `dataSize` */ 17 | def split(range: TR, splitSize: Long): Seq[TR] = { 18 | val rangeSize = range.dataSize 19 | val rangeTokenCount = tokenFactory.distance(range.start, range.end) 20 | val n = math.max(1, math.round(rangeSize.toDouble / splitSize).toInt) 21 | 22 | val left = range.start.value 23 | val right = range.end.value 24 | val splitPoints = 25 | (for (i <- 0 until n) yield left + (rangeTokenCount * i / n).toLong) :+ right 26 | 27 | for (Seq(l, r) <- splitPoints.sliding(2).toSeq) yield 28 | new TokenRange[Long, LongToken]( 29 | new LongToken(l), 30 | new LongToken(r), 31 | range.replicas, 32 | rangeSize / n) 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/rdd/reader/KeyValueRowReader.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd.reader 2 | 3 | import com.datastax.driver.core.{ProtocolVersion, Row} 4 | import com.datastax.spark.connector.{ColumnSelector, ColumnRef} 5 | import com.datastax.spark.connector.cql.TableDef 6 | 7 | private[connector] class KeyValueRowReaderFactory[K, V]( 8 | keySelection: ColumnSelector, 9 | keyRRF: RowReaderFactory[K], 10 | valueRRF: RowReaderFactory[V]) 11 | extends RowReaderFactory[(K, V)] { 12 | 13 | override def rowReader(table: TableDef, columnSelection: IndexedSeq[ColumnRef]): RowReader[(K, V)] = { 14 | val keyReader = keyRRF.rowReader(table, keySelection.selectFrom(table)) 15 | val valueReader = valueRRF.rowReader(table, columnSelection) 16 | new KeyValueRowReader(keyReader, valueReader) 17 | } 18 | 19 | override def targetClass: Class[(K, V)] = classOf[(K, V)] 20 | } 21 | 22 | private[connector] class KeyValueRowReader[K, V](keyReader: RowReader[K], valueReader: RowReader[V]) 23 | extends RowReader[(K, V)] { 24 | 25 | override def neededColumns: Option[Seq[ColumnRef]] = 26 | (for (keyNames <- keyReader.neededColumns; valueNames <- valueReader.neededColumns) yield keyNames ++ valueNames) 27 | .orElse(keyReader.neededColumns).orElse(valueReader.neededColumns) 28 | 29 | override def read(row: Row, columnNames: Array[String])(implicit protocolVersion: ProtocolVersion): (K, V) = { 30 | (keyReader.read(row, columnNames), valueReader.read(row, columnNames)) 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /spark-cassandra-connector-demos/twitter-streaming/src/main/scala/com/datastax/spark/connector/demo/TwitterStreamingTopicsByInterval.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.demo 2 | 3 | import scala.util.matching.Regex 4 | import org.apache.spark.storage.StorageLevel 5 | import org.apache.spark.streaming.{Time, Seconds, StreamingContext} 6 | import org.apache.spark.streaming.twitter.TwitterUtils 7 | import org.joda.time.{DateTimeZone, DateTime} 8 | import twitter4j.auth.Authorization 9 | import com.datastax.spark.connector.streaming._ 10 | import com.datastax.spark.connector.SomeColumns 11 | 12 | class TwitterStreamingTopicsByInterval extends Serializable { 13 | 14 | def start(auth: Option[Authorization], ssc: StreamingContext, topics: Set[String], keyspace: String, table: String): Unit = { 15 | 16 | val stream = TwitterUtils.createStream(ssc, auth, Nil, StorageLevel.MEMORY_ONLY_SER_2) 17 | 18 | /** Note that Cassandra is doing the sorting for you here. */ 19 | stream.flatMap(_.getText.toLowerCase.split("""\s+""")) 20 | .filter(topics.contains(_)) 21 | .countByValueAndWindow(Seconds(5), Seconds(5)) 22 | .transform((rdd, time) => rdd.map { case (term, count) => (term, count, now(time))}) 23 | .saveToCassandra(keyspace, table, SomeColumns("topic", "mentions", "interval")) 24 | 25 | ssc.checkpoint("./checkpoint") 26 | ssc.start() 27 | ssc.awaitTermination() 28 | } 29 | 30 | private def now(time: Time): String = 31 | new DateTime(time.milliseconds, DateTimeZone.UTC).toString("yyyyMMddHH:mm:ss.SSS") 32 | } 33 | 34 | 35 | -------------------------------------------------------------------------------- /doc/10_embedded.md: -------------------------------------------------------------------------------- 1 | # Documentation 2 | ## The `spark-cassandra-connector-embedded` Artifact 3 | The `spark-cassandra-connector-embedded` artifact can be used as a test or prototype dependency to spin up embedded servers for testing ideas, quickly learning, integration, etc. 4 | Pulling this dependency in allows you to do 5 | 6 | - Integration Tests (IT) tests with an embedded Cassandra instance 7 | - if your sbt project is configured to [run IT configs](https://github.com/datastax/spark-cassandra-connector/blob/master/project/Settings.scala#L78-L94) 8 | - Easily write and run a Spark Streaming app using 9 | - Apache Kafka streams (including an embedded Zookeeper), all with no Ops work involved 10 | - Twitter streams (needs the 4 auth credentials required by twitter) 11 | - And of course Cassandra but you currently need to sping up a local instance: [Download Cassandra latest](http://cassandra.apache.org/download/), open the tar, and run `sudo ./apache-cassandra-2.1.0/bin/cassandra` 12 | 13 | ## The Code 14 | See: [https://github.com/datastax/spark-cassandra-connector/tree/master/spark-cassandra-connector-embedded/src/main/scala/com/datastax/spark/connector/embedded](https://github.com/datastax/spark-cassandra-connector/tree/master/spark-cassandra-connector-embedded/src/main/scala/com/datastax/spark/connector/embedded) 15 | 16 | ## How To Add The Dependency 17 | Simply add this to your SBT build, or in the appropriate format for a Maven build 18 | 19 | "com.datastax.spark" %% "spark-cassandra-connector-embedded" % {latest.verson} 20 | 21 | [Next - Performance Monitoring](11_metrics.md) 22 | -------------------------------------------------------------------------------- /spark-cassandra-connector-embedded/scala-2.10/src/main/scala/com/datastax/spark/connector/embedded/SparkRepl.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.embedded 2 | 3 | import java.io.{PrintWriter, StringWriter, StringReader, BufferedReader} 4 | import java.net.URLClassLoader 5 | 6 | import scala.collection.mutable.ArrayBuffer 7 | import org.apache.spark.repl.SparkILoop 8 | 9 | trait SparkRepl { 10 | 11 | def runInterpreter(master: String, input: String): String = { 12 | System.setProperty("spark.cassandra.connection.host", EmbeddedCassandra.getHost(0).getHostAddress) 13 | val in = new BufferedReader(new StringReader(input + "\n")) 14 | val out = new StringWriter() 15 | val cl = getClass.getClassLoader 16 | var paths = new ArrayBuffer[String] 17 | cl match { 18 | case urlLoader: URLClassLoader => 19 | for (url <- urlLoader.getURLs) { 20 | if (url.getProtocol == "file") { 21 | paths += url.getFile 22 | } 23 | } 24 | case _ => 25 | } 26 | 27 | val interp = new SparkILoop(in, new PrintWriter(out), master) 28 | org.apache.spark.repl.Main.interp = interp 29 | val separator = System.getProperty("path.separator") 30 | interp.process(Array("-classpath", paths.mkString(separator))) 31 | org.apache.spark.repl.Main.interp = null 32 | if (interp.sparkContext != null) { 33 | interp.sparkContext.stop() 34 | } 35 | // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown 36 | System.clearProperty("spark.driver.port") 37 | out.toString 38 | } 39 | 40 | } 41 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/scala/org/apache/spark/sql/cassandra/ConsolidateSettingsSpec.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.cassandra 2 | 3 | import org.scalatest.{Matchers, FlatSpec} 4 | import org.apache.spark.SparkConf 5 | import org.apache.spark.sql.cassandra.CassandraSourceRelation._ 6 | 7 | class ConsolidateSettingsSpec extends FlatSpec with Matchers { 8 | 9 | it should "consolidate Cassandra conf settings in order of " + 10 | "table level -> keyspace -> cluster -> default" in { 11 | val tableRef = TableRef("table", "keyspace", Option("cluster")) 12 | val sparkConf = new SparkConf 13 | val rowSize = "spark.cassandra.input.fetch.size_in_rows" 14 | 15 | sparkConf.set(rowSize, "10") 16 | val consolidatedConf = 17 | consolidateConfs(sparkConf, Map.empty, tableRef, Map.empty) 18 | consolidatedConf.get(rowSize) shouldBe "10" 19 | 20 | val sqlConf = Map[String, String](s"cluster/$rowSize" -> "100") 21 | val confWithClusterLevelSettings = 22 | consolidateConfs(sparkConf, sqlConf, tableRef, Map.empty) 23 | confWithClusterLevelSettings.get(rowSize) shouldBe "100" 24 | 25 | val sqlConf1 = sqlConf + (s"cluster:keyspace/$rowSize" -> "200") 26 | val confWithKeyspaceLevelSettings = 27 | consolidateConfs(sparkConf, sqlConf1, tableRef, Map.empty) 28 | confWithKeyspaceLevelSettings.get(rowSize) shouldBe "200" 29 | 30 | val tableConf = Map(rowSize -> "1000") 31 | val confWithTableLevelSettings = 32 | consolidateConfs(sparkConf, sqlConf1, tableRef, tableConf) 33 | confWithTableLevelSettings.get(rowSize) shouldBe "1000" 34 | } 35 | } -------------------------------------------------------------------------------- /spark-cassandra-connector-demos/simple-demos/src/main/scala/com/datastax/spark/connector/demo/BasicReadWriteDemo.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.demo 2 | 3 | import com.datastax.spark.connector.cql.CassandraConnector 4 | 5 | object BasicReadWriteDemo extends DemoApp { 6 | 7 | CassandraConnector(conf).withSessionDo { session => 8 | session.execute("CREATE KEYSPACE IF NOT EXISTS test WITH REPLICATION = {'class': 'SimpleStrategy', 'replication_factor': 1 }") 9 | session.execute("CREATE TABLE IF NOT EXISTS test.key_value (key INT PRIMARY KEY, value VARCHAR)") 10 | session.execute("TRUNCATE test.key_value") 11 | session.execute("INSERT INTO test.key_value(key, value) VALUES (1, 'first row')") 12 | session.execute("INSERT INTO test.key_value(key, value) VALUES (2, 'second row')") 13 | session.execute("INSERT INTO test.key_value(key, value) VALUES (3, 'third row')") 14 | } 15 | 16 | import com.datastax.spark.connector._ 17 | 18 | // Read table test.kv and print its contents: 19 | val rdd = sc.cassandraTable("test", "key_value").select("key", "value") 20 | rdd.collect().foreach(row => log.info(s"Existing Data: $row")) 21 | 22 | // Write two new rows to the test.kv table: 23 | val col = sc.parallelize(Seq((4, "fourth row"), (5, "fifth row"))) 24 | col.saveToCassandra("test", "key_value", SomeColumns("key", "value")) 25 | 26 | // Assert the two new rows were stored in test.kv table: 27 | assert(col.collect().length == 2) 28 | 29 | col.collect().foreach(row => log.info(s"New Data: $row")) 30 | log.info(s"Work completed, stopping the Spark context.") 31 | sc.stop() 32 | } 33 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/GettableByIndexData.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector 2 | 3 | import com.datastax.spark.connector.types.TypeConverter.StringConverter 4 | 5 | trait GettableByIndexData extends Serializable { 6 | 7 | def columnValues: IndexedSeq[AnyRef] 8 | 9 | /** Returns a column value by index without applying any conversion. 10 | * The underlying type is the same as the type returned by the low-level Cassandra driver, 11 | * is implementation defined and may change in the future. 12 | * Cassandra nulls are returned as Scala nulls. */ 13 | def getRaw(index: Int): AnyRef = columnValues(index) 14 | 15 | /** Total number of columns in this row. Includes columns with null values. */ 16 | def length = columnValues.size 17 | 18 | /** Total number of columns in this row. Includes columns with null values. */ 19 | def size = columnValues.size 20 | 21 | /** Returns true if column value is Cassandra null */ 22 | def isNullAt(index: Int): Boolean = 23 | columnValues(index) == null 24 | 25 | /** Displays the content in human readable form, including the names and values of the columns */ 26 | def dataAsString: String = 27 | columnValues 28 | .map(StringConverter.convert) 29 | .mkString("(", ", ", ")") 30 | 31 | override def toString = 32 | dataAsString 33 | 34 | override def equals(o: Any): Boolean = o match { 35 | case that: GettableByIndexData if this.columnValues == that.columnValues => true 36 | case _ => false 37 | } 38 | 39 | override def hashCode: Int = 40 | columnValues.hashCode() 41 | } 42 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/cql/MultipleRetryPolicy.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.cql 2 | 3 | import com.datastax.driver.core.policies.RetryPolicy 4 | import com.datastax.driver.core.policies.RetryPolicy.RetryDecision 5 | import com.datastax.driver.core.{ConsistencyLevel, Statement, WriteType} 6 | 7 | /** Always retries with the same CL, constant number of times, regardless of circumstances */ 8 | class MultipleRetryPolicy(maxRetryCount: Int, retryDelay: CassandraConnectorConf.RetryDelayConf) 9 | extends RetryPolicy { 10 | 11 | private def retryOrThrow(cl: ConsistencyLevel, nbRetry: Int): RetryDecision = { 12 | if (nbRetry < maxRetryCount) { 13 | if (nbRetry > 0) { 14 | val delay = retryDelay.forRetry(nbRetry).toMillis 15 | if (delay > 0) Thread.sleep(delay) 16 | } 17 | RetryDecision.retry(cl) 18 | } else { 19 | RetryDecision.rethrow() 20 | } 21 | } 22 | 23 | override def onReadTimeout(stmt: Statement, cl: ConsistencyLevel, 24 | requiredResponses: Int, receivedResponses: Int, 25 | dataRetrieved: Boolean, nbRetry: Int) = retryOrThrow(cl, nbRetry) 26 | 27 | override def onUnavailable(stmt: Statement, cl: ConsistencyLevel, 28 | requiredReplica: Int, aliveReplica: Int, nbRetry: Int) = retryOrThrow(cl, nbRetry) 29 | 30 | override def onWriteTimeout(stmt: Statement, cl: ConsistencyLevel, writeType: WriteType, 31 | requiredAcks: Int, receivedAcks: Int, nbRetry: Int) = retryOrThrow(cl, nbRetry) 32 | 33 | } 34 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/ColumnSelector.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector 2 | 3 | import scala.language.implicitConversions 4 | 5 | import com.datastax.spark.connector.cql.TableDef 6 | 7 | sealed trait ColumnSelector { 8 | def aliases: Map[String, String] 9 | def selectFrom(table: TableDef): IndexedSeq[ColumnRef] 10 | } 11 | 12 | case object AllColumns extends ColumnSelector { 13 | override def aliases: Map[String, String] = Map.empty.withDefault(x => x) 14 | override def selectFrom(table: TableDef) = 15 | table.columns.map(_.ref) 16 | } 17 | 18 | case object PartitionKeyColumns extends ColumnSelector { 19 | override def aliases: Map[String, String] = Map.empty.withDefault(x => x) 20 | override def selectFrom(table: TableDef) = 21 | table.partitionKey.map(_.ref).toIndexedSeq 22 | } 23 | 24 | case class SomeColumns(columns: ColumnRef*) extends ColumnSelector { 25 | 26 | override def aliases: Map[String, String] = columns.map { 27 | case ref => (ref.selectedAs, ref.cqlValueName) 28 | }.toMap 29 | 30 | override def selectFrom(table: TableDef): IndexedSeq[ColumnRef] = { 31 | val missing = table.missingColumns(columns).filterNot(_ == RowCountRef) 32 | if (missing.nonEmpty) throw new NoSuchElementException( 33 | s"Columns not found in table ${table.name}: ${missing.mkString(", ")}") 34 | 35 | columns.toIndexedSeq 36 | } 37 | } 38 | 39 | object SomeColumns { 40 | @deprecated("Use com.datastax.spark.connector.rdd.SomeColumns instead of Seq", "1.0") 41 | implicit def seqToSomeColumns(columns: Seq[String]): SomeColumns = 42 | SomeColumns(columns.map(x => x: ColumnRef): _*) 43 | } 44 | 45 | 46 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/rdd/reader/ValueRowReader.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd.reader 2 | 3 | import com.datastax.driver.core.{ProtocolVersion, Row} 4 | import com.datastax.spark.connector._ 5 | import com.datastax.spark.connector.cql.TableDef 6 | import com.datastax.spark.connector.types.TypeConverter 7 | import com.datastax.spark.connector.util.JavaApiHelper 8 | 9 | class ValueRowReader[T: TypeConverter](columnRef: ColumnRef) extends RowReader[T] { 10 | 11 | private val converter = implicitly[TypeConverter[T]] 12 | 13 | /** Reads column values from low-level `Row` and turns them into higher level representation. 14 | * @param row row fetched from Cassandra 15 | * @param columnNames column names available in the `row` */ 16 | override def read(row: Row, columnNames: Array[String])(implicit protocolVersion: ProtocolVersion): T = 17 | converter.convert(GettableData.get(row, columnRef.cqlValueName)) 18 | 19 | /** List of columns this `RowReader` is going to read. 20 | * Useful to avoid fetching the columns that are not needed. */ 21 | override def neededColumns: Option[Seq[ColumnRef]] = 22 | Some(Seq(columnRef)) 23 | 24 | } 25 | 26 | class ValueRowReaderFactory[T: TypeConverter] 27 | extends RowReaderFactory[T] { 28 | 29 | override def rowReader(table: TableDef, selectedColumns: IndexedSeq[ColumnRef]): RowReader[T] = { 30 | require(selectedColumns.nonEmpty, "ValueRowReader requires a non-empty column selection") 31 | new ValueRowReader[T](selectedColumns.head) 32 | } 33 | 34 | override def targetClass: Class[T] = JavaApiHelper.getRuntimeClass(implicitly[TypeConverter[T]].targetTypeTag) 35 | } 36 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/it/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | 17 | # for production, you should probably set pattern to %c instead of %l. 18 | # (%l is slower.) 19 | 20 | # output messages into a rolling log file as well as stdout 21 | log4j.rootLogger=WARN,stdout 22 | 23 | # stdout 24 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 25 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 26 | log4j.appender.stdout.layout.ConversionPattern=%5p %d{HH:mm:ss,SSS} %C (%F:%L) - %m%n 27 | 28 | # Avoid "no host ID found" when starting a fresh node 29 | log4j.logger.org.apache.cassandra.db.SystemKeyspace=ERROR 30 | 31 | # Avoid "address already in use" when starting multiple local Spark masters 32 | log4j.logger.org.eclipse.jetty.util.component.AbstractLifeCycle=ERROR 33 | 34 | # Show spark sql predicate pushdown 35 | log4j.category.org.apache.spark.sql.cassandra=INFO 36 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/writer/RowWriterFactory.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.writer 2 | 3 | import scala.reflect.runtime.universe._ 4 | 5 | import com.datastax.spark.connector.ColumnRef 6 | import com.datastax.spark.connector.cql.TableDef 7 | import com.datastax.spark.connector.mapper.ColumnMapper 8 | 9 | /** Creates instances of [[RowWriter]] objects for the given row type `T`. 10 | * `RowWriterFactory` is the trait you need to implement if you want to support row representations 11 | * which cannot be simply mapped by a [[com.datastax.spark.connector.mapper.ColumnMapper ColumnMapper]]. */ 12 | trait RowWriterFactory[T] { 13 | 14 | /** Creates a new `RowWriter` instance. 15 | * @param table target table the user wants to write into 16 | * @param selectedColumns columns selected by the user; the user might wish to write only a 17 | * subset of columns */ 18 | def rowWriter(table: TableDef, selectedColumns: IndexedSeq[ColumnRef]): RowWriter[T] 19 | } 20 | 21 | /** Provides a low-priority implicit `RowWriterFactory` able to write objects of any class for which 22 | * a [[com.datastax.spark.connector.mapper.ColumnMapper ColumnMapper]] is defined. */ 23 | trait LowPriorityRowWriterFactoryImplicits { 24 | implicit def defaultRowWriterFactory[T : TypeTag : ColumnMapper]: RowWriterFactory[T] = 25 | DefaultRowWriter.factory 26 | } 27 | 28 | /** Provides an implicit `RowWriterFactory` for saving [[com.datastax.spark.connector.CassandraRow CassandraRow]] objects. */ 29 | object RowWriterFactory extends LowPriorityRowWriterFactoryImplicits { 30 | implicit val genericRowWriterFactory = CassandraRowWriter.Factory 31 | } -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/mapper/ColumnMap.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.mapper 2 | 3 | import com.datastax.spark.connector.ColumnRef 4 | 5 | /** A column map for saving objects to Cassandra. 6 | * Lists available getters. */ 7 | trait ColumnMapForWriting extends Serializable { 8 | /** Maps a getter method name to a column reference */ 9 | def getters: Map[String, ColumnRef] 10 | } 11 | 12 | /** A column map for reading objects from Cassandra. 13 | * Describes object's constructor and setters. */ 14 | trait ColumnMapForReading extends Serializable { 15 | 16 | /** A sequence of column references associated with parameters of the main constructor. 17 | * If the class contains multiple constructors, the main constructor is assumed to be the one with the 18 | * highest number of parameters. Multiple constructors with the same number of parameters are not allowed. */ 19 | def constructor: Seq[ColumnRef] 20 | 21 | /** Maps a setter method name to a column reference */ 22 | def setters: Map[String, ColumnRef] 23 | 24 | /** Whether Java nulls are allowed to be directly set as object properties. 25 | * This is desired for compatibility with Java classes, but in Scala, we have Option, 26 | * therefore we need to fail fast if one wants to assign a Cassandra null 27 | * value to a non-optional property. */ 28 | def allowsNull: Boolean 29 | } 30 | 31 | case class SimpleColumnMapForWriting(getters: Map[String, ColumnRef]) extends ColumnMapForWriting 32 | 33 | case class SimpleColumnMapForReading( 34 | constructor: Seq[ColumnRef], 35 | setters: Map[String, ColumnRef], 36 | allowsNull: Boolean = false) extends ColumnMapForReading 37 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/types/TimestampParser.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.types 2 | 3 | import java.util.Date 4 | 5 | import org.joda.time.format.DateTimeFormat 6 | 7 | import scala.util.{Success, Try} 8 | 9 | /** Parses CQL timestamps. 10 | * 11 | * Supported formats: 12 | * - `yyyy-MM-dd HH:mm` 13 | * - `yyyy-MM-dd HH:mmZ` 14 | * - `yyyy-MM-dd HH:mm:ss` 15 | * - `yyyy-MM-dd HH:mm:ssZ` 16 | * - `yyyy-MM-dd HH:mm:ss.SSS` 17 | * - `yyyy-MM-dd HH:mm:ss.SSSZ` 18 | * - `yyyy-MM-dd'T'HH:mm` 19 | * - `yyyy-MM-dd'T'HH:mmZ` 20 | * - `yyyy-MM-dd'T'HH:mm:ss` 21 | * - `yyyy-MM-dd'T'HH:mm:ssZ` 22 | * - `yyyy-MM-dd'T'HH:mm:ss.SSS` 23 | * - `yyyy-MM-dd'T'HH:mm:ss.SSSZ` 24 | * - `yyyy-MM-dd` 25 | * - `yyyy-MM-ddZ` 26 | */ 27 | object TimestampParser { 28 | private val dateStringPatterns = Array[String]( 29 | "yyyy-MM-dd HH:mm", 30 | "yyyy-MM-dd HH:mmZ", 31 | "yyyy-MM-dd HH:mm:ss", 32 | "yyyy-MM-dd HH:mm:ssZ", 33 | "yyyy-MM-dd HH:mm:ss.SSS", 34 | "yyyy-MM-dd HH:mm:ss.SSSZ", 35 | "yyyy-MM-dd'T'HH:mm", 36 | "yyyy-MM-dd'T'HH:mmZ", 37 | "yyyy-MM-dd'T'HH:mm:ss", 38 | "yyyy-MM-dd'T'HH:mm:ssZ", 39 | "yyyy-MM-dd'T'HH:mm:ss.SSS", 40 | "yyyy-MM-dd'T'HH:mm:ss.SSSZ", 41 | "yyyy-MM-dd", 42 | "yyyy-MM-ddZ") 43 | 44 | private val parsers = 45 | dateStringPatterns.map(DateTimeFormat.forPattern) 46 | 47 | def parse(date: String): Date = { 48 | parsers.view.map(p => Try(p.parseDateTime(date))).find(_.isSuccess) match { 49 | case Some(Success(d)) => d.toDate 50 | case _ => throw new IllegalArgumentException(s"Invalid date: $date") 51 | } 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/perf/scala/com/datastax/spark/connector/writer/BasicWriteBenchmark.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.writer 2 | 3 | import com.datastax.spark.connector._ 4 | import com.datastax.spark.connector.cql.CassandraConnector 5 | import com.datastax.spark.connector.util.BenchmarkUtil 6 | import org.apache.spark.{SparkConf, SparkContext} 7 | 8 | object BasicWriteBenchmark extends App { 9 | 10 | val cassandraPartitionsToWrite = 100000 11 | val rowsPerCassandraPartition = 10 12 | val rowsPerSparkPartition = 50000 13 | 14 | val conf = new SparkConf(true) 15 | .setAppName("Write performance test") 16 | .set("spark.cassandra.connection.host", "localhost") 17 | .set("spark.cassandra.output.concurrent.writes", "16") 18 | .set("spark.cassandra.output.batch.size.bytes", "4096") 19 | .set("spark.cassandra.output.batch.grouping.key", "partition") 20 | .setMaster("local[1]") 21 | 22 | val sc = new SparkContext(conf) 23 | val conn = CassandraConnector(conf) 24 | 25 | conn.withSessionDo { session => 26 | session.execute("CREATE KEYSPACE IF NOT EXISTS benchmarks WITH REPLICATION = {'class': 'SimpleStrategy', 'replication_factor': 1 }") 27 | session.execute("CREATE TABLE IF NOT EXISTS benchmarks.write_benchmark (p INT, c INT, value VARCHAR, PRIMARY KEY(p, c))") 28 | } 29 | 30 | BenchmarkUtil.printTime { 31 | val col = 32 | for (i <- 1 to cassandraPartitionsToWrite; j <- 1 to rowsPerCassandraPartition) 33 | yield (i, j, "data:" + i + ":" + j) 34 | val rdd = sc.parallelize(col, cassandraPartitionsToWrite * rowsPerCassandraPartition / rowsPerSparkPartition) 35 | rdd.saveToCassandra("benchmarks", "write_benchmark") 36 | } 37 | 38 | sc.stop() 39 | } 40 | -------------------------------------------------------------------------------- /spark-cassandra-connector-demos/simple-demos/src/main/scala/com/datastax/spark/connector/demo/SQLDemo.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.demo 2 | 3 | import com.datastax.spark.connector.cql.CassandraConnector 4 | import org.apache.spark.sql.cassandra.CassandraSQLContext 5 | 6 | /** This demo creates a table in Cassandra, populates it with sample data, 7 | * then queries it using SparkSQL and finally displays the query results to the standard output. 8 | * You need to start Cassandra on local node prior to executing this demo. */ 9 | object SQLDemo extends DemoApp { 10 | 11 | val cc = new CassandraSQLContext(sc) 12 | 13 | CassandraConnector(conf).withSessionDo { session => 14 | session.execute("CREATE KEYSPACE IF NOT EXISTS test WITH REPLICATION = {'class': 'SimpleStrategy', 'replication_factor': 1 }") 15 | session.execute("DROP TABLE IF EXISTS test.sql_demo") 16 | session.execute("CREATE TABLE test.sql_demo (key INT PRIMARY KEY, grp INT, value DOUBLE)") 17 | session.execute("INSERT INTO test.sql_demo(key, grp, value) VALUES (1, 1, 1.0)") 18 | session.execute("INSERT INTO test.sql_demo(key, grp, value) VALUES (2, 1, 2.5)") 19 | session.execute("INSERT INTO test.sql_demo(key, grp, value) VALUES (3, 1, 10.0)") 20 | session.execute("INSERT INTO test.sql_demo(key, grp, value) VALUES (4, 2, 4.0)") 21 | session.execute("INSERT INTO test.sql_demo(key, grp, value) VALUES (5, 2, 2.2)") 22 | session.execute("INSERT INTO test.sql_demo(key, grp, value) VALUES (6, 2, 2.8)") 23 | } 24 | 25 | cc.setKeyspace("test") 26 | val rdd = cc.cassandraSql("SELECT grp, max(value) AS mv FROM sql_demo GROUP BY grp ORDER BY mv") 27 | rdd.collect().foreach(println) // [2, 4.0] [1, 10.0] 28 | 29 | sc.stop() 30 | } 31 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/RandomPartitionerTokenRangeSplitter.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd.partitioner 2 | 3 | import com.datastax.spark.connector.rdd.partitioner.dht.{BigIntToken, TokenFactory, TokenRange} 4 | 5 | 6 | /** Fast token range splitter assuming that data are spread out evenly in the whole range. 7 | * @param dataSize estimate of the size of the data in the whole ring */ 8 | class RandomPartitionerTokenRangeSplitter(dataSize: Long) 9 | extends TokenRangeSplitter[BigInt, BigIntToken] { 10 | 11 | private val tokenFactory = 12 | TokenFactory.RandomPartitionerTokenFactory 13 | 14 | private def wrap(token: BigInt): BigInt = { 15 | val max = tokenFactory.maxToken.value 16 | if (token <= max) token else token - max 17 | } 18 | 19 | private type TR = TokenRange[BigInt, BigIntToken] 20 | 21 | /** Splits the token range uniformly into sub-ranges. 22 | * @param splitSize requested sub-split size, given in the same units as `dataSize` */ 23 | def split(range: TR, splitSize: Long): Seq[TR] = { 24 | val rangeSize = range.dataSize 25 | val rangeTokenCount = tokenFactory.distance(range.start, range.end) 26 | val n = math.max(1, math.round(rangeSize.toDouble / splitSize)).toInt 27 | 28 | val left = range.start.value 29 | val right = range.end.value 30 | val splitPoints = 31 | (for (i <- 0 until n) yield wrap(left + (rangeTokenCount * i / n))) :+ right 32 | 33 | for (Seq(l, r) <- splitPoints.sliding(2).toSeq) yield 34 | new TokenRange[BigInt, BigIntToken]( 35 | new BigIntToken(l.bigInteger), 36 | new BigIntToken(r.bigInteger), 37 | range.replicas, 38 | rangeSize / n) 39 | } 40 | } -------------------------------------------------------------------------------- /spark-cassandra-connector-embedded/scala-2.11/src/main/scala/com/datastax/spark/connector/embedded/SparkRepl.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.embedded 2 | 3 | import java.io.{PrintWriter, StringWriter, StringReader, BufferedReader} 4 | import java.net.URLClassLoader 5 | 6 | import scala.collection.mutable.ArrayBuffer 7 | import scala.tools.nsc.interpreter.SparkILoop 8 | 9 | trait SparkRepl { 10 | 11 | def runInterpreter(master: String, input: String): String = { 12 | System.setProperty("spark.master", master) 13 | System.setProperty("spark.cassandra.connection.host", EmbeddedCassandra.getHost(0).getHostAddress) 14 | 15 | val in = new BufferedReader(new StringReader(input + "\n")) 16 | val out = new StringWriter() 17 | val cl = getClass.getClassLoader 18 | var paths = new ArrayBuffer[String] 19 | cl match { 20 | case urlLoader: URLClassLoader => 21 | for (url <- urlLoader.getURLs) { 22 | if (url.getProtocol == "file") { 23 | paths += url.getFile 24 | } 25 | } 26 | case _ => 27 | } 28 | 29 | val interp = new SparkILoop(Some(in), new PrintWriter(out)) 30 | org.apache.spark.repl.Main.interp = interp 31 | val separator = System.getProperty("path.separator") 32 | org.apache.spark.repl.Main.s.processArguments(List("-classpath", paths.mkString(separator)), true) 33 | // 2.10 interp.process(Array("-classpath", paths.mkString(separator))) 34 | org.apache.spark.repl.Main.interp = null 35 | Option(org.apache.spark.repl.Main.sparkContext).map(_.stop()) 36 | // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown 37 | System.clearProperty("spark.driver.port") 38 | out.toString 39 | } 40 | 41 | } -------------------------------------------------------------------------------- /spark-cassandra-connector-demos/twitter-streaming/src/main/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | 17 | # for production, you should probably set pattern to %c instead of %l. 18 | # (%l is slower.) 19 | 20 | # output messages into a rolling log file as well as stdout 21 | log4j.rootLogger=INFO,stdout 22 | 23 | # stdout 24 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 25 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 26 | log4j.appender.stdout.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n 27 | 28 | # Avoid "no host ID found" when starting a fresh node 29 | log4j.logger.org.apache.cassandra.db.SystemKeyspace=ERROR 30 | 31 | # If running spark local, ignore block input exists warnings, which are expected. 32 | log4j.logger.org.apache.spark.storage.BlockManager=ERROR 33 | log4j.logger.com.datastax.spark.connector=INFO 34 | log4j.logger.org.apache.spark=WARN 35 | log4j.logger.com.datastax.driver.core=WARN -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/util/MagicalTypeTricks.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.util 2 | 3 | import scala.language.implicitConversions 4 | 5 | object MagicalTypeTricks { 6 | 7 | trait DoesntHaveImplicit[A, B] 8 | implicit def doesntHaveImplicit[A, B]: A DoesntHaveImplicit B = null 9 | implicit def doesntHaveImplicitAmbiguity1[A, B](implicit ev: B): A DoesntHaveImplicit B = null 10 | implicit def doesntHaveImplicitAmbiguity2[A, B](implicit ev: B): A DoesntHaveImplicit B = null 11 | 12 | trait IsNotEqualTo[A, B] 13 | implicit def neq[A, B]: A IsNotEqualTo B = null 14 | implicit def neqAmbiguity1[A]: A IsNotEqualTo A = null 15 | implicit def neqAmbiguity2[A]: A IsNotEqualTo A = null 16 | 17 | trait IsNotSubclassOf[A, B] 18 | implicit def nsub[A, B]: A IsNotSubclassOf B = null 19 | implicit def nsubAmbiguity1[A, B >: A]: A IsNotSubclassOf B = null 20 | implicit def nsubAmbiguity2[A, B >: A]: A IsNotSubclassOf B = null 21 | 22 | type ¬[A] = A => Nothing 23 | type λ[A] = ¬[¬[A]] 24 | 25 | /** 26 | * Example of how disjunction can be used: 27 | * {{{ 28 | * import com.datastax.spark.connector.util.MagicalTypeTricks._ 29 | * 30 | * def function[T](t: T)(implicit ev: (λ[T] <:< (Int ∪ String))) = { 31 | * println("t = " + t) 32 | * } 33 | * 34 | * function(5) // t = 5 35 | * function("five") // t = five 36 | * function(5d) // error: Cannot prove that 37 | * // (Double => Nothing) => Nothing <:< Int => Nothing with String => Nothing => Nothing. 38 | * }}} 39 | * 40 | * Based on [[http://www.chuusai.com/2011/06/09/scala-union-types-curry-howard/ this article]]. 41 | */ 42 | type ∪[T, U] = ¬[¬[T] with ¬[U]] 43 | 44 | } 45 | -------------------------------------------------------------------------------- /spark-cassandra-connector-java/src/main/java/com/datastax/spark/connector/japi/DStreamJavaFunctions.java: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.japi; 2 | 3 | import com.datastax.spark.connector.ColumnSelector; 4 | import com.datastax.spark.connector.cql.CassandraConnector; 5 | import com.datastax.spark.connector.streaming.DStreamFunctions; 6 | import com.datastax.spark.connector.writer.RowWriterFactory; 7 | import com.datastax.spark.connector.writer.WriteConf; 8 | import org.apache.spark.SparkConf; 9 | import org.apache.spark.streaming.dstream.DStream; 10 | 11 | /** 12 | * A Java API wrapper over {@link DStream} to provide Spark Cassandra Connector functionality. 13 | *

14 | * To obtain an instance of this wrapper, use one of the factory methods in {@link CassandraJavaUtil} 15 | * class. 16 | */ 17 | @SuppressWarnings("UnusedDeclaration") 18 | public class DStreamJavaFunctions extends RDDAndDStreamCommonJavaFunctions { 19 | public final DStream dstream; 20 | private final DStreamFunctions dsf; 21 | 22 | DStreamJavaFunctions(DStream dStream) { 23 | this.dstream = dStream; 24 | this.dsf = new DStreamFunctions<>(dStream); 25 | } 26 | 27 | @Override 28 | public CassandraConnector defaultConnector() { 29 | return dsf.connector(); 30 | } 31 | 32 | @Override 33 | protected SparkConf getConf() { 34 | return dstream.ssc().conf(); 35 | } 36 | 37 | @Override 38 | protected void saveToCassandra(String keyspace, String table, RowWriterFactory rowWriterFactory, 39 | ColumnSelector columnNames, WriteConf conf, CassandraConnector connector) { 40 | dsf.saveToCassandra(keyspace, table, columnNames, conf, connector, rowWriterFactory); 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/it/scala/com/datastax/spark/connector/cql/CassandraAuthenticatedConnectorSpec.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.cql 2 | 3 | import com.datastax.spark.connector.embedded.SparkTemplate._ 4 | import org.apache.spark.SparkConf 5 | 6 | import com.datastax.spark.connector.SparkCassandraITFlatSpecBase 7 | import com.datastax.spark.connector.embedded.EmbeddedCassandra 8 | 9 | class CassandraAuthenticatedConnectorSpec extends SparkCassandraITFlatSpecBase { 10 | 11 | useCassandraConfig(Seq("cassandra-password-auth.yaml.template")) 12 | 13 | // Wait for the default user to be created in Cassandra. 14 | Thread.sleep(1000) 15 | 16 | val conf = defaultConf 17 | conf.set(DefaultAuthConfFactory.CassandraUserNameProperty, "cassandra") 18 | conf.set(DefaultAuthConfFactory.CassandraPasswordProperty, "cassandra") 19 | 20 | "A CassandraConnector" should "authenticate with username and password when using native protocol" in { 21 | val conn2 = CassandraConnector(conf) 22 | conn2.withSessionDo { session => 23 | assert(session !== null) 24 | assert(session.isClosed === false) 25 | assert(session.getCluster.getMetadata.getClusterName != null) 26 | } 27 | } 28 | 29 | it should "pick up user and password from SparkConf" in { 30 | val host = EmbeddedCassandra.getHost(0).getHostAddress 31 | val conf = defaultConf 32 | .set(CassandraConnectorConf.CassandraConnectionHostProperty, host) 33 | .set(DefaultAuthConfFactory.CassandraUserNameProperty, "cassandra") 34 | .set(DefaultAuthConfFactory.CassandraPasswordProperty, "cassandra") 35 | 36 | // would throw exception if connection unsuccessful 37 | val conn2 = CassandraConnector(conf) 38 | conn2.withSessionDo { session => } 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/scala/com/datastax/spark/connector/writer/AsyncExecutorTest.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.writer 2 | 3 | import java.util.concurrent.atomic.AtomicInteger 4 | import java.util.concurrent.{Callable, Executors} 5 | 6 | import com.google.common.util.concurrent.MoreExecutors 7 | import org.junit.Assert._ 8 | import org.junit.Test 9 | 10 | class AsyncExecutorTest { 11 | 12 | @Test 13 | def test() { 14 | val taskCount = 20 15 | val maxParallel = 5 16 | 17 | val currentlyRunningCounter = new AtomicInteger(0) 18 | val maxParallelCounter = new AtomicInteger(0) 19 | val totalFinishedExecutionsCounter = new AtomicInteger(0) 20 | 21 | val task = new Callable[String] { 22 | override def call() = { 23 | val c = currentlyRunningCounter.incrementAndGet() 24 | var m = maxParallelCounter.get() 25 | while (m < c && !maxParallelCounter.compareAndSet(m, c)) 26 | m = maxParallelCounter.get() 27 | Thread.sleep(100) 28 | currentlyRunningCounter.decrementAndGet() 29 | totalFinishedExecutionsCounter.incrementAndGet() 30 | "ok" 31 | } 32 | } 33 | 34 | val underlyingExecutor = MoreExecutors.listeningDecorator(Executors.newCachedThreadPool()) 35 | val asyncExecutor = new AsyncExecutor[Callable[String], String](underlyingExecutor.submit(_: Callable[String]), maxParallel, None, None) 36 | 37 | for (i <- 1 to taskCount) 38 | asyncExecutor.executeAsync(task) 39 | 40 | asyncExecutor.waitForCurrentlyExecutingTasks() 41 | assertEquals(maxParallel, maxParallelCounter.get()) 42 | assertEquals(taskCount, totalFinishedExecutionsCounter.get()) 43 | assertEquals(true, asyncExecutor.successful) 44 | } 45 | 46 | 47 | 48 | 49 | } 50 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/scala/com/datastax/spark/connector/rdd/partitioner/dht/Murmur3TokenFactorySpec.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd.partitioner.dht 2 | 3 | import org.scalatest.{FlatSpec, Matchers} 4 | 5 | class Murmur3TokenFactorySpec extends FlatSpec with Matchers { 6 | 7 | val factory = TokenFactory.Murmur3TokenFactory 8 | 9 | "Murmur3TokenFactory" should "create a token from String" in { 10 | factory.tokenFromString("0") shouldBe LongToken(0L) 11 | factory.tokenFromString("-1") shouldBe LongToken(-1L) 12 | factory.tokenFromString(Long.MaxValue.toString) shouldBe factory.maxToken 13 | factory.tokenFromString(Long.MinValue.toString) shouldBe factory.minToken 14 | } 15 | 16 | it should "create a String representation of a token" in { 17 | factory.tokenToString(LongToken(0L)) shouldBe "0" 18 | factory.tokenToString(LongToken(-1L)) shouldBe "-1" 19 | factory.tokenToString(factory.maxToken) shouldBe Long.MaxValue.toString 20 | factory.tokenToString(factory.minToken) shouldBe Long.MinValue.toString 21 | } 22 | 23 | it should "calculate the distance between tokens if right > left" in { 24 | factory.distance(LongToken(0L), LongToken(1L)) shouldBe BigInt(1) 25 | } 26 | 27 | it should "calculate the distance between tokens if right <= left" in { 28 | factory.distance(LongToken(0L), LongToken(0L)) shouldBe factory.totalTokenCount 29 | factory.distance(factory.maxToken, factory.minToken) shouldBe BigInt(0) 30 | } 31 | 32 | it should "calculate ring fraction" in { 33 | factory.ringFraction(LongToken(0L), LongToken(0L)) shouldBe 1.0 34 | factory.ringFraction(LongToken(0L), factory.maxToken) shouldBe 0.5 35 | factory.ringFraction(factory.maxToken, LongToken(0L)) shouldBe 0.5 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/scala/com/datastax/spark/connector/ColumnSelectorSpec.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector 2 | 3 | import org.scalatest.{Matchers, WordSpec} 4 | 5 | import com.datastax.spark.connector.cql._ 6 | import com.datastax.spark.connector.types.{TimestampType, VarCharType, IntType} 7 | 8 | class ColumnSelectorSpec extends WordSpec with Matchers { 9 | "A ColumnSelector#selectFrom method" should { 10 | val column1 = ColumnDef("c1", PartitionKeyColumn, IntType) 11 | val column2 = ColumnDef("c2", PartitionKeyColumn, VarCharType) 12 | val column3 = ColumnDef("c3", ClusteringColumn(0), VarCharType) 13 | val column4 = ColumnDef("c4", ClusteringColumn(1), VarCharType) 14 | val column5 = ColumnDef("c5", RegularColumn, VarCharType) 15 | val column6 = ColumnDef("c6", RegularColumn, TimestampType) 16 | 17 | val tableDef = TableDef("keyspace", "table", Seq(column1, column2), Seq(column3, column4), Seq(column5, column6)) 18 | 19 | "return all columns" in { 20 | val columns = AllColumns.selectFrom(tableDef) 21 | columns should equal(tableDef.columns.map(_.ref)) 22 | } 23 | 24 | "return partition key columns" in { 25 | val columns = PartitionKeyColumns.selectFrom(tableDef) 26 | columns should equal(tableDef.partitionKey.map(_.ref)) 27 | } 28 | 29 | "return some columns" in { 30 | val columns = SomeColumns("c1", "c3", "c5").selectFrom(tableDef) 31 | columns.map(_.columnName) should be equals Seq("c1", "c3", "c5") 32 | } 33 | 34 | "throw a NoSuchElementException when selected column name is invalid" in { 35 | a[NoSuchElementException] should be thrownBy { 36 | SomeColumns("c1", "c3", "unknown_column").selectFrom(tableDef) 37 | } 38 | } 39 | 40 | } 41 | 42 | } 43 | -------------------------------------------------------------------------------- /spark-cassandra-connector-demos/simple-demos/src/main/resources/data/words: -------------------------------------------------------------------------------- 1 | Kitty lorem ipsum inspect anything brought into the house stick sit in box so loves cheeseburgers 2 | Run in circles shake treat bag hide when guests come over use lap as chair Sit in box throwup on your pillow purr while eating 3 | but present belly scratch hand when pet Chase dog then run away burrow under covers scratch hand when pet burrow under covers 4 | intrigued by the shower why must they do that Curl into a furry donut need to chase tail burrow under covers 5 | so swat at dog or sleep in the bathroom sink inspect anything brought into the house Hide when guests come over Under the bed 6 | I like big cats and i can not lie leave dead animals as gifts Curl into a furry donut shake treat bag hunt by meowing loudly at 5am 7 | next to human slave food dispenser need to chase tail and chew iPad power cord 8 | Hack up furballs hunt anything that moves but favor packaging over toy yet stand in front of the computer screen 9 | Favor packaging over toy throwup on your pillow who's the baby Give attitude Purr while eating chew iPad power cord hopped up on catnip 10 | so always hungry Stare at ceiling kick up litter or hunt by meowing loudly at 5am next to human slave food dispenser or stretch 11 | yet under the bed claw drapes Intently stare at the same spot make muffins but intently stare at the same spot 12 | or kick up litter and why must they do that Missing until dinner time hopped up on catnip who's the baby 13 | Sleep on keyboard favor packaging over toy why must they do that but bathe private parts with tongue then lick owner's face 14 | Purr for no reason Scamper sweet beast but mark territory and stand in front of the computer screen favor packaging over toy 15 | Sleep on desk infront of laptop sit on keyboard push mouse jump on bed and make bird sound at 3am -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/rdd/reader/PrefetchingResultSetIterator.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd.reader 2 | 3 | import java.util.concurrent.TimeUnit 4 | 5 | import com.codahale.metrics.Timer 6 | import com.datastax.driver.core.{Row, ResultSet} 7 | import com.google.common.util.concurrent.{ListenableFuture, FutureCallback, Futures} 8 | 9 | /** Allows to efficiently iterate over a large, paged ResultSet, 10 | * asynchronously prefetching the next page. 11 | * 12 | * @param resultSet result set obtained from the Java driver 13 | * @param prefetchWindowSize if there are less than this rows available without blocking, 14 | * initiates fetching the next page 15 | * @param timer a Codahale timer to optionally gather the metrics of fetching time 16 | */ 17 | class PrefetchingResultSetIterator(resultSet: ResultSet, prefetchWindowSize: Int, timer: Option[Timer] = None) extends Iterator[Row] { 18 | 19 | private[this] val iterator = resultSet.iterator() 20 | 21 | override def hasNext = iterator.hasNext 22 | 23 | private[this] def maybePrefetch(): Unit = { 24 | if (!resultSet.isFullyFetched && resultSet.getAvailableWithoutFetching < prefetchWindowSize) { 25 | val t0 = System.nanoTime() 26 | val future: ListenableFuture[Void] = resultSet.fetchMoreResults() 27 | if (timer.isDefined) 28 | Futures.addCallback(future, new FutureCallback[Void] { 29 | override def onSuccess(ignored: Void): Unit = { 30 | timer.get.update(System.nanoTime() - t0, TimeUnit.NANOSECONDS) 31 | } 32 | 33 | override def onFailure(ignored: Throwable): Unit = { } 34 | }) 35 | } 36 | } 37 | 38 | override def next() = { 39 | maybePrefetch() 40 | iterator.next() 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /spark-cassandra-connector-demos/kafka-streaming/src/main/resources/data/words: -------------------------------------------------------------------------------- 1 | Kitty lorem ipsum inspect anything brought into the house stick sit in box so loves cheeseburgers 2 | Run in circles shake treat bag hide when guests come over use lap as chair Sit in box throwup on your pillow purr while eating 3 | but present belly scratch hand when pet Chase dog then run away burrow under covers scratch hand when pet burrow under covers 4 | intrigued by the shower why must they do that Curl into a furry donut need to chase tail burrow under covers 5 | so swat at dog or sleep in the bathroom sink inspect anything brought into the house Hide when guests come over Under the bed 6 | I like big cats and i can not lie leave dead animals as gifts Curl into a furry donut shake treat bag hunt by meowing loudly at 5am 7 | next to human slave food dispenser need to chase tail and chew iPad power cord 8 | Hack up furballs hunt anything that moves but favor packaging over toy yet stand in front of the computer screen 9 | Favor packaging over toy throwup on your pillow who's the baby Give attitude Purr while eating chew iPad power cord hopped up on catnip 10 | so always hungry Stare at ceiling kick up litter or hunt by meowing loudly at 5am next to human slave food dispenser or stretch 11 | yet under the bed claw drapes Intently stare at the same spot make muffins but intently stare at the same spot 12 | or kick up litter and why must they do that Missing until dinner time hopped up on catnip who's the baby 13 | Sleep on keyboard favor packaging over toy why must they do that but bathe private parts with tongue then lick owner's face 14 | Purr for no reason Scamper sweet beast but mark territory and stand in front of the computer screen favor packaging over toy 15 | Sleep on desk infront of laptop sit on keyboard push mouse jump on bed and make bird sound at 3am -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/util/BufferedIterator2.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.util 2 | 3 | import scala.collection.mutable.ArrayBuffer 4 | 5 | /** Serves the same purpose as `BufferedIterator` in Scala, but its `takeWhile` method 6 | * properly doesn't consume the next element. */ 7 | class BufferedIterator2[T](iterator: Iterator[T]) extends Iterator[T] { 8 | 9 | // Instead of a pair of T and Boolean we could use Option, but 10 | // this would allocate a new Option object per each item, which might be 11 | // too big overhead in tight loops using this iterator. 12 | private[this] var headDefined: Boolean = false 13 | private[this] var headElement: T = advance() 14 | 15 | def head = 16 | if (headDefined) headElement 17 | else throw new NoSuchElementException("Head of empty iterator") 18 | 19 | def headOption = headElement 20 | 21 | private def advance(): T = { 22 | if (iterator.hasNext) { 23 | headDefined = true 24 | iterator.next() 25 | } 26 | else { 27 | headDefined = false 28 | null.asInstanceOf[T] 29 | } 30 | } 31 | 32 | override def hasNext = headDefined 33 | 34 | override def next() = { 35 | val result = head 36 | headElement = advance() 37 | result 38 | } 39 | 40 | 41 | override def takeWhile(p: T => Boolean): Iterator[T] = { 42 | new Iterator[T]() { 43 | override def hasNext = headDefined && p(headElement) 44 | override def next() = 45 | if (hasNext) BufferedIterator2.this.next() 46 | else throw new NoSuchElementException 47 | } 48 | } 49 | 50 | def appendWhile(p: (T) => Boolean, target: ArrayBuffer[T]): Unit = { 51 | while (headDefined && p(headElement)) { 52 | target += headElement 53 | headElement = advance() 54 | } 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /spark-cassandra-connector-java/src/main/java/com/datastax/spark/connector/japi/PairRDDJavaFunctions.java: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.japi; 2 | 3 | import com.datastax.spark.connector.PairRDDFunctions; 4 | import com.datastax.spark.connector.util.JavaApiHelper; 5 | import org.apache.spark.api.java.JavaPairRDD; 6 | import org.apache.spark.rdd.RDD; 7 | import scala.Tuple2; 8 | import scala.collection.Seq; 9 | import scala.reflect.ClassTag; 10 | 11 | import java.util.Collection; 12 | 13 | import static com.datastax.spark.connector.japi.CassandraJavaUtil.classTag; 14 | 15 | public class PairRDDJavaFunctions extends RDDJavaFunctions> { 16 | 17 | public final PairRDDFunctions pairRDDFunctions; 18 | 19 | public PairRDDJavaFunctions(RDD> rdd) { 20 | super(rdd); 21 | pairRDDFunctions = new PairRDDFunctions<>(rdd); 22 | } 23 | 24 | /** 25 | * Groups items with the same key, assuming the items with the same key are next to each other in the 26 | * collection. It does not perform shuffle, therefore it is much faster than using much more 27 | * universal Spark RDD `groupByKey`. For this method to be useful with Cassandra tables, the key must 28 | * represent a prefix of the primary key, containing at least the partition key of the Cassandra 29 | * table. 30 | */ 31 | public JavaPairRDD> spanByKey(ClassTag keyClassTag) { 32 | ClassTag>> tupleClassTag = classTag(Tuple2.class); 33 | ClassTag> vClassTag = classTag(Collection.class); 34 | RDD>> newRDD = pairRDDFunctions.spanByKey() 35 | .map(JavaApiHelper.>valuesAsJavaCollection(), tupleClassTag); 36 | 37 | return new JavaPairRDD<>(newRDD, keyClassTag, vClassTag); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/scala/com/datastax/spark/connector/rdd/partitioner/dht/RandomPartitionerTokenFactorySpec.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd.partitioner.dht 2 | 3 | import org.scalatest.{FlatSpec, Matchers} 4 | 5 | class RandomPartitionerTokenFactorySpec extends FlatSpec with Matchers { 6 | 7 | val factory = TokenFactory.RandomPartitionerTokenFactory 8 | 9 | "RandomPartitionerTokenFactory" should "create a token from String" in { 10 | factory.tokenFromString("0") shouldBe BigIntToken(0L) 11 | factory.tokenFromString("-1") shouldBe BigIntToken(-1L) 12 | factory.tokenFromString("170141183460469231731687303715884105728") shouldBe 13 | BigIntToken(BigInt("170141183460469231731687303715884105728")) 14 | } 15 | 16 | it should "create a String representation of a token" in { 17 | factory.tokenToString(BigIntToken(0)) shouldBe "0" 18 | factory.tokenToString(BigIntToken(-1)) shouldBe "-1" 19 | factory.tokenToString(factory.maxToken) shouldBe "170141183460469231731687303715884105728" 20 | } 21 | 22 | it should "calculate the distance between tokens if right > left" in { 23 | factory.distance(BigIntToken(0), BigIntToken(1)) shouldBe BigInt(1) 24 | } 25 | 26 | it should "calculate the distance between tokens if right <= left" in { 27 | factory.distance(BigIntToken(0), BigIntToken(0)) shouldBe factory.totalTokenCount 28 | factory.distance(factory.maxToken, factory.minToken) shouldBe BigInt(0) 29 | } 30 | 31 | it should "calculate ring fraction" in { 32 | factory.ringFraction(BigIntToken(0L), BigIntToken(0L)) shouldBe 1.0 33 | factory.ringFraction(BigIntToken(0L), factory.maxToken) shouldBe 1.0 34 | factory.ringFraction(factory.maxToken, factory.minToken) shouldBe 0.0 35 | factory.ringFraction(BigIntToken(0L), BigIntToken(factory.maxToken.value / 2)) shouldBe 0.5 36 | } 37 | 38 | } 39 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/it/scala/com/datastax/spark/connector/streaming/StreamingSpec.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.streaming 2 | 3 | import com.datastax.spark.connector.testkit._ 4 | import com.datastax.spark.connector.embedded._ 5 | import org.scalatest.{ConfigMap, BeforeAndAfterAll} 6 | 7 | /** 8 | * Usages: Create the [[org.apache.spark.streaming.StreamingContext]] then write async to the stream. 9 | * 10 | * val ssc = new StreamingContext(conf, Milliseconds(500)) 11 | * 12 | * Akka 13 | * {{{ 14 | * val stream = ssc.actorStream[String](Props[SimpleActor], actorName, StorageLevel.MEMORY_AND_DISK) 15 | * }}} 16 | * 17 | * On upgrade examples: 18 | * Kafka 19 | * {{{ 20 | * val stream: ReceiverInputDStream[(String, String)] = 21 | * KafkaUtils.createStream(ssc, kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2) 22 | * }}} 23 | * 24 | * ZeroMQ 25 | * {{{ 26 | * val stream: ReceiverInputDStream[String] = ZeroMQUtils.createStream(ssc, publishUrl, subscribe, bytesToObjects) 27 | * }}} 28 | * 29 | * Twitter 30 | * {{{ 31 | * val stream: ReceiverInputDStream[Status] = TwitterUtils.createStream(ssc, None) 32 | * }}} 33 | * 34 | * etc. 35 | */ 36 | trait StreamingSpec extends AbstractSpec with SharedEmbeddedCassandra with SparkTemplate with BeforeAndAfterAll { 37 | import org.apache.spark.streaming.StreamingContext 38 | import scala.concurrent.duration._ 39 | 40 | val duration = 10.seconds 41 | 42 | useCassandraConfig(Seq("cassandra-default.yaml.template")) 43 | 44 | def ssc: StreamingContext 45 | 46 | after { 47 | // Spark Context is shared among all integration test so we don't want to stop it here 48 | ssc.stop(stopSparkContext = false, stopGracefully = true) 49 | } 50 | 51 | override def afterAll(configMap: ConfigMap) { 52 | if (ssc.sparkContext != null) { 53 | ssc.sparkContext.stop() 54 | } 55 | } 56 | 57 | } -------------------------------------------------------------------------------- /spark-cassandra-connector-java/src/main/java/com/datastax/spark/connector/japi/GenericJavaRowReaderFactory.java: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.japi; 2 | 3 | import com.datastax.driver.core.ProtocolVersion; 4 | import com.datastax.driver.core.Row; 5 | import com.datastax.spark.connector.ColumnRef; 6 | import com.datastax.spark.connector.cql.TableDef; 7 | import com.datastax.spark.connector.rdd.reader.RowReader; 8 | import com.datastax.spark.connector.rdd.reader.RowReaderFactory; 9 | import scala.Option; 10 | import scala.collection.IndexedSeq; 11 | import scala.collection.Seq; 12 | 13 | public class GenericJavaRowReaderFactory { 14 | public final static RowReaderFactory instance = new RowReaderFactory() { 15 | 16 | @Override 17 | public RowReader rowReader(TableDef table, IndexedSeq selectedColumns) { 18 | return JavaRowReader.instance; 19 | } 20 | 21 | @Override 22 | public Class targetClass() { 23 | return CassandraRow.class; 24 | } 25 | }; 26 | 27 | 28 | public static class JavaRowReader implements RowReader { 29 | public final static JavaRowReader instance = new JavaRowReader(); 30 | 31 | private JavaRowReader() { 32 | } 33 | 34 | @Override 35 | public CassandraRow read(Row row, String[] columnNames, ProtocolVersion protocolVersion) { 36 | assert row.getColumnDefinitions().size() == columnNames.length : 37 | "Number of columns in a row must match the number of columns in the table metadata"; 38 | return CassandraRow$.MODULE$.fromJavaDriverRow(row, columnNames, protocolVersion); 39 | } 40 | 41 | @Override 42 | public Option> neededColumns() { 43 | return Option.empty(); 44 | } 45 | } 46 | 47 | } 48 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/CassandraPartitionedRDD.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd.partitioner 2 | 3 | import org.apache.spark.rdd.RDD 4 | import org.apache.spark.{Partition, Partitioner, TaskContext} 5 | 6 | import scala.reflect.ClassTag 7 | 8 | /** 9 | * RDD created by repartitionByCassandraReplica with preferred locations mapping to the CassandraReplicas 10 | * each partition was created for. 11 | */ 12 | class CassandraPartitionedRDD[T]( 13 | prev: RDD[T], 14 | keyspace:String, 15 | table:String)( 16 | implicit 17 | ct: ClassTag[T]) 18 | extends RDD[T](prev) { 19 | 20 | // We aren't going to change the data 21 | override def compute(split: Partition, context: TaskContext): Iterator[T] = 22 | prev.iterator(split, context) 23 | 24 | @transient 25 | override val partitioner: Option[Partitioner] = prev.partitioner 26 | 27 | private val replicaPartitioner: ReplicaPartitioner = 28 | partitioner match { 29 | case Some(rp: ReplicaPartitioner) => rp 30 | case _ => throw new IllegalArgumentException("CassandraPartitionedRDD hasn't been " + 31 | "partitioned by ReplicaPartitioner. Unable to do any work with data locality.") 32 | } 33 | 34 | private lazy val nodeAddresses = new NodeAddresses(replicaPartitioner.connector) 35 | 36 | override def getPartitions: Array[Partition] = 37 | prev.partitions.map(partition => replicaPartitioner.getEndpointPartition(partition)) 38 | 39 | override def getPreferredLocations(split: Partition): Seq[String] = split match { 40 | case epp: ReplicaPartition => 41 | epp.endpoints.flatMap(nodeAddresses.hostNames).toSeq 42 | case other: Partition => throw new IllegalArgumentException( 43 | "CassandraPartitionedRDD doesn't have Endpointed Partitions. PrefferedLocations cannot be" + 44 | "deterimined") 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/scala/com/datastax/spark/connector/util/SpanningIteratorSpec.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.util 2 | 3 | import org.scalatest.{FlatSpec, Matchers} 4 | 5 | class SpanningIteratorSpec extends FlatSpec with Matchers { 6 | 7 | "SpanningIterator" should "group an empty collection" in { 8 | new SpanningIterator[Int, Int](Iterator.empty, identity).isEmpty shouldBe true 9 | } 10 | 11 | it should "group a sequence of elements with the same key into a single item and should preserve order" in { 12 | val collection = Seq(1, 2, 3, 4, 5) 13 | val grouped = new SpanningIterator(collection.iterator, (_: Int) => 0).toSeq 14 | grouped should have length 1 15 | grouped.head._2 should contain inOrder(1, 2, 3, 4, 5) 16 | } 17 | 18 | it should "group a sequence of elements with distinct keys the same number of groups" in { 19 | val collection = Seq(1, 2, 3, 4, 5) 20 | val grouped = new SpanningIterator(collection.iterator, identity[Int]).toSeq 21 | grouped should have length 5 22 | grouped.distinct should have length 5 // to check if something wasn't included more than once 23 | } 24 | 25 | it should "group a sequence of elements with two keys into two groups" in { 26 | val collection = Seq(1 -> 10, 1 -> 11, 1 -> 12, 2 -> 20, 2 -> 21) 27 | val grouped = new SpanningIterator(collection.iterator, (x: (Int, Int)) => x._1).toIndexedSeq 28 | grouped should have length 2 29 | grouped(0)._1 should be(1) 30 | grouped(0)._2 should contain inOrder(1 -> 10, 1 -> 11, 1 -> 12) 31 | grouped(1)._1 should be(2) 32 | grouped(1)._2 should contain inOrder(2 -> 20, 2 -> 21) 33 | } 34 | 35 | it should "be lazy and work with infinite streams" in { 36 | val stream = Stream.from(0) 37 | val grouped = new SpanningIterator(stream.iterator, identity[Int]) 38 | grouped.take(5).toSeq.map(_._1) should contain inOrder(0, 1, 2, 3, 4) 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/it/scala/com/datastax/spark/connector/util/MultiThreadedSpec.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.util 2 | 3 | import com.datastax.spark.connector.embedded.SparkTemplate._ 4 | import com.datastax.spark.connector.{SparkCassandraITFlatSpecBase, _} 5 | import com.datastax.spark.connector.cql.CassandraConnector 6 | import com.datastax.spark.connector.embedded.EmbeddedCassandra 7 | import org.scalatest.concurrent.AsyncAssertions 8 | 9 | 10 | class MultiThreadedSpec extends SparkCassandraITFlatSpecBase with AsyncAssertions{ 11 | 12 | useCassandraConfig(Seq("cassandra-default.yaml.template")) 13 | useSparkConf(defaultSparkConf) 14 | 15 | val conn = CassandraConnector(defaultConf) 16 | val count = 10000 17 | 18 | val ks = "multi_threaded" 19 | val tab = "mt_test" 20 | 21 | conn.withSessionDo { session => 22 | session.execute(s"CREATE KEYSPACE IF NOT EXISTS $ks WITH replication = {'class': " + 23 | "'SimpleStrategy', 'replication_factor': 1 }") 24 | session.execute(s"CREATE TABLE $ks.$tab (pkey int PRIMARY KEY, value varchar)") 25 | 26 | for (i <- 1 to count) { 27 | session.execute( 28 | s"INSERT INTO $ks.$tab (pkey, value) VALUES (?, ?)", 29 | i: java.lang.Integer, "value " + i) 30 | } 31 | } 32 | 33 | "A Spark Context " should " be able to read a Cassandra table in different threads" in { 34 | 35 | val w = new Waiter 36 | 37 | val threads = for (theadnum <- 1 to 10) yield new Thread(new Runnable { 38 | def run() { 39 | val rdd = sc.cassandraTable[(Int, String)](ks, tab) 40 | val result = rdd.collect 41 | w { result should have size (count) } 42 | w.dismiss() 43 | } 44 | }) 45 | for (thread <- threads) thread.start() 46 | import org.scalatest.time.SpanSugar._ 47 | 48 | w.await(timeout(10 seconds), dismissals(10)) 49 | for (thread <- threads) thread.join() 50 | } 51 | 52 | } 53 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/NodeAddresses.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd.partitioner 2 | 3 | import java.net.InetAddress 4 | 5 | import scala.collection.JavaConversions._ 6 | 7 | import com.datastax.spark.connector.cql.CassandraConnector 8 | 9 | /** Looks up listen address of a cluster node given its RPC address. 10 | * Uses system.peers table as the source of information. 11 | * If such information for a node is missing, it assumes its listen 12 | * address equals its RPC address */ 13 | class NodeAddresses(conn: CassandraConnector) extends Serializable { 14 | 15 | /** Maps rpc addresses to listen addresses for every cluster node. 16 | * If rpc address is not known, returns the same address. */ 17 | lazy val rpcToListenAddress: InetAddress => InetAddress = { 18 | conn.withSessionDo { session => 19 | val table = "system.peers" 20 | val rpcAddressColumn = "rpc_address" 21 | val listenAddressColumn = "peer" 22 | 23 | // TODO: fetch information about the local node from system.local, when CASSANDRA-9436 is done 24 | val rs = session.execute(s"SELECT $rpcAddressColumn, $listenAddressColumn FROM $table") 25 | for { 26 | row <- rs.all() 27 | rpcAddress <- Option(row.getInet(rpcAddressColumn)) 28 | listenAddress = row.getInet(listenAddressColumn) 29 | } yield (rpcAddress, listenAddress) 30 | }.toMap.withDefault(identity) 31 | } 32 | 33 | /** Returns a list of IP-addresses and host names that identify a node. 34 | * Useful for giving Spark the list of preferred nodes for the Spark partition. */ 35 | def hostNames(rpcAddress: InetAddress): Set[String] = { 36 | val listenAddress = rpcToListenAddress(rpcAddress) 37 | Set( 38 | rpcAddress.getHostAddress, 39 | rpcAddress.getHostName, 40 | listenAddress.getHostAddress, 41 | listenAddress.getHostName 42 | ) 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/writer/ObjectSizeEstimator.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.writer 2 | 3 | import java.io.{OutputStream, ObjectOutputStream} 4 | import java.nio.ByteBuffer 5 | 6 | import scala.collection.JavaConversions._ 7 | 8 | import com.datastax.spark.connector.util.ByteBufferUtil 9 | 10 | 11 | /** Estimates amount of memory required to serialize Java/Scala objects */ 12 | object ObjectSizeEstimator { 13 | 14 | private def makeSerializable(obj: Any): AnyRef = { 15 | obj match { 16 | case bb: ByteBuffer => ByteBufferUtil.toArray(bb) 17 | case list: java.util.List[_] => list.map(makeSerializable) 18 | case list: List[_] => list.map(makeSerializable) 19 | case set: java.util.Set[_] => set.map(makeSerializable) 20 | case set: Set[_] => set.map(makeSerializable) 21 | case map: java.util.Map[_, _] => map.map { case (k, v) => (makeSerializable(k), makeSerializable(v)) } 22 | case map: Map[_, _] => map.map { case (k, v) => (makeSerializable(k), makeSerializable(v)) } 23 | case other => other.asInstanceOf[AnyRef] 24 | } 25 | } 26 | 27 | /** Records only how many bytes were written but the actual data is discarded */ 28 | private class CountingOutputStream extends OutputStream { 29 | private var _length = 0 30 | override def write(b: Int) = _length += 1 31 | override def write(b: Array[Byte]) = _length += b.length 32 | override def write(b: Array[Byte], off: Int, len: Int) = _length += len 33 | def length = _length 34 | } 35 | 36 | /** Serializes passed objects and reports their total size */ 37 | def measureSerializedSize(objects: Seq[Any]): Int = { 38 | val countingStream = new CountingOutputStream 39 | val objectStream = new ObjectOutputStream(countingStream) 40 | for (obj <- objects) 41 | objectStream.writeObject(makeSerializable(obj)) 42 | objectStream.close() 43 | countingStream.length 44 | } 45 | 46 | } 47 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/cql/PreparedStatementCache.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.cql 2 | 3 | import com.datastax.driver.core.{RegularStatement, Session, Cluster, PreparedStatement} 4 | import org.apache.spark.Logging 5 | 6 | import scala.collection.concurrent.TrieMap 7 | 8 | /** Caches prepared statements so they are not prepared 9 | * multiple times by different threads. */ 10 | object PreparedStatementCache extends Logging { 11 | 12 | private val clusterCache = 13 | TrieMap[Cluster, TrieMap[String, PreparedStatement]]() 14 | 15 | private def get(cluster: Cluster, query: String): Option[PreparedStatement] = 16 | for (statementCache <- clusterCache.get(cluster); 17 | statement <- statementCache.get(query)) yield statement 18 | 19 | private def put(cluster: Cluster, query: String, statement: PreparedStatement): PreparedStatement = { 20 | clusterCache.get(cluster) match { 21 | case Some(statementCache) => statementCache.put(query, statement) 22 | case None => clusterCache.put(cluster, TrieMap(query -> statement)) 23 | } 24 | statement 25 | } 26 | 27 | /** Removes all statements associated with the `Cluster` from the cache. */ 28 | def remove(cluster: Cluster) { 29 | synchronized { 30 | clusterCache.remove(cluster) 31 | } 32 | } 33 | 34 | /** Retrieves a `PreparedStatement` from cache or 35 | * creates a new one if not found and updates the cache. */ 36 | def prepareStatement(session: Session, query: RegularStatement): PreparedStatement = { 37 | val cluster = session.getCluster 38 | get(cluster, query.toString) match { 39 | case Some(stmt) => stmt 40 | case None => 41 | synchronized { 42 | get(cluster, query.toString) match { 43 | case Some(stmt) => stmt 44 | case None => 45 | val stmt = session.prepare(query) 46 | put(cluster, query.toString, stmt) 47 | } 48 | } 49 | } 50 | } 51 | 52 | } 53 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/util/Symbols.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.util 2 | 3 | import scala.collection.immutable.{TreeMap, TreeSet} 4 | import scala.reflect.runtime.universe._ 5 | 6 | import org.apache.commons.lang3.tuple 7 | 8 | object Symbols { 9 | val OptionSymbol = typeOf[Option[Any]].asInstanceOf[TypeRef].sym 10 | val ListSymbol = typeOf[List[Any]].asInstanceOf[TypeRef].sym 11 | val VectorSymbol = typeOf[Vector[Any]].asInstanceOf[TypeRef].sym 12 | val SetSymbol = typeOf[Set[Any]].asInstanceOf[TypeRef].sym 13 | val TreeSetSymbol = typeOf[TreeSet[Any]].asInstanceOf[TypeRef].sym 14 | val SeqSymbol = typeOf[Seq[Any]].asInstanceOf[TypeRef].sym 15 | val IndexedSeqSymbol = typeOf[IndexedSeq[Any]].asInstanceOf[TypeRef].sym 16 | val IterableSymbol = typeOf[Iterable[Any]].asInstanceOf[TypeRef].sym 17 | val MapSymbol = typeOf[Map[Any, Any]].asInstanceOf[TypeRef].sym 18 | val TreeMapSymbol = typeOf[TreeMap[Any, Any]].asInstanceOf[TypeRef].sym 19 | 20 | val JavaListSymbol = typeOf[java.util.List[Any]].asInstanceOf[TypeRef].sym 21 | val JavaArrayListSymbol = typeOf[java.util.ArrayList[Any]].asInstanceOf[TypeRef].sym 22 | val JavaSetSymbol = typeOf[java.util.Set[Any]].asInstanceOf[TypeRef].sym 23 | val JavaHashSetSymbol = typeOf[java.util.HashSet[Any]].asInstanceOf[TypeRef].sym 24 | val JavaMapSymbol = typeOf[java.util.Map[Any, Any]].asInstanceOf[TypeRef].sym 25 | val JavaHashMapSymbol = typeOf[java.util.HashMap[Any, Any]].asInstanceOf[TypeRef].sym 26 | 27 | val PairSymbol = typeOf[tuple.Pair[Any, Any]].asInstanceOf[TypeRef].sym 28 | val TripleSymbol = typeOf[tuple.Triple[Any, Any, Any]].asInstanceOf[TypeRef].sym 29 | 30 | val ListSymbols = Set( 31 | ListSymbol, VectorSymbol, SeqSymbol, IndexedSeqSymbol, IterableSymbol, 32 | JavaListSymbol, JavaArrayListSymbol) 33 | val SetSymbols = Set(SetSymbol, TreeSetSymbol, JavaSetSymbol, JavaHashSetSymbol) 34 | val MapSymbols = Set(MapSymbol, TreeMapSymbol, JavaMapSymbol, JavaHashMapSymbol) 35 | } 36 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/types/CollectionColumnType.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.types 2 | 3 | import scala.language.existentials 4 | import scala.reflect.runtime.universe._ 5 | 6 | trait CollectionColumnType[T] extends ColumnType[T] { 7 | def isCollection = true 8 | } 9 | 10 | case class ListType[T](elemType: ColumnType[T]) extends CollectionColumnType[Vector[T]] { 11 | 12 | @transient 13 | lazy val scalaTypeTag = TypeTag.synchronized { 14 | implicit val elemTypeTag = elemType.scalaTypeTag 15 | implicitly[TypeTag[Vector[T]]] 16 | } 17 | 18 | def cqlTypeName = s"list<${elemType.cqlTypeName}>" 19 | 20 | override def converterToCassandra: TypeConverter[_ <: AnyRef] = 21 | new TypeConverter.OptionToNullConverter(TypeConverter.listConverter(elemType.converterToCassandra)) 22 | } 23 | 24 | case class SetType[T](elemType: ColumnType[T]) extends CollectionColumnType[Set[T]] { 25 | 26 | @transient 27 | lazy val scalaTypeTag = TypeTag.synchronized { 28 | implicit val elemTypeTag = elemType.scalaTypeTag 29 | implicitly[TypeTag[Set[T]]] 30 | } 31 | 32 | def cqlTypeName = s"set<${elemType.cqlTypeName}>" 33 | 34 | override def converterToCassandra: TypeConverter[_ <: AnyRef] = 35 | new TypeConverter.OptionToNullConverter(TypeConverter.setConverter(elemType.converterToCassandra)) 36 | } 37 | 38 | case class MapType[K, V](keyType: ColumnType[K], valueType: ColumnType[V]) 39 | extends CollectionColumnType[Map[K, V]] { 40 | 41 | @transient 42 | lazy val scalaTypeTag = TypeTag.synchronized { 43 | implicit val keyTypeTag = keyType.scalaTypeTag 44 | implicit val valueTypeTag = valueType.scalaTypeTag 45 | implicitly[TypeTag[Map[K, V]]] 46 | } 47 | 48 | def cqlTypeName = s"map<${keyType.cqlTypeName}, ${valueType.cqlTypeName}>" 49 | 50 | override def converterToCassandra: TypeConverter[_ <: AnyRef] = 51 | new TypeConverter.OptionToNullConverter( 52 | TypeConverter.mapConverter(keyType.converterToCassandra, valueType.converterToCassandra)) 53 | } 54 | 55 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/org/apache/spark/sql/cassandra/types/UUIDType.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.cassandra.types 2 | 3 | import scala.reflect.runtime.universe.typeTag 4 | 5 | import java.util.UUID 6 | 7 | import org.apache.spark.annotation.DeveloperApi 8 | import org.apache.spark.sql.catalyst.ScalaReflectionLock 9 | import org.apache.spark.sql.types.AtomicType 10 | 11 | /** 12 | * :: DeveloperApi :: 13 | * 14 | * The data type representing `UUID` values. 15 | * 16 | * @group dataType 17 | */ 18 | @DeveloperApi 19 | class UUIDType private() extends AtomicType { 20 | // The companion object and this class is separated so the companion object also subclasses 21 | // this type. Otherwise, the companion object would be of type "UUIDType$" in byte code. 22 | // Defined with a private constructor so the companion object is the only possible instantiation. 23 | private[sql] type InternalType = UUID 24 | @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } 25 | 26 | // Because this new type is not a Spark internal supported data type, there is no 27 | // CAST method to convert a UUID to a String. Spark internally converts it 28 | // to Double, so it errors out the query such as 29 | // select * from table where uuid_column = '123e4567-e89b-12d3-a456-426655440000' 30 | // The correct query should be 31 | // select * from table where CAST(uuid_column as string) = '123e4567-e89b-12d3-a456-426655440000' 32 | // 33 | // The following code provides only a reference ordering implementation. But it will 34 | // never be called by Spark, for there is no CAST function or UDF to convert String 35 | // to UUID 36 | // 37 | private[sql] val ordering = new Ordering[InternalType] { 38 | def compare(x: UUID, y: UUID) = x.compareTo(y) 39 | } 40 | /** 41 | * The default size of a value of the UUIDType is 16 bytes. 42 | */ 43 | override def defaultSize: Int = 16 44 | 45 | private[spark] override def asNullable: UUIDType = this 46 | } 47 | 48 | case object UUIDType extends UUIDType -------------------------------------------------------------------------------- /spark-cassandra-connector-java/src/main/scala/com/datastax/spark/connector/japi/CassandraRow.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.japi 2 | 3 | import com.datastax.driver.core.{ProtocolVersion, Row} 4 | import com.datastax.spark.connector.GettableData 5 | 6 | final class CassandraRow(val columnNames: IndexedSeq[String], val columnValues: IndexedSeq[AnyRef]) 7 | extends JavaGettableData with Serializable { 8 | 9 | private[spark] def this() = this(null: IndexedSeq[String], null) // required by Kryo for deserialization :( 10 | 11 | def this(columnNames: Array[String], columnValues: Array[AnyRef]) = 12 | this(columnNames.toIndexedSeq, columnValues.toIndexedSeq) 13 | 14 | protected def fieldNames = columnNames 15 | protected def fieldValues = columnValues 16 | 17 | def iterator = columnValues.iterator 18 | override def toString = "CassandraRow" + dataAsString 19 | } 20 | 21 | 22 | object CassandraRow { 23 | 24 | /** Deserializes first n columns from the given `Row` and returns them as 25 | * a `CassandraRow` object. The number of columns retrieved is determined by the length 26 | * of the columnNames argument. The columnNames argument is used as metadata for 27 | * the newly created `CassandraRow`, but it is not used to fetch data from 28 | * the input `Row` in order to improve performance. Fetching column values by name is much 29 | * slower than fetching by index. */ 30 | def fromJavaDriverRow(row: Row, columnNames: Array[String])(implicit protocolVersion: ProtocolVersion): CassandraRow = { 31 | val data = new Array[Object](columnNames.length) 32 | for (i <- columnNames.indices) 33 | data(i) = GettableData.get(row, i) 34 | new CassandraRow(columnNames, data) 35 | } 36 | 37 | /** Creates a CassandraRow object from a map with keys denoting column names and 38 | * values denoting column values. */ 39 | def fromMap(map: Map[String, Any]): CassandraRow = { 40 | val (columnNames, values) = map.unzip 41 | new CassandraRow(columnNames.toArray, values.map(_.asInstanceOf[AnyRef]).toArray) 42 | } 43 | 44 | } 45 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/rdd/reader/ClassBasedRowReader.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd.reader 2 | 3 | import scala.collection.JavaConversions._ 4 | import scala.reflect.runtime.universe._ 5 | 6 | import com.datastax.driver.core.{ProtocolVersion, Row} 7 | 8 | import com.datastax.spark.connector._ 9 | import com.datastax.spark.connector.cql.TableDef 10 | import com.datastax.spark.connector.mapper._ 11 | import com.datastax.spark.connector.util.JavaApiHelper 12 | 13 | 14 | /** Transforms a Cassandra Java driver `Row` into an object of a user provided class, 15 | * calling the class constructor */ 16 | final class ClassBasedRowReader[R : TypeTag : ColumnMapper]( 17 | table: TableDef, 18 | selectedColumns: IndexedSeq[ColumnRef]) 19 | extends RowReader[R] { 20 | 21 | private val converter = 22 | new GettableDataToMappedTypeConverter[R](table, selectedColumns) 23 | 24 | private val isReadingTuples = 25 | TypeTag.synchronized(typeTag[R].tpe.typeSymbol.fullName startsWith "scala.Tuple") 26 | 27 | override val neededColumns = { 28 | val ctorRefs = converter.columnMap.constructor 29 | val setterRefs = converter.columnMap.setters.values 30 | Some(ctorRefs ++ setterRefs) 31 | } 32 | 33 | override def read(row: Row, ignored: Array[String])(implicit protocolVersion: ProtocolVersion): R = { 34 | // can't use passed array of column names, because it is already after applying aliases 35 | val columnNames = row.getColumnDefinitions.iterator.map(_.getName).toArray 36 | val cassandraRow = CassandraRow.fromJavaDriverRow(row, columnNames) 37 | converter.convert(cassandraRow) 38 | } 39 | } 40 | 41 | 42 | class ClassBasedRowReaderFactory[R : TypeTag : ColumnMapper] extends RowReaderFactory[R] { 43 | 44 | def columnMapper = implicitly[ColumnMapper[R]] 45 | 46 | override def rowReader(tableDef: TableDef, selection: IndexedSeq[ColumnRef]) = 47 | new ClassBasedRowReader[R](tableDef, selection) 48 | 49 | override def targetClass: Class[R] = JavaApiHelper.getRuntimeClass(typeTag[R]) 50 | } 51 | -------------------------------------------------------------------------------- /spark-cassandra-connector-java/src/it/java/com/datastax/spark/connector/CassandraStreamingJavaUtilTest.java: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector; 2 | 3 | import com.datastax.spark.connector.japi.DStreamJavaFunctions; 4 | import com.datastax.spark.connector.japi.StreamingContextJavaFunctions; 5 | import org.apache.spark.streaming.StreamingContext; 6 | import org.apache.spark.streaming.api.java.JavaDStream; 7 | import org.apache.spark.streaming.api.java.JavaStreamingContext; 8 | import org.apache.spark.streaming.dstream.DStream; 9 | import org.junit.Test; 10 | 11 | import static com.datastax.spark.connector.japi.CassandraStreamingJavaUtil.*; 12 | import static org.hamcrest.CoreMatchers.is; 13 | import static org.hamcrest.MatcherAssert.assertThat; 14 | import static org.mockito.Mockito.mock; 15 | import static org.mockito.Mockito.when; 16 | 17 | @SuppressWarnings("unchecked") 18 | public class CassandraStreamingJavaUtilTest 19 | { 20 | 21 | @Test 22 | public void testJavaFunctions2() throws Exception { 23 | StreamingContext ssc = mock(StreamingContext.class); 24 | StreamingContextJavaFunctions scjf = javaFunctions(ssc); 25 | assertThat(scjf.ssc, is(ssc)); 26 | } 27 | 28 | @Test 29 | public void testJavaFunctions3() throws Exception { 30 | JavaStreamingContext jsc = mock(JavaStreamingContext.class); 31 | StreamingContext ssc = mock(StreamingContext.class); 32 | when(jsc.ssc()).thenReturn(ssc); 33 | StreamingContextJavaFunctions scjf = javaFunctions(jsc); 34 | assertThat(scjf.ssc, is(ssc)); 35 | } 36 | 37 | @Test 38 | public void testJavaFunctions6() throws Exception { 39 | DStream ds = mock(DStream.class); 40 | DStreamJavaFunctions dsjf = javaFunctions(ds); 41 | assertThat(dsjf.dstream, is(ds)); 42 | } 43 | 44 | @Test 45 | public void testJavaFunctions7() throws Exception { 46 | JavaDStream jds = mock(JavaDStream.class); 47 | DStream dstream = mock(DStream.class); 48 | when(jds.dstream()).thenReturn(dstream); 49 | DStreamJavaFunctions dsjf = javaFunctions(jds); 50 | assertThat(dsjf.dstream, is(jds.dstream())); 51 | } 52 | } -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/writer/RoutingKeyGenerator.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.writer 2 | 3 | import java.nio.ByteBuffer 4 | 5 | import com.datastax.driver.core.BoundStatement 6 | import com.datastax.spark.connector.cql.TableDef 7 | 8 | /** This class computes the routing key of a bound statement. */ 9 | class RoutingKeyGenerator(table: TableDef, columnNames: Seq[String]) 10 | extends ((BoundStatement) => ByteBuffer) { 11 | 12 | private val partitionKeyIdxs = { 13 | val missing = table.partitionKey 14 | .map(_.columnName) 15 | .filterNot(columnNames.contains) 16 | require( 17 | missing.isEmpty, 18 | s"Not all partition key columns of ${table.name} were selected: [${missing.mkString(", ")}]") 19 | table.partitionKey 20 | .map(pkColumn ⇒ columnNames.indexOf(pkColumn.columnName)) 21 | .filter(_ >= 0) 22 | } 23 | 24 | @transient 25 | protected lazy val routingKey = new ThreadLocal[Array[ByteBuffer]] { 26 | override def initialValue() = Array.ofDim[ByteBuffer](partitionKeyIdxs.size) 27 | } 28 | 29 | // this method is copied from Java Driver 30 | private def composeRoutingKeys(buffers: Array[ByteBuffer]): ByteBuffer = { 31 | val totalLength = buffers.map(_.remaining() + 3).sum 32 | val out = ByteBuffer.allocate(totalLength) 33 | 34 | for (buffer ← buffers) { 35 | val bb = buffer.duplicate 36 | out.put(((bb.remaining >> 8) & 0xFF).toByte) 37 | out.put((bb.remaining & 0xFF).toByte) 38 | out.put(bb) 39 | out.put(0.toByte) 40 | } 41 | out.flip 42 | out 43 | } 44 | 45 | private def fillRoutingKey(stmt: BoundStatement): Array[ByteBuffer] = { 46 | val rk = routingKey.get 47 | for (i ← partitionKeyIdxs.indices) { 48 | if (stmt.isNull(partitionKeyIdxs(i))) 49 | throw NullKeyColumnException(columnNames(partitionKeyIdxs(i))) 50 | rk(i) = stmt.getBytesUnsafe(partitionKeyIdxs(i)) 51 | } 52 | rk 53 | } 54 | 55 | def apply(stmt: BoundStatement): ByteBuffer = { 56 | val rk = fillRoutingKey(stmt) 57 | if (rk.length == 1) rk(0) else composeRoutingKeys(rk) 58 | } 59 | 60 | } 61 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/perf/scala/com/datastax/spark/connector/util/BenchmarkUtil.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.util 2 | 3 | import scala.annotation.tailrec 4 | 5 | /** Utilities for benchmarking code */ 6 | object BenchmarkUtil { 7 | 8 | private val sampleSize = 5 9 | private val minTime = 1000000000 // 1s 10 | 11 | def formatTime(time: Double): String = { 12 | if (time < 1e-7) 13 | f"${time * 1000000000.0}%.2f ns" 14 | else if (time < 1e-4) 15 | f"${time * 1000000.0}%.2f us" 16 | else if (time < 0.1) 17 | f"${time * 1000.0}%.2f ms" 18 | else 19 | f"$time%.3f s" 20 | } 21 | 22 | @inline 23 | private def time(loops: Long)(code: => Any): Long = { 24 | val start = System.nanoTime() 25 | var counter = 0L 26 | while (counter < loops) { 27 | code 28 | counter += 1L 29 | } 30 | val end = System.nanoTime() 31 | end - start 32 | } 33 | 34 | @inline 35 | private def timeN(loops: Long)(code: => Any): Long = { 36 | var i = 0 37 | var bestTime = Long.MaxValue 38 | while (bestTime >= minTime && i < sampleSize) { 39 | val t = time(loops)(code) 40 | if (t < bestTime) 41 | bestTime = t 42 | i += 1 43 | } 44 | bestTime 45 | } 46 | 47 | /** Runs the given code multiple times and prints how long it took. */ 48 | @tailrec 49 | @inline 50 | def timeIt[T](loops: Long)(code: => T): T = { 51 | val bestTime = timeN(loops)(code) 52 | if (bestTime < minTime) 53 | timeIt(loops * 10)(code) 54 | else { 55 | val timePerLoop = bestTime / loops 56 | printf("loops: %d, time per loop: %s, loops/s: %.3f\n", 57 | loops, formatTime(timePerLoop / 1000000000.0), 1000000000.0 / timePerLoop) 58 | code 59 | } 60 | } 61 | 62 | @inline 63 | def timeIt[T](code: => T): T = 64 | timeIt(1)(code) 65 | 66 | 67 | def printTime[T](message: String)(code: => T): T = { 68 | val start = System.nanoTime() 69 | val result = code 70 | val end = System.nanoTime() 71 | println(message + ": " + formatTime((end - start) / 1000000000.0)) 72 | result 73 | } 74 | 75 | def printTime[T](code: => T): T = 76 | printTime("elapsed")(code) 77 | } 78 | -------------------------------------------------------------------------------- /sbt/sbt: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # 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, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | # This script launches sbt for this project. If present it uses the system 21 | # version of sbt. If there is no system version of sbt it attempts to download 22 | # sbt locally. 23 | SBT_VERSION=0.13.8 24 | URL1=http://typesafe.artifactoryonline.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar 25 | URL2=http://repo.typesafe.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar 26 | JAR=sbt/sbt-launch-${SBT_VERSION}.jar 27 | 28 | # Download sbt launch jar if it hasn't been downloaded yet 29 | if [ ! -f ${JAR} ]; then 30 | # Download 31 | printf "Attempting to fetch sbt\n" 32 | JAR_DL=${JAR}.part 33 | if hash curl 2>/dev/null; then 34 | (curl --progress-bar ${URL1} > ${JAR_DL} || curl --progress-bar ${URL2} > ${JAR_DL}) && mv ${JAR_DL} ${JAR} 35 | elif hash wget 2>/dev/null; then 36 | (wget --progress=bar ${URL1} -O ${JAR_DL} || wget --progress=bar ${URL2} -O ${JAR_DL}) && mv ${JAR_DL} ${JAR} 37 | else 38 | printf "You do not have curl or wget installed, please install sbt manually from http://www.scala-sbt.org/\n" 39 | exit -1 40 | fi 41 | fi 42 | if [ ! -f ${JAR} ]; then 43 | # We failed to download 44 | printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n" 45 | exit -1 46 | fi 47 | printf "Launching sbt from ${JAR}\n" 48 | java \ 49 | -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=256m \ 50 | -jar ${JAR} \ 51 | "$@" 52 | -------------------------------------------------------------------------------- /spark-cassandra-connector-embedded/src/main/scala/com/datastax/spark/connector/embedded/Assertions.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.embedded 2 | 3 | import scala.annotation.tailrec 4 | import scala.concurrent.duration._ 5 | 6 | /** 7 | * Simple helper assertions. Some stolen from Akka akka.testkit.TestKit.scala for now. 8 | */ 9 | trait Assertions { 10 | 11 | /** Obtain current time (`System.nanoTime`) as Duration. */ 12 | def now: FiniteDuration = System.nanoTime.nanos 13 | 14 | private var end: Duration = Duration.Undefined 15 | 16 | /** 17 | * Obtain time remaining for execution of the innermost enclosing `within` 18 | * block or missing that it returns the properly dilated default for this 19 | * case from settings (key "akka.test.single-expect-default"). 20 | */ 21 | def remainingOrDefault = remainingOr(1.seconds.dilated) 22 | 23 | /** 24 | * Obtain time remaining for execution of the innermost enclosing `within` 25 | * block or missing that it returns the given duration. 26 | */ 27 | def remainingOr(duration: FiniteDuration): FiniteDuration = end match { 28 | case x if x eq Duration.Undefined => duration 29 | case x if !x.isFinite => throw new IllegalArgumentException("`end` cannot be infinite") 30 | case f: FiniteDuration => f - now 31 | } 32 | 33 | /** 34 | * Await until the given condition evaluates to `true` or the timeout 35 | * expires, whichever comes first. 36 | * If no timeout is given, take it from the innermost enclosing `within` 37 | * block. 38 | */ 39 | def awaitCond(p: => Boolean, max: Duration = 3.seconds, interval: Duration = 100.millis, message: String = "") { 40 | val _max = remainingOrDilated(max) 41 | val stop = now + _max 42 | 43 | @tailrec 44 | def poll(t: Duration) { 45 | if (!p) { 46 | assert(now < stop, s"timeout ${_max} expired: $message") 47 | Thread.sleep(t.toMillis) 48 | poll((stop - now) min interval) 49 | } 50 | } 51 | 52 | poll(_max min interval) 53 | } 54 | 55 | private def remainingOrDilated(max: Duration): FiniteDuration = max match { 56 | case x if x eq Duration.Undefined => remainingOrDefault 57 | case x if !x.isFinite => throw new IllegalArgumentException("max duration cannot be infinite") 58 | case f: FiniteDuration => f.dilated 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/mapper/JavaBeanColumnMapper.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.mapper 2 | 3 | import java.lang.reflect.Method 4 | 5 | import com.datastax.spark.connector.ColumnRef 6 | import com.datastax.spark.connector.cql.{TableDef, StructDef} 7 | 8 | import scala.reflect.ClassTag 9 | 10 | class JavaBeanColumnMapper[T : ClassTag](columnNameOverride: Map[String, String] = Map.empty) 11 | extends ReflectionColumnMapper[T] { 12 | 13 | import com.datastax.spark.connector.mapper.JavaBeanColumnMapper._ 14 | 15 | private def propertyName(accessorName: String) = { 16 | val AccessorRegex(_, strippedName) = accessorName 17 | strippedName(0).toLower + strippedName.substring(1) 18 | } 19 | 20 | override protected def isGetter(method: Method): Boolean = 21 | GetterRegex.findFirstMatchIn(method.getName).isDefined && 22 | method.getParameterTypes.isEmpty && 23 | method.getReturnType != Void.TYPE 24 | 25 | override protected def isSetter(method: Method): Boolean = 26 | SetterRegex.findFirstMatchIn(method.getName).isDefined && 27 | method.getParameterTypes.size == 1 && 28 | method.getReturnType == Void.TYPE 29 | 30 | private def resolve(name: String, columns: Map[String, ColumnRef]): Option[ColumnRef] = { 31 | val overridenName = columnNameOverride.getOrElse(name, name) 32 | ColumnMapperConvention.columnForProperty(overridenName, columns) 33 | } 34 | 35 | override protected def getterToColumnName(getterName: String, columns: Map[String, ColumnRef]) = { 36 | val p = propertyName(getterName) 37 | resolve(p, columns) 38 | } 39 | 40 | override protected def setterToColumnName(setterName: String, columns: Map[String, ColumnRef]) = { 41 | val p = propertyName(setterName) 42 | resolve(p, columns) 43 | } 44 | 45 | override protected def constructorParamToColumnName( 46 | paramName: String, 47 | columns: Map[String, ColumnRef]) = { 48 | resolve(paramName, columns) 49 | } 50 | 51 | /** Java Beans allow nulls in property values */ 52 | override protected def allowsNull = true 53 | 54 | // TODO: Implement 55 | override def newTable(keyspaceName: String, tableName: String): TableDef = ??? 56 | } 57 | 58 | object JavaBeanColumnMapper { 59 | val GetterRegex = "^(get|is)(.+)$".r 60 | val SetterRegex = "^(set)(.+)$".r 61 | val AccessorRegex = "^(get|is|set)(.+)$".r 62 | } -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/scala/com/datastax/spark/connector/writer/RateLimiterSpec.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.writer 2 | 3 | import org.scalamock.scalatest.MockFactory 4 | import org.scalatest.concurrent.Eventually 5 | import org.scalatest.{FlatSpec, Matchers} 6 | 7 | 8 | class RateLimiterSpec extends FlatSpec with Matchers with MockFactory with Eventually{ 9 | 10 | val TestRates = Seq(1L, 2L, 4L, 6L, 8L, 16L, 32L, WriteConf.DefaultThroughputMiBPS.toLong) 11 | 12 | "RateLimiter" should "not cause delays if rate is not exceeded" in { 13 | var now: Long = 0 14 | val sleep = mockFunction[Long, Any]("sleep") 15 | sleep.expects(*).never() 16 | 17 | val limiter = new RateLimiter(Long.MaxValue, 1000, () => now, sleep) 18 | for (i <- 1 to 1000000) { 19 | now += 1 20 | limiter.maybeSleep(1000) 21 | } 22 | } 23 | 24 | it should "sleep to not exceed the target rate" in { 25 | var now: Long = 0 26 | var sleepTime: Long = 0 27 | 28 | def sleep(delay: Long) = { 29 | sleepTime += delay 30 | now += delay 31 | } 32 | 33 | // 10 units per second + 5 units burst allowed 34 | val bucketSize = 5 35 | val rate = 10 36 | val limiter = new RateLimiter(rate, bucketSize, () => now, sleep) 37 | 38 | val iterations = 25 39 | for (i <- 1 to iterations) 40 | limiter.maybeSleep(1) 41 | 42 | sleepTime should be((iterations - bucketSize) * 1000L / rate) 43 | } 44 | 45 | it should "sleep and leak properly with different Rates" in { 46 | for (rate <- TestRates) { 47 | val bucketSize = rate * 2 48 | var now: Long = 0 49 | var sleepTime: Long = 0 50 | 51 | def sleep(delay: Long) = { 52 | sleepTime += delay 53 | now += delay 54 | } 55 | 56 | val limiter = new RateLimiter(rate, rate * 2, () => now, sleep) 57 | for (leakNum <- 1 to 1000) { 58 | assert( 59 | limiter.bucketFill.get() >= 0, 60 | "bucketFill has been overflowed, or has had a large negative number added to it") 61 | limiter.maybeSleep(rate) 62 | } 63 | 64 | eventually { 65 | limiter.leak() 66 | val delay = (limiter.bucketFill.get() - bucketSize) * 1000 / rate 67 | assert(delay <= 0, "Rate limiter was unable to leak it's way back to 0 delay") 68 | } 69 | sleepTime should not be (0) 70 | } 71 | } 72 | 73 | } 74 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/writer/WriteOption.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.writer 2 | 3 | import java.util.Date 4 | 5 | import scala.concurrent.duration.{Duration ⇒ ScalaDuration} 6 | 7 | import org.joda.time.{DateTime, Duration ⇒ JodaDuration} 8 | 9 | sealed trait WriteOptionValue[+T] 10 | 11 | case class StaticWriteOptionValue[T](value: T) extends WriteOptionValue[T] 12 | 13 | case class PerRowWriteOptionValue[T](placeholder: String) extends WriteOptionValue[T] 14 | 15 | sealed trait WriteOption[+T] 16 | 17 | case class TTLOption(value: WriteOptionValue[Int]) extends WriteOption[Int] 18 | 19 | case class TimestampOption(value: WriteOptionValue[Long]) extends WriteOption[Long] 20 | 21 | case object DefaultValue extends WriteOptionValue[Nothing] 22 | 23 | object WriteOption { 24 | def unapply(writeOption: WriteOption[_]): Option[WriteOptionValue[_]] = writeOption match { 25 | case TTLOption(value) => Some(value) 26 | case TimestampOption(value) => Some(value) 27 | case _ => None 28 | } 29 | } 30 | 31 | object TTLOption { 32 | 33 | val defaultValue = TTLOption(DefaultValue) 34 | 35 | def forever: TTLOption = TTLOption(StaticWriteOptionValue[Int](0)) 36 | 37 | /** @param ttl TTL in seconds */ 38 | def constant(ttl: Int): TTLOption = { 39 | require(ttl > 0, "Explicitly specified TTL must be greater than zero.") 40 | TTLOption(StaticWriteOptionValue(ttl)) 41 | } 42 | 43 | def constant(ttl: JodaDuration): TTLOption = constant(ttl.getStandardSeconds.toInt) 44 | 45 | def constant(ttl: ScalaDuration): TTLOption = if (ttl.isFinite()) constant(ttl.toSeconds.toInt) else forever 46 | 47 | def perRow(placeholder: String): TTLOption = TTLOption(PerRowWriteOptionValue[Int](placeholder)) 48 | 49 | } 50 | 51 | object TimestampOption { 52 | 53 | val defaultValue = TimestampOption(DefaultValue) 54 | 55 | def constant(microseconds: Long): TimestampOption = { 56 | require(microseconds > 0, "Explicitly specified time must be greater than zero.") 57 | TimestampOption(StaticWriteOptionValue(microseconds)) 58 | } 59 | 60 | def constant(timestamp: Date): TimestampOption = constant(timestamp.getTime * 1000L) 61 | 62 | def constant(timestamp: DateTime): TimestampOption = constant(timestamp.getMillis * 1000L) 63 | 64 | def perRow(placeholder: String): TimestampOption = 65 | TimestampOption(PerRowWriteOptionValue(placeholder)) 66 | } 67 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/writer/WritableToCassandra.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.writer 2 | 3 | import com.datastax.spark.connector.ColumnSelector 4 | import com.datastax.spark.connector.cql.CassandraConnector 5 | import org.apache.spark.SparkContext 6 | 7 | abstract class WritableToCassandra[T] { 8 | 9 | def sparkContext: SparkContext 10 | 11 | private[connector] lazy val connector = CassandraConnector(sparkContext.getConf) 12 | 13 | /** 14 | * Saves the data from [[org.apache.spark.rdd.RDD RDD]] to a Cassandra table. 15 | * By default, it saves all properties that have corresponding Cassandra columns. 16 | * 17 | * Example: 18 | * {{{ 19 | * CREATE KEYSPACE test WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1 }; 20 | * CREATE TABLE test.words(word VARCHAR PRIMARY KEY, count INT, other VARCHAR); 21 | * }}} 22 | * 23 | * {{{ 24 | * case class WordCount(word: String, count: Int, other: String) 25 | * val rdd = sc.parallelize(Seq(WordCount("foo", 5, "bar"))) 26 | * }}} 27 | * 28 | * By default, the underlying RDD class must provide data for all columns: 29 | * {{{ 30 | * rdd.saveToCassandra("test", "words") 31 | * }}} 32 | * 33 | * By default, writes are performed at ConsistencyLevel.ONE in order to leverage data-locality and minimize network traffic. 34 | * This write consistency level is controlled by the following property: 35 | * - spark.cassandra.output.consistency.level: consistency level for RDD writes, string matching the ConsistencyLevel enum name. 36 | * 37 | * @param keyspaceName the name of the Keyspace to use 38 | * @param tableName the name of the Table to use 39 | * @param columnNames The list of column names to save data to. 40 | * Uses only the unique column names, and you must select at least all primary key 41 | * columns. All other fields are discarded. Non-selected property/column names are left unchanged. 42 | * @param writeConf additional configuration object allowing to set consistency level, batch size, etc. 43 | */ 44 | def saveToCassandra(keyspaceName: String, 45 | tableName: String, 46 | columnNames: ColumnSelector, 47 | writeConf: WriteConf) 48 | (implicit connector: CassandraConnector, rwf: RowWriterFactory[T]) 49 | 50 | } 51 | -------------------------------------------------------------------------------- /spark-cassandra-connector-java/src/main/java/com/datastax/spark/connector/japi/CassandraStreamingJavaUtil.java: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.japi; 2 | 3 | import org.apache.spark.streaming.StreamingContext; 4 | import org.apache.spark.streaming.api.java.JavaDStream; 5 | import org.apache.spark.streaming.api.java.JavaStreamingContext; 6 | import org.apache.spark.streaming.dstream.DStream; 7 | 8 | /** 9 | * The main entry point to Spark Cassandra Connector Java API for Spark Streaming. 10 | * 11 | *

There are several helpful static factory methods which build useful wrappers around Streaming 12 | * Context and DStream.

13 | */ 14 | @SuppressWarnings("UnusedDeclaration") 15 | public class CassandraStreamingJavaUtil 16 | { 17 | 18 | private CassandraStreamingJavaUtil() { 19 | assert false; 20 | } 21 | 22 | // ------------------------------------------------------------------------- 23 | // Java API wrappers factory methods 24 | // ------------------------------------------------------------------------- 25 | 26 | /** 27 | * A static factory method to create a {@link StreamingContextJavaFunctions} based on an existing 28 | * {@link StreamingContext} instance. 29 | */ 30 | public static StreamingContextJavaFunctions javaFunctions(StreamingContext streamingContext) { 31 | return new StreamingContextJavaFunctions(streamingContext); 32 | } 33 | 34 | /** 35 | * A static factory method to create a {@link StreamingContextJavaFunctions} based on an existing 36 | * {@link JavaStreamingContext} instance. 37 | */ 38 | public static StreamingContextJavaFunctions javaFunctions(JavaStreamingContext streamingContext) { 39 | return new StreamingContextJavaFunctions(streamingContext.ssc()); 40 | } 41 | 42 | /** 43 | * A static factory method to create a {@link DStreamJavaFunctions} based on an existing 44 | * {@link DStream} instance. 45 | */ 46 | public static DStreamJavaFunctions javaFunctions(DStream dStream) { 47 | return new DStreamJavaFunctions<>(dStream); 48 | } 49 | 50 | /** 51 | * A static factory method to create a {@link DStreamJavaFunctions} based on an existing 52 | * {@link JavaDStream} instance. 53 | */ 54 | public static DStreamJavaFunctions javaFunctions(JavaDStream dStream) { 55 | return new DStreamJavaFunctions<>(dStream.dstream()); 56 | } 57 | 58 | } 59 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/org/apache/spark/sql/cassandra/types/InetAddressType.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.cassandra.types 2 | 3 | 4 | import scala.reflect.runtime.universe.typeTag 5 | 6 | import java.net.InetAddress 7 | 8 | import org.apache.spark.Logging 9 | import org.apache.spark.annotation.DeveloperApi 10 | import org.apache.spark.sql.catalyst.ScalaReflectionLock 11 | import org.apache.spark.sql.types.AtomicType 12 | 13 | 14 | /** 15 | * :: DeveloperApi :: 16 | * 17 | * The data type representing `InetAddress` values. 18 | * 19 | * @group dataType 20 | */ 21 | @DeveloperApi 22 | class InetAddressType private() extends AtomicType with Logging { 23 | // The companion object and this class is separated so the companion object also subclasses 24 | // this type. Otherwise, the companion object would be of type "InetAddressType$" in byte code. 25 | // Defined with a private constructor so the companion object is the only possible instantiation. 26 | private[sql] type InternalType = InetAddress 27 | @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } 28 | 29 | // Because this new type is not a Spark internal supported data type, there is no 30 | // CAST method to convert a InetAddress to a String. Spark internally converts it 31 | // to Double, so it errors out the query such as 32 | // select * from table where inet_address_column = '/74.125.239.135' 33 | // The correct query should be 34 | // select * from table where CAST(inet_address_column as string) = '/74.125.239.135' 35 | // 36 | // The following code provides only a reference ordering implementation. But it will 37 | // never be called by Spark, for there is no CAST function or UDF to convert String 38 | // to InetAddress 39 | // 40 | // Convert to host address to compare InetAddress because it doesn't support comparison. 41 | // It's not a good solution though. 42 | import Ordering.Implicits._ 43 | private[sql] def unsignedByte(x: Byte) = (x + 256) % 256 44 | private[sql] def address(inet: InetAddress): Iterable[Int] = 45 | inet.getAddress.map(unsignedByte) 46 | 47 | private[sql] val ordering = Ordering by address 48 | /** 49 | * The default size of a value of the InetAddressType is 16 bytes. 50 | */ 51 | override def defaultSize: Int = 16 52 | 53 | private[spark] override def asNullable: InetAddressType = this 54 | } 55 | 56 | case object InetAddressType extends InetAddressType -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/cql/RefCountMap.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.cql 2 | 3 | import scala.collection.concurrent.TrieMap 4 | import scala.annotation.tailrec 5 | 6 | /** Atomically counts references to objects of any type */ 7 | class RefCountMap[T] { 8 | 9 | private val refCounts = new TrieMap[T, Int] 10 | 11 | /** Returns current reference count for the given key. 12 | * This value may be constantly changing, so do not use it for synchronization purposes. */ 13 | final def get(key: T): Int = 14 | refCounts.getOrElse(key, 0) 15 | 16 | /** Atomically increases reference count only if the reference counter is already greater than 0. 17 | * @return true if reference counter was greater than zero and has been increased */ 18 | @tailrec 19 | final def acquireIfNonZero(key: T): Int = { 20 | refCounts.get(key) match { 21 | case Some(count) if count > 0 => 22 | if (refCounts.replace(key, count, count + 1)) 23 | count + 1 24 | else 25 | acquireIfNonZero(key) 26 | case _ => 27 | 0 28 | } 29 | } 30 | 31 | /** Atomically increases reference count by one. 32 | * @return reference count after increase */ 33 | @tailrec 34 | final def acquire(key: T): Int = { 35 | refCounts.get(key) match { 36 | case Some(count) => 37 | if (refCounts.replace(key, count, count + 1)) 38 | count + 1 39 | else 40 | acquire(key) 41 | case None => 42 | if (!refCounts.putIfAbsent(key, 1).isDefined) 43 | 1 44 | else 45 | acquire(key) 46 | } 47 | } 48 | 49 | /** Atomically decreases reference count by `n`. 50 | * @return reference count after decrease 51 | * @throws IllegalStateException if the reference count before decrease is less than `n` */ 52 | @tailrec 53 | final def release(key: T, n: Int = 1): Int = { 54 | refCounts.get(key) match { 55 | case Some(count) if count > n => 56 | if (refCounts.replace(key, count, count - n)) 57 | count - n 58 | else 59 | release(key, n) 60 | case Some(count) if count == n => 61 | if (refCounts.remove(key, n)) 62 | 0 63 | else 64 | release(key, n) 65 | case _ => 66 | throw new IllegalStateException("Release without acquire for key: " + key) 67 | } 68 | } 69 | 70 | /** Resets state of all counters to 0 */ 71 | def clear(): Unit = refCounts.clear() 72 | 73 | } 74 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/writer/AsyncExecutor.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.writer 2 | 3 | import java.util.concurrent.Semaphore 4 | 5 | import org.apache.spark.Logging 6 | import com.google.common.util.concurrent.{FutureCallback, Futures, ListenableFuture, SettableFuture} 7 | 8 | import scala.collection.concurrent.TrieMap 9 | import scala.util.Try 10 | 11 | import AsyncExecutor.Handler 12 | 13 | /** Asynchronously executes tasks but blocks if the limit of unfinished tasks is reached. */ 14 | class AsyncExecutor[T, R](asyncAction: T => ListenableFuture[R], maxConcurrentTasks: Int, 15 | successHandler: Option[Handler[T]] = None, failureHandler: Option[Handler[T]]) extends Logging { 16 | 17 | @volatile private var _successful = true 18 | 19 | private val semaphore = new Semaphore(maxConcurrentTasks) 20 | private val pendingFutures = new TrieMap[ListenableFuture[R], Boolean] 21 | 22 | /** Executes task asynchronously or blocks if more than `maxConcurrentTasks` limit is reached */ 23 | def executeAsync(task: T): ListenableFuture[R] = { 24 | val submissionTimestamp = System.nanoTime() 25 | semaphore.acquire() 26 | 27 | val settable = SettableFuture.create[R]() 28 | pendingFutures.put(settable, true) 29 | 30 | val executionTimestamp = System.nanoTime() 31 | val future = asyncAction(task) 32 | 33 | Futures.addCallback(future, new FutureCallback[R] { 34 | def release() { 35 | semaphore.release() 36 | pendingFutures.remove(settable) 37 | } 38 | def onSuccess(result: R) { 39 | release() 40 | settable.set(result) 41 | successHandler.foreach(_(task, submissionTimestamp, executionTimestamp)) 42 | } 43 | def onFailure(throwable: Throwable) { 44 | logError("Failed to execute: " + task, throwable) 45 | if (_successful) _successful = false 46 | release() 47 | settable.setException(throwable) 48 | failureHandler.foreach(_(task, submissionTimestamp, executionTimestamp)) 49 | } 50 | }) 51 | 52 | settable 53 | } 54 | 55 | /** Waits until the tasks being currently executed get completed. 56 | * It will not wait for tasks scheduled for execution during this method call, 57 | * nor tasks for which the [[executeAsync]] method did not complete. */ 58 | def waitForCurrentlyExecutingTasks() { 59 | for ((future, _) <- pendingFutures.snapshot()) 60 | Try(future.get()) 61 | } 62 | 63 | def successful = _successful 64 | 65 | } 66 | 67 | object AsyncExecutor { 68 | type Handler[T] = (T, Long, Long) => Unit 69 | } -------------------------------------------------------------------------------- /spark-cassandra-connector/src/test/scala/com/datastax/spark/connector/util/BufferedIterator2Spec.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.util 2 | 3 | import org.scalatest.{Matchers, FlatSpec} 4 | 5 | import scala.collection.mutable.ArrayBuffer 6 | 7 | class BufferedIterator2Spec extends FlatSpec with Matchers { 8 | 9 | "BufferedIterator" should "return the same items as the standard Iterator" in { 10 | val iterator = new BufferedIterator2(Seq(1, 2, 3, 4, 5).iterator) 11 | iterator.hasNext shouldBe true 12 | iterator.next() shouldBe 1 13 | iterator.hasNext shouldBe true 14 | iterator.next() shouldBe 2 15 | iterator.hasNext shouldBe true 16 | iterator.next() shouldBe 3 17 | iterator.hasNext shouldBe true 18 | iterator.next() shouldBe 4 19 | iterator.hasNext shouldBe true 20 | iterator.next() shouldBe 5 21 | iterator.hasNext shouldBe false 22 | } 23 | 24 | it should "be convertible to a Seq" in { 25 | val iterator = new BufferedIterator2(Seq(1, 2, 3, 4, 5).iterator) 26 | iterator.toSeq should contain inOrder(1, 2, 3, 4, 5) 27 | } 28 | 29 | it should "wrap an empty iterator" in { 30 | val iterator = new BufferedIterator2(Iterator.empty) 31 | iterator.isEmpty shouldBe true 32 | iterator.hasNext shouldBe false 33 | } 34 | 35 | it should "offer the head element without consuming the underlying iterator" in { 36 | val iterator = new BufferedIterator2(Seq(1, 2, 3, 4, 5).iterator) 37 | iterator.head shouldBe 1 38 | iterator.next() shouldBe 1 39 | } 40 | 41 | it should "offer takeWhile that consumes only the elements matching the predicate" in { 42 | val iterator = new BufferedIterator2(Seq(1, 2, 3, 4, 5).iterator) 43 | val firstThree = iterator.takeWhile(_ <= 3).toList 44 | 45 | firstThree should contain inOrder (1, 2, 3) 46 | iterator.head shouldBe 4 47 | iterator.next() shouldBe 4 48 | } 49 | 50 | it should "offer appendWhile that copies elements to ArrayBuffer and consumes only the elements matching the predicate" in { 51 | val iterator = new BufferedIterator2(Seq(1, 2, 3, 4, 5).iterator) 52 | val buffer = new ArrayBuffer[Int] 53 | iterator.appendWhile(_ <= 3, buffer) 54 | 55 | buffer should contain inOrder (1, 2, 3) 56 | iterator.head shouldBe 4 57 | iterator.next() shouldBe 4 58 | } 59 | 60 | it should "throw NoSuchElementException if trying to get next() element that doesn't exist" in { 61 | val iterator = new BufferedIterator2(Seq(1, 2).iterator) 62 | iterator.next() 63 | iterator.next() 64 | a [NoSuchElementException] should be thrownBy iterator.next() 65 | } 66 | } 67 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/cql/SessionProxy.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.cql 2 | 3 | import java.lang.reflect.{InvocationHandler, InvocationTargetException, Method, Proxy} 4 | 5 | import com.datastax.driver.core.{RegularStatement, Session, SimpleStatement} 6 | 7 | /** Wraps a `Session` and intercepts: 8 | * - `close` method to invoke `afterClose` handler 9 | * - `prepare` methods to cache `PreparedStatement` objects. */ 10 | class SessionProxy(session: Session, afterClose: Session => Any) extends InvocationHandler { 11 | 12 | private var closed = false 13 | 14 | override def invoke(proxy: Any, method: Method, args: Array[AnyRef]) = { 15 | try { 16 | val StringClass = classOf[String] 17 | val RegularStatementClass = classOf[String] 18 | 19 | (method.getName, method.getParameterTypes) match { 20 | case ("close", Array()) => 21 | null 22 | case ("closeUnderlying", Array()) => 23 | session.close() 24 | null 25 | case ("isClosed", Array()) => 26 | closed.asInstanceOf[AnyRef] 27 | case ("prepare", Array(StringClass)) => 28 | PreparedStatementCache.prepareStatement(session, new SimpleStatement(args(0).asInstanceOf[String])) 29 | case ("prepare", Array(RegularStatementClass)) => 30 | PreparedStatementCache.prepareStatement(session, args(0).asInstanceOf[RegularStatement]) 31 | case _ => 32 | try { 33 | method.invoke(session, args: _*) 34 | } 35 | catch { 36 | case e: InvocationTargetException => 37 | throw e.getCause 38 | } 39 | } 40 | } 41 | finally { 42 | if (method.getName == "close" && !closed) { 43 | closed = true 44 | afterClose(session) 45 | } 46 | } 47 | } 48 | } 49 | 50 | object SessionProxy { 51 | 52 | /** Creates a new `SessionProxy` delegating to the given `Session`. 53 | * The proxy adds prepared statement caching functionality. */ 54 | def wrap(session: Session): Session = 55 | wrapWithCloseAction(session)(_ => ()) 56 | 57 | /** Creates a new `SessionProxy` delegating to the given `Session`. 58 | * Additionally registers a callback on `Session#close` method. 59 | * @param afterClose code to be invoked after the session has been closed */ 60 | def wrapWithCloseAction(session: Session)(afterClose: Session => Any): Session = 61 | Proxy.newProxyInstance( 62 | session.getClass.getClassLoader, 63 | Array(classOf[Session]), 64 | new SessionProxy(session, afterClose)).asInstanceOf[Session] 65 | } -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/rdd/ReadConf.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.rdd 2 | 3 | import com.datastax.driver.core.ConsistencyLevel 4 | import com.datastax.spark.connector.util.ConfigCheck 5 | import org.apache.spark.SparkConf 6 | 7 | /** Read settings for RDD 8 | * 9 | * @param splitCount number of partitions to divide the data into; unset by default 10 | * @param splitSizeInMB size of Cassandra data to be read in a single Spark task; 11 | * determines the number of partitions, but ignored if `splitCount` is set 12 | * @param fetchSizeInRows number of CQL rows to fetch in a single round-trip to Cassandra 13 | * @param consistencyLevel consistency level for reads, default LOCAL_ONE; 14 | * higher consistency level will disable data-locality 15 | * @param taskMetricsEnabled whether or not enable task metrics updates (requires Spark 1.2+) */ 16 | case class ReadConf( 17 | splitCount: Option[Int] = None, 18 | splitSizeInMB: Int = ReadConf.DefaultSplitSizeInMB, 19 | fetchSizeInRows: Int = ReadConf.DefaultFetchSizeInRows, 20 | consistencyLevel: ConsistencyLevel = ReadConf.DefaultConsistencyLevel, 21 | taskMetricsEnabled: Boolean = ReadConf.DefaultReadTaskMetricsEnabled) 22 | 23 | 24 | object ReadConf { 25 | val ReadSplitSizeInMBProperty = "spark.cassandra.input.split.size_in_mb" 26 | val ReadFetchSizeInRowsProperty = "spark.cassandra.input.fetch.size_in_rows" 27 | val ReadConsistencyLevelProperty = "spark.cassandra.input.consistency.level" 28 | val ReadTaskMetricsProperty = "spark.cassandra.input.metrics" 29 | 30 | // Whitelist for allowed Read environment variables 31 | val Properties = Set( 32 | ReadSplitSizeInMBProperty, 33 | ReadFetchSizeInRowsProperty, 34 | ReadConsistencyLevelProperty, 35 | ReadTaskMetricsProperty 36 | ) 37 | 38 | val DefaultSplitSizeInMB = 64 // 64 MB 39 | val DefaultFetchSizeInRows = 1000 40 | val DefaultConsistencyLevel = ConsistencyLevel.LOCAL_ONE 41 | val DefaultReadTaskMetricsEnabled = true 42 | 43 | def fromSparkConf(conf: SparkConf): ReadConf = { 44 | 45 | ConfigCheck.checkConfig(conf) 46 | 47 | ReadConf( 48 | fetchSizeInRows = conf.getInt(ReadFetchSizeInRowsProperty, DefaultFetchSizeInRows), 49 | splitSizeInMB = conf.getInt(ReadSplitSizeInMBProperty, DefaultSplitSizeInMB), 50 | consistencyLevel = ConsistencyLevel.valueOf( 51 | conf.get(ReadConsistencyLevelProperty, DefaultConsistencyLevel.name())), 52 | taskMetricsEnabled = conf.getBoolean(ReadTaskMetricsProperty, DefaultReadTaskMetricsEnabled) 53 | ) 54 | } 55 | 56 | } 57 | 58 | -------------------------------------------------------------------------------- /spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/writer/RateLimiter.scala: -------------------------------------------------------------------------------- 1 | package com.datastax.spark.connector.writer 2 | 3 | import java.util.concurrent.atomic.AtomicLong 4 | 5 | import scala.annotation.tailrec 6 | 7 | /** A leaking bucket rate limiter. 8 | * It can be used to limit rate of anything, 9 | * but typically it is used to limit rate of data transfer. 10 | * 11 | * It starts with an empty bucket. 12 | * When packets arrive, they are added to the bucket. 13 | * The bucket has a constant size and is leaking at a constant rate. 14 | * If the bucket overflows, the thread is delayed by the 15 | * amount of time proportional to the amount of the overflow. 16 | * 17 | * This class is thread safe and lockless. 18 | * 19 | * @param rate maximum allowed long-term rate per 1000 units of time 20 | * @param bucketSize maximum acceptable "burst" 21 | * @param time source of time; typically 1 unit = 1 ms 22 | * @param sleep a function to call to slow down the calling thread; 23 | * must use the same time units as `time` 24 | */ 25 | class RateLimiter( 26 | rate: Long, 27 | bucketSize: Long, 28 | time: () => Long = System.currentTimeMillis, 29 | sleep: Long => Any = Thread.sleep) { 30 | 31 | require(rate > 0, "A positive rate is required") 32 | require(bucketSize > 0, "A positive bucket size is required") 33 | 34 | private[writer] val bucketFill = new AtomicLong(0L) 35 | private[writer] val lastTime = new AtomicLong(time()) //Avoid a large initial step 36 | 37 | @tailrec 38 | private def leak(toLeak: Long): Unit = { 39 | val fill = bucketFill.get() 40 | val reallyToLeak = math.min(fill, toLeak) // we can't leak more than there is now 41 | if (!bucketFill.compareAndSet(fill, fill - reallyToLeak)) 42 | leak(toLeak) 43 | } 44 | 45 | private[writer] def leak(): Unit = { 46 | val currentTime = time() 47 | val prevTime = lastTime.getAndSet(currentTime) 48 | val elapsedTime = math.max(currentTime - prevTime, 0L) // Protect against negative time 49 | leak(elapsedTime * rate / 1000L) 50 | } 51 | 52 | /** Processes a single packet. 53 | * If the packet is bigger than the current amount of 54 | * space available in the bucket, this method will 55 | * sleep for appropriate amount of time, in order 56 | * to not exceed the target rate. */ 57 | def maybeSleep(packetSize: Long): Unit = { 58 | leak() 59 | val currentFill = bucketFill.addAndGet(packetSize) 60 | val overflow = currentFill - bucketSize 61 | val delay = 1000L * overflow / rate 62 | if (delay > 0L) 63 | sleep(delay) 64 | } 65 | 66 | } 67 | --------------------------------------------------------------------------------