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