where sha is the last known working ccm commit
25 | run: pip install git+https://github.com/apache/cassandra-ccm.git@trunk
26 |
27 | - name: Setup Java
28 | uses: actions/setup-java@v4
29 | with:
30 | distribution: "temurin"
31 | java-version: | # order is important, the last one is the default which will be used by SBT
32 | 11
33 | 8
34 |
35 | - name: sbt tests
36 | env:
37 | TEST_PARALLEL_TASKS: 1
38 | CCM_CASSANDRA_VERSION: ${{ matrix.db-version }}
39 | PUBLISH_VERSION: test
40 | JAVA8_HOME: ${{ env.JAVA_HOME_8_X64 }}
41 | JAVA11_HOME: ${{ env.JAVA_HOME_11_X64 }}
42 | run: sbt/sbt ++${{ matrix.scala }} test it:test
43 |
44 | - name: Publish Test Report
45 | uses: mikepenz/action-junit-report@v4
46 | if: always()
47 | with:
48 | report_paths: '**/target/test-reports/*.xml'
49 | annotate_only: true
50 |
--------------------------------------------------------------------------------
/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/SmallIntTypeTest.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.rdd.typeTests
20 |
21 | import com.datastax.oss.driver.api.core.DefaultProtocolVersion
22 | import com.datastax.oss.driver.api.core.cql.Row
23 | import com.datastax.spark.connector.cluster.DefaultCluster
24 |
25 | class SmallIntTypeTest extends AbstractTypeTest[Short, java.lang.Short] with DefaultCluster {
26 | override val minPV = DefaultProtocolVersion.V4
27 | override protected val typeName: String = "smallint"
28 |
29 | override protected val typeData: Seq[Short] = (1 to 10).map(_.toShort)
30 | override protected val addData: Seq[Short] = (11 to 20).map(_.toShort)
31 |
32 | override def getDriverColumn(row: Row, colName: String): Short = row.getShort(colName)
33 |
34 | }
35 |
--------------------------------------------------------------------------------
/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/FloatTypeTest.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.rdd.typeTests
20 |
21 | import java.lang.Float
22 |
23 | import com.datastax.oss.driver.api.core.cql.Row
24 | import com.datastax.spark.connector.cluster.DefaultCluster
25 |
26 | class FloatTypeTest extends AbstractTypeTest[Float, Float] with DefaultCluster {
27 | override val typeName = "float"
28 |
29 | override val typeData: Seq[Float] = Seq(new Float(100.1), new Float(200.2),new Float(300.3), new Float(400.4), new Float(500.5))
30 | override val addData: Seq[Float] = Seq(new Float(600.6), new Float(700.7), new Float(800.8), new Float(900.9), new Float(1000.12))
31 |
32 | override def getDriverColumn(row: Row, colName: String): Float = {
33 | row.getFloat(colName)
34 | }
35 |
36 | }
37 |
38 |
--------------------------------------------------------------------------------
/connector/src/main/scala-2.12/com/datastax/spark/connector/util/RuntimeUtil.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.util
20 |
21 | import org.apache.spark.repl.SparkILoop
22 | import scala.tools.nsc.Settings
23 | import java.io.{BufferedReader, PrintWriter}
24 | import scala.collection.parallel.ParIterable
25 |
26 | class Scala213SparkILoop(in: BufferedReader, out: PrintWriter) extends SparkILoop(in, out) {
27 |
28 | def run(interpreterSettings: Settings): Boolean = {
29 | super.process(interpreterSettings)
30 | }
31 | }
32 |
33 |
34 | object RuntimeUtil {
35 |
36 | def toParallelIterable[A](iterable: Iterable[A]): ParIterable[A] = {
37 | iterable.par
38 | }
39 |
40 | def createSparkILoop(in: BufferedReader, out: PrintWriter): Scala213SparkILoop = {
41 | new Scala213SparkILoop(in, out)
42 | }
43 | }
44 |
--------------------------------------------------------------------------------
/connector/src/main/scala/com/datastax/spark/connector/PairRDDFunctions.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector
20 |
21 | import com.datastax.spark.connector.rdd.SpannedByKeyRDD
22 | import org.apache.spark.rdd.RDD
23 |
24 | class PairRDDFunctions[K, V](rdd: RDD[(K, V)]) extends Serializable {
25 |
26 | /**
27 | * Groups items with the same key, assuming the items with the same key are next to each other
28 | * in the collection. It does not perform shuffle, therefore it is much faster than using
29 | * much more universal Spark RDD `groupByKey`. For this method to be useful with Cassandra tables,
30 | * the key must represent a prefix of the primary key, containing at least the partition key of the
31 | * Cassandra table. */
32 | def spanByKey: RDD[(K, Seq[V])] =
33 | new SpannedByKeyRDD[K, V](rdd)
34 |
35 | }
36 |
--------------------------------------------------------------------------------
/connector/src/main/scala/com/datastax/spark/connector/writer/RowWriter.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.writer
20 |
21 |
22 | /** `RowWriter` knows how to extract column names and values from custom row objects
23 | * and how to convert them to values that can be written to Cassandra.
24 | * `RowWriter` is required to apply any user-defined data type conversion. */
25 | trait RowWriter[T] extends Serializable {
26 |
27 | /** List of columns this `RowWriter` is going to write.
28 | * Used to construct appropriate INSERT or UPDATE statement. */
29 | def columnNames: Seq[String]
30 |
31 | /** Extracts column values from `data` object and writes them into the given buffer
32 | * in the same order as they are listed in the columnNames sequence. */
33 | def readColumnValues(data: T, buffer: Array[Any])
34 |
35 | }
36 |
--------------------------------------------------------------------------------
/driver/src/main/scala/com/datastax/spark/connector/mapper/ColumnMapperConvention.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.mapper
20 |
21 | import com.datastax.spark.connector.ColumnRef
22 | import org.apache.commons.lang3.StringUtils
23 |
24 | object ColumnMapperConvention {
25 |
26 | def camelCaseToUnderscore(str: String): String =
27 | StringUtils.splitByCharacterTypeCamelCase(str).mkString("_").replaceAll("_+", "_").toLowerCase
28 |
29 | def columnForProperty(propertyName: String, columnByName: Map[String, ColumnRef]): Option[ColumnRef] = {
30 | val underscoreName = camelCaseToUnderscore(propertyName)
31 | val candidateColumnNames = Seq(propertyName, underscoreName)
32 | candidateColumnNames.iterator
33 | .map(name => columnByName.get(name))
34 | .find(_.isDefined)
35 | .flatten
36 | }
37 | }
38 |
--------------------------------------------------------------------------------
/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/BigintTypeTest.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.rdd.typeTests
20 |
21 | import java.lang.Long
22 |
23 | import com.datastax.oss.driver.api.core.cql.Row
24 | import com.datastax.spark.connector.cluster.DefaultCluster
25 |
26 | class BigintTypeTest extends AbstractTypeTest[Long, Long] with DefaultCluster {
27 | override val typeName = "bigint"
28 |
29 | override val typeData: Seq[Long] = Seq(new Long(1000000L), new Long(2000000L), new Long(3000000L), new Long(4000000L), new Long(5000000L))
30 | override val addData: Seq[Long] = Seq(new Long(6000000000L), new Long(70000000L), new Long(80000000L), new Long(9000000L), new Long(10000000L))
31 |
32 | override def getDriverColumn(row: Row, colName: String): Long = {
33 | row.getLong(colName)
34 | }
35 | }
36 |
37 |
--------------------------------------------------------------------------------
/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/DoubleTypeTest.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.rdd.typeTests
20 |
21 | import java.lang.Double
22 |
23 | import com.datastax.oss.driver.api.core.cql.Row
24 | import com.datastax.spark.connector.cluster.DefaultCluster
25 |
26 | class DoubleTypeTest extends AbstractTypeTest[Double, Double] with DefaultCluster {
27 | override val typeName = "double"
28 |
29 | override val typeData: Seq[Double] = Seq(new Double(100.1), new Double(200.2),new Double(300.3), new Double(400.4), new Double(500.5))
30 | override val addData: Seq[Double] = Seq(new Double(600.6), new Double(700.7), new Double(800.8), new Double(900.9), new Double(1000.12))
31 |
32 | override def getDriverColumn(row: Row, colName: String): Double = {
33 | row.getDouble(colName)
34 | }
35 | }
36 |
37 |
--------------------------------------------------------------------------------
/test-support/src/main/scala/com/datastax/spark/connector/ccm/mode/ExistingModeExecutor.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.ccm.mode
20 | import com.datastax.spark.connector.ccm.CcmConfig
21 |
22 | import java.nio.file.{Files, Path}
23 |
24 | /**
25 | * A special ClusterModeExecutor which bypasses ccm and assumes a Cassandra instance on localhost
26 | * with default ports and no authentication.
27 | * */
28 | private[ccm] class ExistingModeExecutor(val config: CcmConfig) extends ClusterModeExecutor {
29 | override protected val dir: Path = Files.createTempDirectory("test")
30 |
31 | override def create(clusterName: String): Unit = {
32 | // do nothing
33 | }
34 |
35 | override def start(nodeNo: Int): Unit = {
36 | // do nothing
37 | }
38 |
39 | override def remove(): Unit = {
40 | // do nothing
41 | }
42 | }
43 |
--------------------------------------------------------------------------------
/connector/src/main/java/com/datastax/spark/connector/japi/StreamingContextJavaFunctions.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.japi;
20 |
21 | import org.apache.spark.streaming.StreamingContext;
22 |
23 | /**
24 | * Java API wrapper over {@link org.apache.spark.streaming.StreamingContext} to provide Spark Cassandra Connector
25 | * functionality.
26 | *
27 | * To obtain an instance of this wrapper, use one of the factory methods in {@link
28 | * com.datastax.spark.connector.japi.CassandraJavaUtil} class.
29 | */
30 | @SuppressWarnings("UnusedDeclaration")
31 | public class StreamingContextJavaFunctions extends SparkContextJavaFunctions {
32 | public final StreamingContext ssc;
33 |
34 | StreamingContextJavaFunctions(StreamingContext ssc) {
35 | super(ssc.sparkContext());
36 | this.ssc = ssc;
37 | }
38 | }
39 |
--------------------------------------------------------------------------------
/connector/src/main/scala/com/datastax/spark/connector/DocUtil.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector
20 |
21 | import java.nio.file.{FileSystems, Files, Paths}
22 |
23 | import com.datastax.spark.connector.util.{ConfigCheck, RefBuilder}
24 |
25 | object DocUtil {
26 |
27 | def main(args: Array[String]) {
28 |
29 | val DefaultReferenceFile = Paths.get("..").resolve("doc").resolve("reference.md")
30 |
31 | println("Generating Reference Documentation for Spark Cassandra Conenctor")
32 | println(s"Found ${ConfigCheck.validStaticProperties.size} Parameters")
33 |
34 | val markdown = RefBuilder.getMarkDown()
35 |
36 | println(s"Generating Reference Documentation for Spark Cassandra Conenctor to ${DefaultReferenceFile.toAbsolutePath}")
37 |
38 | Files.write(DefaultReferenceFile, markdown.getBytes)
39 |
40 | }
41 | }
42 |
--------------------------------------------------------------------------------
/driver/src/main/scala/com/datastax/spark/connector/util/Reflect.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.util
20 |
21 | import scala.reflect.runtime.universe._
22 |
23 | private[connector] object Reflect {
24 |
25 | def constructor(tpe: Type): Symbol = tpe.decl(termNames.CONSTRUCTOR)
26 |
27 | def member(tpe: Type, name: String): Symbol = tpe.member(TermName(name))
28 |
29 | def methodSymbol(tpe: Type): MethodSymbol = {
30 | val constructors = constructor(tpe).asTerm.alternatives.map(_.asMethod)
31 | val paramCount = constructors.map(_.paramLists.flatten.size).max
32 | constructors.filter(_.paramLists.flatten.size == paramCount) match {
33 | case List(onlyOne) => onlyOne
34 | case _ => throw new IllegalArgumentException(
35 | "Multiple constructors with the same number of parameters not allowed.")
36 | }
37 | }
38 | }
39 |
40 |
--------------------------------------------------------------------------------
/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/DecimalTypeTest.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.rdd.typeTests
20 |
21 | import com.datastax.oss.driver.api.core.cql.Row
22 | import com.datastax.spark.connector.cluster.DefaultCluster
23 |
24 |
25 | class DecimalTypeTest extends AbstractTypeTest[BigDecimal, java.math.BigDecimal] with DefaultCluster {
26 |
27 | implicit def toBigDecimal(str: String) = BigDecimal(str)
28 |
29 | override def convertToDriverInsertable(testValue: BigDecimal): java.math.BigDecimal = testValue.bigDecimal
30 |
31 | override val typeName = "decimal"
32 |
33 | override val typeData: Seq[BigDecimal] = Seq("100.1", "200.2", "301.1")
34 | override val addData: Seq[BigDecimal] = Seq("600.6", "700.7", "721.444")
35 |
36 | override def getDriverColumn(row: Row, colName: String): BigDecimal = {
37 | BigDecimal(row.getBigDecimal(colName))
38 | }
39 | }
40 |
--------------------------------------------------------------------------------
/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/TinyIntTypeTest.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.rdd.typeTests
20 |
21 | import com.datastax.oss.driver.api.core.DefaultProtocolVersion
22 | import com.datastax.oss.driver.api.core.cql.Row
23 | import com.datastax.spark.connector.cluster.DefaultCluster
24 |
25 | class TinyIntTypeTest extends AbstractTypeTest[Int, java.lang.Byte] with DefaultCluster {
26 | override val minPV = DefaultProtocolVersion.V4
27 | override protected val typeName: String = "tinyint"
28 |
29 | override protected val typeData: Seq[Int] =Seq(1, 2, 3, 4, 5)
30 | override protected val addData: Seq[Int] = Seq(6, 7, 8, 9, 10)
31 |
32 | override def getDriverColumn(row: Row, colName: String): Int = {
33 | row.getByte(colName).toInt
34 | }
35 |
36 | override def convertToDriverInsertable(testValue: Int): java.lang.Byte = testValue.toByte
37 |
38 |
39 | }
40 |
--------------------------------------------------------------------------------
/driver/src/main/scala/com/datastax/spark/connector/japi/UDTValue.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.japi
20 |
21 | import com.datastax.spark.connector.types.{NullableTypeConverter, TypeConverter}
22 | import com.datastax.spark.connector.{CassandraRowMetadata, UDTValue => ConnectorUDTValue}
23 |
24 | import scala.reflect.runtime.universe._
25 |
26 | final class UDTValue(val metaData: CassandraRowMetadata, val columnValues: IndexedSeq[AnyRef])
27 | extends JavaGettableData with Serializable
28 |
29 | object UDTValue {
30 |
31 | val UDTValueTypeTag = implicitly[TypeTag[UDTValue]]
32 |
33 | implicit object UDTValueConverter extends NullableTypeConverter[UDTValue] {
34 | def targetTypeTag = UDTValueTypeTag
35 |
36 | def convertPF = {
37 | case x: UDTValue => x
38 | case x: ConnectorUDTValue =>
39 | new UDTValue(x.metaData, x.columnValues)
40 | }
41 | }
42 |
43 | TypeConverter.registerConverter(UDTValueConverter)
44 |
45 | }
--------------------------------------------------------------------------------
/test-support/src/main/resources/client.key:
--------------------------------------------------------------------------------
1 | -----BEGIN PRIVATE KEY-----
2 | MIIEvAIBADANBgkqhkiG9w0BAQEFAASCBKYwggSiAgEAAoIBAQCrQnQShlCc6/Yp
3 | Gu/AD7Vkv1VneEN1AzCQN+K2gY0Hq6m8xRml/xb5F5CZpjrJpxgm2EfOxlN3cb3s
4 | HDzX/dU0dAH0+dcaZu6kwtVd4VfKxIKzWc8R0mVMFsldYdKHjUATh6MlqgKhO6KA
5 | obAj2VNs448PP9YxyKj2dqCwIl2x0bb5PGZHtGXLMQgA0nTCmcskQh4mZFLaWgtW
6 | 4JTiVn2EMs3ntw/2MRUPgeTaQkPgTHHsb0dAUf7/rcfpe0QNmwlxiV1HvIE2waun
7 | P0K6lQsJLVHjR49AQiVReEzh+3+GyWNDJstC+99c1VO+TlaBjss1oEnLoD9N+DfZ
8 | JcSffBS/AgMBAAECggEAMHATNEoY8skqTmX3+XJ3847KMQGq0qWcTq3/yW7K3KiI
9 | 0YNNxc1oSfuIQmzpo69G/XWembUuVlItTWKPMufwLW3CP++KD0WdqawRfQQHOKpr
10 | 7R4xmvDPBb5MJcVNLlmdDekHE9gJ9mBPjeItV3ZYSivygnWjt2DxqQPUXvzZUzlu
11 | munh3H5x6ehXVHDYGzosPgTpfmLHdlNfvF4x9bcklMMbCOyoPttXB2uHWOvUIS+/
12 | 2YEkPmJfZdpudI7RqN75yYi7N8+gpnCTp530zA2yONyZ8THqEG/0nWy+02/zm5sm
13 | gs1saYNwXME2IPekZNM/pJh2DtnTcxZaUt84q2nhAQKBgQDi8mgvE8ekbs6DLfKK
14 | YAtTuOcLRpuvJqxtiQecqaumzgZnmHtkm6yuDNjieqB6OITudP4NdhPpyvOFJw46
15 | zTHMpGqZboxHuxoxMOgmyeiO+cdSwGHobr1zUcT8jVmLH7A+LtL5hHi+733EbCRh
16 | sF04Vq9L46Q52mhcZKbs56U8MQKBgQDBLwotnOJH7hZD5sKS0o8/Sfj3pgzXEDpL
17 | RfnrBPGhLn+1zhPEYsEW3mKI/yHiOZHNXZMQ6oYmxThg03qKTjaY8OIm8sg/zrlZ
18 | M+o3wVnAzayrhw5gZ8DzqioHhEUMOAwwRFXRpfxqj8regrLjE9KaYty8ZYAFtwuH
19 | W2S3+MVT7wKBgGQx7XlLXErmeNpFgN1Cxf1ylt7Nj5Jmmp3Jb8jkx9ne/8jg8ylZ
20 | 6YT2OxLSXONY7Kdyk29SADyp05WnxoqDaUcWF9IhkmFg45FwLC5j2f61nCCWuyMp
21 | MQ8mvLdbmHrpxJ/PgGmU6NIzXe1IaU+P07g53S6+FBVOreCMt33ET5khAoGAGgKz
22 | ZCDTdsvfw5S2bf5buzHCi9WXtP1CXBA37iTkQ8d2+oucrbx+Mw4ORlPTxBnsP7Jx
23 | sr1hAqdbR+4xeZ2+TCliycu2mqDC4/fReWBXLVaEATRWAzT1DdnDfu+YPGTvfzA0
24 | Pd4TdmWV8w+19k0c9hyJi/Q+oIZczwTHMt4T85ECgYAe4J0ht6b6kPEG3d9vxmMN
25 | T23S+ucYLHnfT1nacTuBZnMphWHhSqf8UJloIGpusxDU84MdAp22Jpd9SfPi9KK9
26 | yZY9WDJGeb0Yk7ML1R5GcAAkM78lUw/rS2VfMjQFnnUl2jVMS8adcm8/vHcpkcn7
27 | MufMEZzDpeO/aI8nbClktw==
28 | -----END PRIVATE KEY-----
29 |
--------------------------------------------------------------------------------
/connector/src/it/scala/com/datastax/spark/connector/rdd/CassandraRDDMockSpec.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.rdd
20 |
21 | import com.datastax.spark.connector.cluster.DefaultCluster
22 | import com.datastax.spark.connector.{CassandraRow, SparkCassandraITFlatSpecBase}
23 | import com.datastax.spark.connector.cql.CassandraConnector
24 |
25 | class CassandraRDDMockSpec extends SparkCassandraITFlatSpecBase with DefaultCluster {
26 |
27 | override lazy val conn = CassandraConnector(defaultConf)
28 |
29 | "A CassandraRDDMock" should "behave like a CassandraRDD without needing Cassandra" in {
30 | val columns = Seq("key", "value")
31 | //Create a fake CassandraRDD[CassandraRow]
32 | val rdd = sc
33 | .parallelize(1 to 10)
34 | .map(num => CassandraRow.fromMap(columns.zip(Seq(num, num)).toMap))
35 |
36 | val fakeCassandraRDD: CassandraRDD[CassandraRow] = new CassandraRDDMock(rdd)
37 |
38 | fakeCassandraRDD.cassandraCount() should be (10)
39 | }
40 | }
41 |
--------------------------------------------------------------------------------
/connector/src/main/scala/com/datastax/spark/connector/rdd/reader/RowReader.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.rdd.reader
20 |
21 | import com.datastax.oss.driver.api.core.cql.Row
22 | import com.datastax.spark.connector.{CassandraRowMetadata, ColumnRef}
23 |
24 | /** Transforms a Cassandra Java driver `Row` into high-level row representation, e.g. a tuple
25 | * or a user-defined case class object. The target type `T` must be serializable. */
26 | trait RowReader[T] extends Serializable {
27 |
28 | /** Reads column values from low-level `Row` and turns them into higher level representation.
29 | *
30 | * @param row row fetched from Cassandra
31 | * @param rowMetaData column names and codec available in the `row` */
32 | def read(row: Row, rowMetaData: CassandraRowMetadata): T
33 |
34 | /** List of columns this `RowReader` is going to read.
35 | * Useful to avoid fetching the columns that are not needed. */
36 | def neededColumns: Option[Seq[ColumnRef]]
37 |
38 | }
39 |
--------------------------------------------------------------------------------
/connector/src/test/scala/com/datastax/spark/connector/rdd/reader/ClassBasedRowReaderTest.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.rdd.reader
20 |
21 | import com.datastax.spark.connector.cql.{RegularColumn, ColumnDef, PartitionKeyColumn, TableDef}
22 | import org.apache.commons.lang3.SerializationUtils
23 | import org.junit.Test
24 |
25 | import com.datastax.spark.connector.types.{BigIntType, IntType, VarCharType}
26 |
27 | case class TestClass(a: String, b: Int, c: Option[Long])
28 |
29 | class ClassBasedRowReaderTest {
30 |
31 | private val a = ColumnDef("a", PartitionKeyColumn, VarCharType)
32 | private val b = ColumnDef("b", RegularColumn, IntType)
33 | private val c = ColumnDef("c", RegularColumn, BigIntType)
34 | private val table = TableDef("test", "table", Seq(a), Nil, Seq(b, c))
35 |
36 | @Test
37 | def testSerialize() {
38 | val reader = new ClassBasedRowReader[TestClass](table, table.columnRefs)
39 | SerializationUtils.roundtrip(reader)
40 | }
41 |
42 | }
43 |
--------------------------------------------------------------------------------
/connector/src/main/scala/com/datastax/spark/connector/rdd/ClusteringOrder.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.rdd
20 |
21 | import com.datastax.spark.connector.cql.TableDef
22 |
23 | sealed trait ClusteringOrder extends Serializable {
24 | private[connector] def toCql(tableDef: TableDef): String
25 | }
26 |
27 | object ClusteringOrder {
28 | private[connector] def cqlClause(tableDef: TableDef, order: String) =
29 | tableDef.clusteringColumns.headOption.map(cc => s"""ORDER BY "${cc.columnName}" $order""")
30 | .getOrElse(throw new IllegalArgumentException("Order by can be specified only if there are some clustering columns"))
31 |
32 | case object Ascending extends ClusteringOrder {
33 | override private[connector] def toCql(tableDef: TableDef): String = cqlClause(tableDef, "ASC")
34 | }
35 |
36 | case object Descending extends ClusteringOrder {
37 | override private[connector] def toCql(tableDef: TableDef): String = cqlClause(tableDef, "DESC")
38 | }
39 |
40 | }
41 |
--------------------------------------------------------------------------------
/connector/src/test/java/com/datastax/spark/connector/SampleJavaBeanWithoutNoArgsCtor.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector;
20 |
21 | import java.io.Serializable;
22 |
23 | /**
24 | * This is a sample JavaBean style class. In order to test JavaAPI correctly, we cannot implement this in Scala because
25 | * Scala adds some additional accessors and mutators.
26 | */
27 | public class SampleJavaBeanWithoutNoArgsCtor implements Serializable {
28 | private Integer key;
29 | private String value;
30 |
31 | private SampleJavaBeanWithoutNoArgsCtor(Integer key, String value) {
32 | this.key = key;
33 | this.value = value;
34 | }
35 |
36 | public Integer getKey() {
37 | return key;
38 | }
39 |
40 | public void setKey(Integer key) {
41 | this.key = key;
42 | }
43 |
44 | public String getValue() {
45 | return value;
46 | }
47 |
48 | public void setValue(String value) {
49 | this.value = value;
50 | }
51 | }
52 |
--------------------------------------------------------------------------------
/connector/src/main/scala/com/datastax/spark/connector/writer/BatchStatementBuilder.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.writer
20 |
21 | import com.datastax.oss.driver.api.core.ConsistencyLevel
22 | import com.datastax.oss.driver.api.core.cql.BatchType
23 | import com.datastax.spark.connector.util.Logging
24 |
25 | private[connector] class BatchStatementBuilder(
26 | val batchType: BatchType,
27 | val consistencyLevel: ConsistencyLevel) extends Logging {
28 |
29 | /** Converts a sequence of statements into a batch if its size is greater than 1.
30 | * Sets the routing key and consistency level. */
31 | def maybeCreateBatch(stmts: Seq[RichBoundStatementWrapper]): RichStatement = {
32 | require(stmts.nonEmpty, "Statements list cannot be empty")
33 | val stmt = stmts.head
34 |
35 | if (stmts.size == 1) {
36 | stmt.setConsistencyLevel(consistencyLevel)
37 | } else {
38 | new RichBatchStatementWrapper(batchType, consistencyLevel, stmts)
39 | }
40 | }
41 |
42 | }
43 |
--------------------------------------------------------------------------------
/driver/src/main/scala/com/datastax/spark/connector/types/TypeAdapters.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.types
20 |
21 | /**
22 | * Type adapters that serve as a middle step in conversion from one type to another.
23 | *
24 | * Adapters are utilized by types with scheme ([[UserDefinedType]]], [[TupleType]]) to convert an instance of
25 | * an type to corresponding adapter and than to final value of the given type.
26 | */
27 | private[spark] object TypeAdapters {
28 | /**
29 | * Adapter for multi-values types that my be returned as a sequence.
30 | *
31 | * It is used to extend conversion capabilities offered by Tuple type.
32 | */
33 | trait ValuesSeqAdapter {
34 | def toSeq(): Seq[Any]
35 | }
36 |
37 | /**
38 | * Adapter for multi-value types that may return values by name.
39 | *
40 | * It is used to extend conversion capabilities offered by UDT type.
41 | */
42 | trait ValueByNameAdapter {
43 | def getByName(name: String): Any
44 | }
45 | }
46 |
--------------------------------------------------------------------------------
/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraLimit.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.rdd
20 |
21 | sealed trait CassandraLimit
22 |
23 | case class CassandraPartitionLimit(rowsNumber: Long) extends CassandraLimit {
24 | require(rowsNumber > 0, s"$rowsNumber <= 0. Per Partition Limits must be greater than 0")
25 | }
26 | case class SparkPartitionLimit(rowsNumber: Long) extends CassandraLimit {
27 | require(rowsNumber > 0, s"$rowsNumber <= 0. Limits must be greater than 0")
28 | }
29 |
30 | object CassandraLimit {
31 |
32 | def limitToClause
33 | (limit: Option[CassandraLimit]): String = limit match {
34 | case Some(SparkPartitionLimit(rowsNumber)) => s"LIMIT $rowsNumber"
35 | case Some(CassandraPartitionLimit(rowsNumber)) => s"PER PARTITION LIMIT $rowsNumber"
36 | case None => ""
37 | }
38 |
39 | def limitForIterator(limit: Option[CassandraLimit]): Option[Long] = limit.collect {
40 | case SparkPartitionLimit(rowsNumber) => rowsNumber
41 | }
42 | }
43 |
44 |
--------------------------------------------------------------------------------
/connector/src/test/java/com/datastax/spark/connector/SampleJavaBeanSubClass.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector;
20 |
21 | /**
22 | * This is a sample JavaBean style class/subclass. In order to test JavaAPI correctly, we cannot
23 | * implement this in Scala because Scala adds some additional accessors and mutators.
24 | */
25 | public class SampleJavaBeanSubClass extends SampleJavaBean
26 | {
27 | private String subClassField;
28 |
29 | public static SampleJavaBeanSubClass newInstance(Integer key, String value, String subClassField) {
30 | SampleJavaBeanSubClass bean = new SampleJavaBeanSubClass();
31 | bean.setKey(key);
32 | bean.setValue(value);
33 | bean.setSubClassField(subClassField);
34 | return bean;
35 | }
36 |
37 | public String getSubClassField()
38 | {
39 | return subClassField;
40 | }
41 |
42 | public void setSubClassField(String subClassField)
43 | {
44 | this.subClassField = subClassField;
45 | }
46 | }
47 |
--------------------------------------------------------------------------------
/connector/src/test/scala/com/datastax/spark/connector/samples.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector {
20 |
21 | case class SampleScalaCaseClass(key: Int, value: String)
22 |
23 | class SampleScalaClass(val key: Int, val value: String) extends Serializable
24 |
25 | class SampleScalaClassWithNoFields(key: Int, value: String) extends Serializable
26 |
27 | class SampleScalaClassWithMultipleCtors(var key: Int, var value: String) extends Serializable {
28 | def this(key: Int) = this(key, null)
29 |
30 | def this() = this(0, null)
31 | }
32 |
33 | class SampleWithNestedScalaCaseClass extends Serializable {
34 |
35 | case class InnerClass(key: Int, value: String)
36 |
37 | }
38 |
39 | class SampleWithDeeplyNestedScalaCaseClass extends Serializable {
40 |
41 | class IntermediateClass extends Serializable {
42 |
43 | case class InnerClass(key: Int, value: String)
44 |
45 | }
46 |
47 | }
48 |
49 | object SampleObject {
50 |
51 | case class ClassInObject(key: Int, value: String)
52 |
53 | }
54 |
55 | }
--------------------------------------------------------------------------------
/connector/src/main/scala/com/datastax/spark/connector/util/Threads.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.util
20 |
21 | import java.util.concurrent.{Executors, LinkedBlockingQueue, ThreadPoolExecutor, TimeUnit}
22 |
23 | import com.google.common.util.concurrent.ThreadFactoryBuilder
24 |
25 | import scala.concurrent.{ExecutionContext, ExecutionContextExecutorService}
26 |
27 | object Threads extends Logging {
28 |
29 | implicit val BlockingIOExecutionContext: ExecutionContextExecutorService = {
30 | val threadFactory = new ThreadFactoryBuilder()
31 | .setDaemon(true)
32 | .setNameFormat("spark-cassandra-connector-io" + "%d")
33 | .setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler {
34 | override def uncaughtException(t: Thread, e: Throwable): Unit = {
35 | logWarning(s"Unhandled exception in thread ${t.getName}.", e)
36 | }
37 | })
38 | .build
39 | ExecutionContext.fromExecutorService(Executors.newCachedThreadPool(threadFactory))
40 | }
41 | }
42 |
43 |
--------------------------------------------------------------------------------
/driver/src/main/scala/com/datastax/spark/connector/util/DriverUtil.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.util
20 |
21 | import java.net.{InetAddress, InetSocketAddress}
22 | import java.util.Optional
23 |
24 | import com.datastax.oss.driver.api.core.CqlIdentifier
25 | import com.datastax.oss.driver.api.core.metadata.Node
26 |
27 | object DriverUtil {
28 |
29 | //TODO use CqlIdentifier instead? Use implicit conversion to String? To internal string?
30 | def toName(id: CqlIdentifier): String = id.asInternal()
31 |
32 | def toOption[T](optional: Optional[T]): Option[T] =
33 | if (optional.isPresent) Some(optional.get()) else None
34 |
35 | def toAddress(node: Node): Option[InetSocketAddress] = {
36 | node.getEndPoint.resolve() match {
37 | case address: InetSocketAddress => if (address.isUnresolved) {
38 | Option(new InetSocketAddress(address.getHostString, address.getPort))
39 | } else {
40 | Option(address)
41 | }
42 | case _ => toOption(node.getBroadcastAddress)
43 | }
44 | }
45 | }
46 |
--------------------------------------------------------------------------------
/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/TextTypeTest.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.rdd.typeTests
20 |
21 | import com.datastax.oss.driver.api.core.cql.Row
22 | import com.datastax.spark.connector.cluster.DefaultCluster
23 |
24 | class TextTypeTest extends AbstractTypeTest[String, String] with DefaultCluster {
25 | override val typeName = "text"
26 |
27 | override val typeData: Seq[String] = Seq("काचं शक्नोम्यत्तुम् । नोपहिनस्ति माम् ॥", "⠊⠀⠉⠁⠝⠀⠑⠁⠞⠀⠛⠇⠁⠎⠎⠀⠁⠝⠙⠀⠊⠞⠀⠙⠕⠑⠎⠝⠞⠀⠓⠥⠗⠞⠀⠍⠑", "אני יכול לאכול זכוכית וזה לא מזיק לי.", " நான் கண்ணாடி சாப்பிடுவேன், அதனால் எனக்கு ஒரு கேடும் வராது.", " ᠪᠢ ᠰᠢᠯᠢ ᠢᠳᠡᠶᠦ ᠴᠢᠳᠠᠨᠠ ᠂ ᠨᠠᠳᠤᠷ ᠬᠣᠤᠷᠠᠳᠠᠢ ᠪᠢᠰᠢ ")
28 | override val addData: Seq[String] = Seq(" ᚛᚛ᚉᚑᚅᚔᚉᚉᚔᚋ ᚔᚈᚔ ᚍᚂᚐᚅᚑ ᚅᚔᚋᚌᚓᚅᚐ᚜", "I kaun Gloos essen, es tuat ma ned weh.", " Meg tudom enni az üveget, nem lesz tőle bajom", "Можам да јадам стакло, а не ме штета.", "Կրնամ ապակի ուտել և ինծի անհանգիստ չըներ։")
29 |
30 | override def getDriverColumn(row: Row, colName: String): String = {
31 | row.getString(colName)
32 | }
33 | }
34 |
35 |
--------------------------------------------------------------------------------
/connector/src/test/java/com/datastax/spark/connector/SampleJavaBean.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector;
20 |
21 | import java.io.Serializable;
22 |
23 | /**
24 | * This is a sample JavaBean style class. In order to test JavaAPI correctly, we cannot implement this in Scala because
25 | * Scala adds some additional accessors and mutators.
26 | */
27 | public class SampleJavaBean implements Serializable {
28 | private Integer key;
29 | private String value;
30 |
31 | public static SampleJavaBean newInstance(Integer key, String value) {
32 | SampleJavaBean bean = new SampleJavaBean();
33 | bean.setKey(key);
34 | bean.setValue(value);
35 | return bean;
36 | }
37 |
38 | public Integer getKey() {
39 | return key;
40 | }
41 |
42 | public void setKey(Integer key) {
43 | this.key = key;
44 | }
45 |
46 | public String getValue() {
47 | return value;
48 | }
49 |
50 | public void setValue(String value) {
51 | this.value = value;
52 | }
53 | }
54 |
--------------------------------------------------------------------------------
/connector/src/main/scala/com/datastax/spark/connector/CassandraSparkExtensions.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector
20 |
21 | import org.apache.spark.sql.{SparkSessionExtensions, catalyst}
22 | import org.apache.spark.sql.cassandra.execution.CassandraDirectJoinStrategy
23 | import org.apache.spark.sql.cassandra.{CassandraMetaDataRule, CassandraMetadataFunction}
24 | import org.apache.spark.sql.catalyst.FunctionIdentifier
25 | import com.datastax.spark.connector.util.Logging
26 | import org.apache.spark.sql.catalyst.expressions.Expression
27 |
28 | class CassandraSparkExtensions extends (SparkSessionExtensions => Unit) with Logging {
29 | override def apply(extensions: SparkSessionExtensions): Unit = {
30 | extensions.injectPlannerStrategy(CassandraDirectJoinStrategy.apply)
31 | extensions.injectResolutionRule(session => CassandraMetaDataRule)
32 | extensions.injectFunction(CassandraMetadataFunction.cassandraTTLFunctionDescriptor)
33 | extensions.injectFunction(CassandraMetadataFunction.cassandraWriteTimeFunctionDescriptor)
34 | }
35 | }
36 |
--------------------------------------------------------------------------------
/driver/src/main/scala/com/datastax/spark/connector/japi/TupleValue.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.japi
20 |
21 | import com.datastax.spark.connector.types.{NullableTypeConverter, TypeConverter}
22 | import com.datastax.spark.connector.{TupleValue => ConnectorTupleValue}
23 |
24 | import scala.annotation.varargs
25 | import scala.reflect.runtime.universe._
26 |
27 | final class TupleValue private (val columnValues: IndexedSeq[AnyRef])
28 | extends JavaGettableByIndexData with Serializable
29 |
30 |
31 | object TupleValue {
32 |
33 | val TypeTag = typeTag[TupleValue]
34 |
35 | implicit object UDTValueConverter extends NullableTypeConverter[TupleValue] {
36 | def targetTypeTag = TypeTag
37 |
38 | def convertPF = {
39 | case x: TupleValue => x
40 | case x: ConnectorTupleValue =>
41 | new TupleValue(x.columnValues)
42 | }
43 | }
44 |
45 | TypeConverter.registerConverter(UDTValueConverter)
46 |
47 | @varargs
48 | def newTuple(values: Object*): TupleValue =
49 | new TupleValue(values.toIndexedSeq)
50 | }
51 |
52 |
--------------------------------------------------------------------------------
/connector/src/main/scala/com/datastax/spark/connector/rdd/SpannedByKeyRDD.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.rdd
20 |
21 | import org.apache.spark.{TaskContext, Partition}
22 | import org.apache.spark.annotation.DeveloperApi
23 | import org.apache.spark.rdd.RDD
24 |
25 | import com.datastax.spark.connector.util.SpanningIterator
26 |
27 | /**
28 | * Similar to [[SpannedRDD]] but, instead of extracting the key by the given function,
29 | * it groups binary tuples by the first element of each tuple.
30 | */
31 | private[connector] class SpannedByKeyRDD[K, V](parent: RDD[(K, V)]) extends RDD[(K, Seq[V])](parent) {
32 |
33 | override protected def getPartitions = parent.partitions
34 |
35 | @DeveloperApi
36 | override def compute(split: Partition, context: TaskContext) = {
37 | val parentIterator = parent.iterator(split, context)
38 | def keyFunction(item: (K, V)) = item._1
39 | def extractValues(group: (K, Seq[(K, V)])) = (group._1, group._2.map(_._2))
40 | new SpanningIterator(parentIterator, keyFunction).map(extractValues)
41 | }
42 |
43 | }
44 |
--------------------------------------------------------------------------------
/connector/src/test/java/com/datastax/spark/connector/SampleWeirdJavaBean.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector;
20 |
21 | import java.io.Serializable;
22 |
23 | /**
24 | * This is a sample JavaBean style class. In order to test JavaAPI correctly, we cannot implement this in Scala because
25 | * Scala adds some additional accessors and mutators.
26 | */
27 | public class SampleWeirdJavaBean implements Serializable {
28 | private Integer devil;
29 | private String cat;
30 |
31 | public static SampleWeirdJavaBean newInstance(Integer key, String value) {
32 | SampleWeirdJavaBean bean = new SampleWeirdJavaBean();
33 | bean.setDevil(key);
34 | bean.setCat(value);
35 | return bean;
36 | }
37 |
38 | public Integer getDevil() {
39 | return devil;
40 | }
41 |
42 | public void setDevil(Integer devil) {
43 | this.devil = devil;
44 | }
45 |
46 | public String getCat() {
47 | return cat;
48 | }
49 |
50 | public void setCat(String cat) {
51 | this.cat = cat;
52 | }
53 | }
54 |
--------------------------------------------------------------------------------
/connector/src/test/java/com/datastax/spark/connector/japi/rdd/CassandraJoinJavaRDDTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.japi.rdd;
20 |
21 | import static com.datastax.spark.connector.japi.CassandraJavaUtil.someColumns;
22 | import static org.hamcrest.CoreMatchers.is;
23 | import static org.hamcrest.MatcherAssert.assertThat;
24 | import org.junit.Test;
25 | import static org.mockito.Mockito.mock;
26 | import static org.mockito.Mockito.when;
27 |
28 | import com.datastax.spark.connector.rdd.CassandraJoinRDD;
29 |
30 | @SuppressWarnings({"unchecked", "RedundantTypeArguments"})
31 | public class CassandraJoinJavaRDDTest {
32 |
33 | @Test
34 | public void testOn() {
35 | CassandraJoinRDD rdd = mock(CassandraJoinRDD.class);
36 | CassandraJoinRDD rdd2 = mock(CassandraJoinRDD.class);
37 | when(rdd.on(someColumns("a", "b"))).thenReturn(rdd2);
38 | CassandraJoinJavaRDD jrdd = new CassandraJoinJavaRDD<>(rdd, String.class, Integer.class);
39 | assertThat(jrdd.on(someColumns("a", "b")).rdd(), is(rdd2));
40 | }
41 |
42 | }
43 |
--------------------------------------------------------------------------------
/connector/src/main/scala/com/datastax/spark/connector/writer/BatchGroupingKey.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.writer
20 |
21 | sealed trait BatchGroupingKey
22 |
23 | object BatchGroupingKey {
24 |
25 | /** Any row can be added to any batch. This works the same as previous batching implementation. */
26 | case object None extends BatchGroupingKey
27 |
28 | /** Each batch is associated with a set of replicas. If a set of replicas for the inserted row is
29 | * the same as it is for a batch, the row can be added to the batch. */
30 | case object ReplicaSet extends BatchGroupingKey
31 |
32 | /** Each batch is associated with a partition key. If the partition key of the inserted row is the
33 | * same as it is for a batch, the row can be added to the batch. */
34 | case object Partition extends BatchGroupingKey
35 |
36 | def apply(name: String): BatchGroupingKey = name.toLowerCase match {
37 | case "none" => None
38 | case "replica_set" => ReplicaSet
39 | case "partition" => Partition
40 | case _ => throw new IllegalArgumentException(s"Invalid batch level: $name")
41 | }
42 | }
43 |
--------------------------------------------------------------------------------
/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/Murmur3PartitionerTokenRangeSplitter.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.rdd.partitioner
20 |
21 | import com.datastax.spark.connector.rdd.partitioner.dht.LongToken
22 |
23 | /** Fast token range splitter assuming that data are spread out evenly in the whole range. */
24 | private[partitioner] class Murmur3PartitionerTokenRangeSplitter
25 | extends TokenRangeSplitter[Long, LongToken] {
26 |
27 | private type TokenRange = com.datastax.spark.connector.rdd.partitioner.dht.TokenRange[Long, LongToken]
28 |
29 | override def split(tokenRange: TokenRange, splitSize: Int): Seq[TokenRange] = {
30 | val rangeSize = tokenRange.rangeSize
31 | val splitPointsCount = if (rangeSize < splitSize) rangeSize.toInt else splitSize
32 | val splitPoints = (0 until splitPointsCount).map({ i =>
33 | new LongToken(tokenRange.start.value + (rangeSize * i / splitPointsCount).toLong)
34 | }) :+ tokenRange.end
35 |
36 | for (Seq(left, right) <- splitPoints.sliding(2).toSeq) yield
37 | new TokenRange(left, right, tokenRange.replicas, tokenRange.tokenFactory)
38 | }
39 | }
40 |
--------------------------------------------------------------------------------
/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/TimestampTypeTest.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.rdd.typeTests
20 |
21 | import java.text.SimpleDateFormat
22 | import java.time.Instant
23 | import java.util.Date
24 |
25 | import com.datastax.oss.driver.api.core.cql.Row
26 | import com.datastax.spark.connector.cluster.DefaultCluster
27 |
28 | class TimestampTypeTest extends AbstractTypeTest[Instant, Instant] with DefaultCluster {
29 | override val typeName = "timestamp"
30 | val sdf = new SimpleDateFormat("dd/MM/yyyy")
31 |
32 | override val typeData: Seq[Instant] = Seq(
33 | sdf.parse("03/08/1985"),
34 | sdf.parse("03/08/1986"),
35 | sdf.parse("03/08/1987"),
36 | sdf.parse("03/08/1988"),
37 | sdf.parse("03/08/1989")).map(_.toInstant)
38 | override val addData: Seq[Instant] = Seq(
39 | sdf.parse("03/08/1990"),
40 | sdf.parse("03/08/1991"),
41 | sdf.parse("03/08/1992"),
42 | sdf.parse("03/08/1993"),
43 | sdf.parse("03/08/1994")).map(_.toInstant)
44 |
45 | override def getDriverColumn(row: Row, colName: String): Instant = {
46 | row.getInstant(colName)
47 | }
48 |
49 | }
50 |
51 |
--------------------------------------------------------------------------------
/connector/src/test/java/com/datastax/spark/connector/SampleJavaBeanWithMultipleCtors.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector;
20 |
21 | import java.io.Serializable;
22 |
23 | /**
24 | * This is a sample JavaBean style class. In order to test JavaAPI correctly, we cannot implement this in Scala because
25 | * Scala adds some additional accessors and mutators.
26 | */
27 | public class SampleJavaBeanWithMultipleCtors implements Serializable {
28 | private Integer key;
29 | private String value;
30 |
31 | public SampleJavaBeanWithMultipleCtors(Integer key) {
32 | this.key = key;
33 | }
34 |
35 | public SampleJavaBeanWithMultipleCtors() {
36 | }
37 |
38 | public SampleJavaBeanWithMultipleCtors(Integer key, String value) {
39 | this.key = key;
40 | this.value = value;
41 | }
42 |
43 | public Integer getKey() {
44 | return key;
45 | }
46 |
47 | public void setKey(Integer key) {
48 | this.key = key;
49 | }
50 |
51 | public String getValue() {
52 | return value;
53 | }
54 |
55 | public void setValue(String value) {
56 | this.value = value;
57 | }
58 | }
59 |
--------------------------------------------------------------------------------
/connector/src/test/scala/com/datastax/spark/connector/rdd/CqlWhereClauseSpec.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.rdd
20 |
21 | import org.scalatest.{FlatSpec, Matchers}
22 |
23 | class CqlWhereClauseSpec extends FlatSpec with Matchers {
24 |
25 | it should "produce a string for each predicate" in {
26 | val clause = CqlWhereClause(Seq("x < ?", "y = ?"), Seq(1, "aaa"))
27 |
28 | clause.toString shouldBe "[[x < ?, 1],[y = ?, aaa]]"
29 | }
30 |
31 | it should "produce empty predicate string for empty predicate list" in {
32 | val clause = CqlWhereClause(Seq(), Seq())
33 |
34 | clause.toString shouldBe "[]"
35 | }
36 |
37 | it should "produce valid string for IN clause predicate" in {
38 | val clause = CqlWhereClause(Seq("x < ?", "z IN (?, ?)", "y IN (?, ?, ?)", "a = ?"), Seq(1, 2, 3, 4, 5, 6, 7))
39 |
40 | clause.toString shouldBe "[[x < ?, 1],[z IN (?, ?), (2, 3)],[y IN (?, ?, ?), (4, 5, 6)],[a = ?, 7]]"
41 | }
42 |
43 | it should "complain when the number of values doesn't match the number of placeholders '?'" in {
44 | intercept[AssertionError] {
45 | CqlWhereClause(Seq("x < ?"), Seq())
46 | }
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/driver/src/test/scala/com/datastax/spark/connector/types/CanBuildFromTest.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.types
20 |
21 | import org.apache.commons.lang3.SerializationUtils
22 | import org.junit.Assert._
23 | import org.junit.Test
24 |
25 | class CanBuildFromTest {
26 |
27 | @Test
28 | def testBuild() {
29 | val bf = CanBuildFrom.setCanBuildFrom[Int]
30 | val builder = bf.apply()
31 | builder += 1
32 | builder += 2
33 | builder += 3
34 | assertEquals(Set(1,2,3), builder.result())
35 | }
36 |
37 | @Test
38 | def testSerializeAndBuild() {
39 | val bf = CanBuildFrom.setCanBuildFrom[Int]
40 | val bf2 = SerializationUtils.roundtrip(bf)
41 | val builder = bf2.apply()
42 | builder += 1
43 | builder += 2
44 | builder += 3
45 | assertEquals(Set(1,2,3), builder.result())
46 | }
47 |
48 | @Test
49 | def testSerializeAndBuildWithOrdering() {
50 | val bf = CanBuildFrom.treeSetCanBuildFrom[Int]
51 | val bf2 = SerializationUtils.roundtrip(bf)
52 | val builder = bf2.apply()
53 | builder += 1
54 | builder += 2
55 | builder += 3
56 | assertEquals(Set(1,2,3), builder.result())
57 | }
58 |
59 |
60 | }
61 |
--------------------------------------------------------------------------------
/driver/src/test/scala/com/datastax/spark/connector/types/TimestampParserSpec.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.types
20 |
21 | import org.scalatest.FlatSpec
22 |
23 | class TimestampParserSpec extends FlatSpec {
24 |
25 | it should "parse fast all supported date[time[zone]] formats" in {
26 | /* look in [[DateTimeFormatter]] for 'X' definition*/
27 | val validZones = Set(
28 | "",
29 | "Z",
30 | "-08",
31 | "-0830",
32 | "-08:30",
33 | "-083015",
34 | "-08:30:15",
35 | "+08",
36 | "+0830",
37 | "+08:30",
38 | "+083015",
39 | "+08:30:15"
40 | )
41 |
42 | val validDates = Set(
43 | "1986-01-02",
44 | "1986-01-02 21:05",
45 | "1986-01-02 21:05:07",
46 | "1986-01-02 21:05:07.1",
47 | "1986-01-02 21:05:07.12",
48 | "1986-01-02 21:05:07.123"
49 | )
50 |
51 | val datesAndDatesWithT = validDates
52 | .flatMap(date => Set(date) + date.replace(' ', 'T'))
53 |
54 | val allDates = for (date <- datesAndDatesWithT; zone <- validZones) yield {
55 | date + zone
56 | }
57 |
58 | allDates.foreach(TimestampParser.parseFastOrThrow)
59 | }
60 | }
61 |
--------------------------------------------------------------------------------
/connector/src/main/scala/com/datastax/spark/connector/writer/QueryExecutor.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.writer
20 |
21 | import com.datastax.oss.driver.api.core.CqlSession
22 | import com.datastax.oss.driver.api.core.cql.AsyncResultSet
23 | import com.datastax.spark.connector.writer.AsyncExecutor.Handler
24 |
25 | class QueryExecutor(
26 | session: CqlSession,
27 | maxConcurrentQueries: Int,
28 | successHandler: Option[Handler[RichStatement]],
29 | failureHandler: Option[Handler[RichStatement]])
30 |
31 | extends AsyncExecutor[RichStatement, AsyncResultSet](
32 | stmt => session.executeAsync(stmt.stmt),
33 | maxConcurrentQueries,
34 | successHandler,
35 | failureHandler)
36 |
37 | object QueryExecutor {
38 |
39 | /**
40 | * Builds a query executor whose max requests per connection is limited to the MaxRequests per Connection
41 | */
42 | def apply(
43 | session: CqlSession,
44 | maxConcurrentQueries: Int,
45 | successHandler: Option[Handler[RichStatement]],
46 | failureHandler: Option[Handler[RichStatement]]): QueryExecutor = {
47 |
48 | new QueryExecutor(session, maxConcurrentQueries, successHandler, failureHandler)
49 | }
50 | }
51 |
--------------------------------------------------------------------------------
/connector/src/test/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 | log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR
34 |
35 | # Suppress some warnings
36 | log4j.logger.com.datastax.driver.core.NettyUtil=ERROR
37 | log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR
38 | log4j.logger.org.apache.cassandra.utils.CLibrary=ERROR
39 | log4j.logger.org.apache.cassandra.service.StartupChecks=ERROR
40 | log4j.logger.org.spark-project.jetty.server.Server=ERROR
41 | log4j.logger.org.eclipse.jetty.server.Server=ERROR
42 |
--------------------------------------------------------------------------------
/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/TimeUUIDTypeTest.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.rdd.typeTests
20 |
21 | import java.util.UUID
22 |
23 | import com.datastax.oss.driver.api.core.cql.Row
24 | import com.datastax.spark.connector.cluster.DefaultCluster
25 |
26 | class TimeUUIDTypeTest extends AbstractTypeTest[UUID, UUID] with DefaultCluster {
27 |
28 | override val typeName = "timeuuid"
29 |
30 | override val typeData: Seq[UUID] = Seq(UUID.fromString("61129590-FBE4-11E3-A3AC-0800200C9A66"), UUID.fromString("61129591-FBE4-11E3-A3AC-0800200C9A66"),
31 | UUID.fromString("61129592-FBE4-11E3-A3AC-0800200C9A66"), UUID.fromString("61129593-FBE4-11E3-A3AC-0800200C9A66"), UUID.fromString("61129594-FBE4-11E3-A3AC-0800200C9A66"))
32 | override val addData: Seq[UUID] = Seq(UUID.fromString("204FF380-FBE5-11E3-A3AC-0800200C9A66"), UUID.fromString("204FF381-FBE5-11E3-A3AC-0800200C9A66"), UUID.fromString("204FF382-FBE5-11E3-A3AC-0800200C9A66"), UUID.fromString("204FF383-FBE5-11E3-A3AC-0800200C9A66"), UUID.fromString("204FF384-FBE5-11E3-A3AC-0800200C9A66"))
33 |
34 | override def getDriverColumn(row: Row, colName: String): UUID = {
35 | row.getUuid(colName)
36 | }
37 | }
38 |
39 |
--------------------------------------------------------------------------------
/driver/src/test/java/com/datastax/spark/connector/mapper/ColumnMapperTestUDTBean.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.mapper;
20 |
21 | import com.datastax.oss.driver.api.mapper.annotations.CqlName;
22 |
23 | import java.io.Serializable;
24 |
25 | public class ColumnMapperTestUDTBean implements Serializable {
26 | public Integer field;
27 | @CqlName(value = "cassandra_another_field")
28 | public Integer anotherField;
29 | @CqlName(value = "cassandra_yet_another_field")
30 | public Integer completelyUnrelatedField;
31 |
32 | public Integer getField() {
33 | return field;
34 | }
35 |
36 | public void setField(Integer field) {
37 | this.field = field;
38 | }
39 |
40 | public Integer getAnotherField() {
41 | return anotherField;
42 | }
43 |
44 | public void setAnotherField(Integer anotherField) {
45 | this.anotherField = anotherField;
46 | }
47 |
48 | public Integer getCompletelyUnrelatedField() {
49 | return completelyUnrelatedField;
50 | }
51 |
52 | public void setCompletelyUnrelatedField(Integer completelyUnrelatedField) {
53 | this.completelyUnrelatedField = completelyUnrelatedField;
54 | }
55 | }
56 |
57 |
--------------------------------------------------------------------------------
/connector/src/main/scala/com/datastax/spark/connector/util/package.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector
20 |
21 | import com.datastax.dse.driver.api.core.auth.ProxyAuthentication
22 | import com.datastax.oss.driver.api.core.cql.Statement
23 | import com.datastax.spark.connector.cql.{CassandraConnector, Schema, TableDef}
24 |
25 | /** Useful stuff that didn't fit elsewhere. */
26 | package object util {
27 |
28 | def maybeExecutingAs[StatementT <: Statement[StatementT]](stmt: StatementT, proxyUser: Option[String]): StatementT = {
29 | proxyUser match {
30 | case Some(user) =>
31 | ProxyAuthentication.executeAs(user, stmt)
32 | case _ =>
33 | stmt
34 | }
35 | }
36 |
37 | def schemaFromCassandra(
38 | connector: CassandraConnector,
39 | keyspaceName: Option[String] = None,
40 | tableName: Option[String] = None): Schema = {
41 | connector.withSessionDo(Schema.fromCassandra(_, keyspaceName, tableName))
42 | }
43 |
44 | def tableFromCassandra(
45 | connector: CassandraConnector,
46 | keyspaceName: String,
47 | tableName: String): TableDef = {
48 | connector.withSessionDo(Schema.tableFromCassandra(_, keyspaceName, tableName))
49 | }
50 |
51 | }
52 |
--------------------------------------------------------------------------------
/connector/src/main/scala/com/datastax/spark/connector/streaming/StreamingContextFunctions.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.streaming
20 |
21 | import com.datastax.spark.connector.cql.CassandraConnector
22 | import com.datastax.spark.connector.rdd.{ReadConf, ValidRDDType}
23 | import org.apache.spark.streaming.StreamingContext
24 | import com.datastax.spark.connector.SparkContextFunctions
25 | import com.datastax.spark.connector.rdd.reader.RowReaderFactory
26 |
27 | /** Provides Cassandra-specific methods on `org.apache.spark.streaming.StreamingContext`.
28 | * @param ssc the Spark Streaming context
29 | */
30 | class StreamingContextFunctions (ssc: StreamingContext) extends SparkContextFunctions(ssc.sparkContext) {
31 | import scala.reflect.ClassTag
32 |
33 | override def cassandraTable[T](keyspace: String, table: String)(
34 | implicit
35 | connector: CassandraConnector = CassandraConnector(ssc.sparkContext),
36 | readConf: ReadConf = ReadConf.fromSparkConf(sc.getConf),
37 | ct: ClassTag[T],
38 | rrf: RowReaderFactory[T],
39 | ev: ValidRDDType[T]): CassandraStreamingRDD[T] = {
40 |
41 | new CassandraStreamingRDD[T](ssc, connector, keyspace, table, readConf = readConf)
42 | }
43 | }
44 |
--------------------------------------------------------------------------------
/connector/src/test/scala/com/datastax/spark/connector/embedded/SparkTemplate.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.embedded
20 |
21 | import java.nio.file.Files
22 |
23 | import org.apache.log4j.{Level, Logger}
24 | import org.apache.spark.SparkConf
25 |
26 | object SparkTemplate {
27 |
28 | val DefaultParallelism = 2
29 |
30 | /** Default configuration for [[org.apache.spark.SparkContext SparkContext]]. */
31 | private val _defaultConf = new SparkConf(true)
32 | .set("spark.cassandra.connection.keepAliveMS", "5000")
33 | .set("spark.cassandra.connection.timeoutMS", "30000")
34 | .set("spark.ui.showConsoleProgress", "false")
35 | .set("spark.ui.enabled", "false")
36 | .set("spark.cleaner.ttl", "3600")
37 | .set("spark.sql.extensions","com.datastax.spark.connector.CassandraSparkExtensions")
38 | .setMaster(sys.env.getOrElse("IT_TEST_SPARK_MASTER", s"local[$DefaultParallelism]"))
39 | .setAppName("Test")
40 |
41 |
42 | def defaultConf = _defaultConf.clone()
43 |
44 | def withoutLogging[T]( f: => T): T={
45 | val level = Logger.getRootLogger.getLevel
46 | Logger.getRootLogger.setLevel(Level.OFF)
47 | val ret = f
48 | Logger.getRootLogger.setLevel(level)
49 | ret
50 | }
51 |
52 | }
53 |
--------------------------------------------------------------------------------
/connector/src/main/scala/com/datastax/spark/connector/writer/CassandraRowWriter.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.writer
20 |
21 | import com.datastax.spark.connector.{ColumnRef, CassandraRow}
22 | import com.datastax.spark.connector.cql.TableDef
23 |
24 | /** A [[RowWriter]] that can write [[CassandraRow]] objects.*/
25 | class CassandraRowWriter(table: TableDef, selectedColumns: IndexedSeq[ColumnRef]) extends RowWriter[CassandraRow] {
26 |
27 | override val columnNames = selectedColumns.map(_.columnName)
28 |
29 | private val columns = columnNames.map(table.columnByName).toIndexedSeq
30 | private val converters = columns.map(_.columnType.converterToCassandra)
31 |
32 | override def readColumnValues(data: CassandraRow, buffer: Array[Any]) = {
33 | for ((c, i) <- columnNames.zipWithIndex) {
34 | val value = data.getRaw(c)
35 | val convertedValue = converters(i).convert(value)
36 | buffer(i) = convertedValue
37 | }
38 | }
39 | }
40 |
41 |
42 | object CassandraRowWriter {
43 |
44 | object Factory extends RowWriterFactory[CassandraRow] {
45 | override def rowWriter(table: TableDef, selectedColumns: IndexedSeq[ColumnRef]) =
46 | new CassandraRowWriter(table, selectedColumns)
47 | }
48 |
49 | }
50 |
--------------------------------------------------------------------------------
/connector/src/test/java/com/datastax/spark/connector/SampleWithNestedJavaBean.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector;
20 |
21 | import java.io.Serializable;
22 |
23 | /**
24 | * This is a sample JavaBean style class. In order to test JavaAPI correctly, we cannot implement this in Scala because
25 | * Scala adds some additional accessors and mutators.
26 | */
27 | public class SampleWithNestedJavaBean implements Serializable {
28 | public class InnerClass implements Serializable {
29 | private Integer key;
30 | private String value;
31 |
32 | public InnerClass(Integer key) {
33 | this.key = key;
34 | }
35 |
36 | public InnerClass() {
37 | }
38 |
39 | public InnerClass(Integer key, String value) {
40 | this.key = key;
41 | this.value = value;
42 | }
43 |
44 | public Integer getKey() {
45 | return key;
46 | }
47 |
48 | public void setKey(Integer key) {
49 | this.key = key;
50 | }
51 |
52 | public String getValue() {
53 | return value;
54 | }
55 |
56 | public void setValue(String value) {
57 | this.value = value;
58 | }
59 | }
60 | }
61 |
--------------------------------------------------------------------------------
/driver/src/main/scala/com/datastax/spark/connector/TupleValue.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector
20 |
21 | import com.datastax.oss.driver.api.core.data.{TupleValue => DriverTupleValue}
22 | import com.datastax.spark.connector.types.NullableTypeConverter
23 |
24 | import scala.reflect.runtime.universe._
25 |
26 | final case class TupleValue(values: Any*) extends ScalaGettableByIndexData with Product {
27 | override def columnValues = values.toIndexedSeq.map(_.asInstanceOf[AnyRef])
28 |
29 | override def productArity: Int = columnValues.size
30 |
31 | override def productElement(n: Int): Any = getRaw(n)
32 | }
33 |
34 | object TupleValue {
35 |
36 | def fromJavaDriverTupleValue
37 | (value: DriverTupleValue)
38 | : TupleValue = {
39 | val values =
40 | for (i <- 0 until value.getType.getComponentTypes.size()) yield
41 | GettableData.get(value, i)
42 | new TupleValue(values: _*)
43 | }
44 |
45 | val TypeTag = typeTag[TupleValue]
46 | val Symbol = typeOf[TupleValue].asInstanceOf[TypeRef].sym
47 |
48 | implicit object TupleValueConverter extends NullableTypeConverter[TupleValue] {
49 | def targetTypeTag = TypeTag
50 | def convertPF = {
51 | case x: TupleValue => x
52 | }
53 | }
54 | }
--------------------------------------------------------------------------------
/driver/src/main/scala/com/datastax/spark/connector/util/ByteBufferUtil.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.util
20 |
21 | import java.nio.ByteBuffer
22 |
23 | object ByteBufferUtil {
24 |
25 | /** Copies the remaining bytes of the buffer into the given array, starting from offset zero.
26 | * The array must have capacity to store all of the remaining bytes of the buffer.
27 | * The buffer's position remains untouched. */
28 | def copyBuffer(src: ByteBuffer, dest: Array[Byte]): Array[Byte] = {
29 | if (src.hasArray) {
30 | val length: Int = src.remaining
31 | val offset: Int = src.arrayOffset + src.position()
32 | System.arraycopy(src.array, offset, dest, 0, length)
33 | } else {
34 | src.duplicate.get(dest)
35 | }
36 | dest
37 | }
38 |
39 | /** Converts a byte buffer into an array.
40 | * The buffer's position remains untouched. */
41 | def toArray(buffer: ByteBuffer): Array[Byte] = {
42 | if (buffer.hasArray &&
43 | buffer.arrayOffset + buffer.position() == 0 &&
44 | buffer.remaining == buffer.array.length) {
45 | buffer.array
46 | } else {
47 | val dest = new Array[Byte](buffer.remaining)
48 | copyBuffer(buffer, dest)
49 | }
50 | }
51 | }
52 |
--------------------------------------------------------------------------------
/driver/src/test/scala/com/datastax/spark/connector/mapper/PropertyExtractorTest.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.mapper
20 |
21 | import org.junit.Assert._
22 | import org.junit.Test
23 |
24 | class PropertyExtractorTest {
25 |
26 | class TestClass(val field1: String, val field2: Int)
27 |
28 | @Test
29 | def testSimpleExtraction() {
30 | val testObject = new TestClass("a", 1)
31 | val propertyExtractor = new PropertyExtractor(classOf[TestClass], Seq("field1", "field2"))
32 | val result = propertyExtractor.extract(testObject)
33 | assertEquals(2, result.length)
34 | assertEquals("a", result(0))
35 | assertEquals(1, result(1))
36 | }
37 |
38 | @Test
39 | def testAvailableProperties() {
40 | val triedProperties = Seq("field1", "foo", "bar")
41 | val availableProperties = PropertyExtractor.availablePropertyNames(classOf[TestClass], triedProperties)
42 | assertEquals(Seq("field1"), availableProperties)
43 | }
44 |
45 | @Test(expected = classOf[NoSuchMethodException])
46 | def testWrongPropertyName() {
47 | val testObject = new TestClass("a", 1)
48 | val propertyExtractor = new PropertyExtractor(classOf[TestClass], Seq("foo"))
49 | propertyExtractor.extract(testObject)
50 | }
51 |
52 | }
53 |
--------------------------------------------------------------------------------
/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=1.10.0
24 | URL=https://repo1.maven.org/maven2/org/scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch-${SBT_VERSION}.jar
25 | JAR=sbt/sbt-launch-${SBT_VERSION}.jar
26 |
27 | # Download sbt launch jar if it hasn't been downloaded yet
28 | if [ ! -f ${JAR} ]; then
29 | # Download
30 | printf "Attempting to fetch sbt\n"
31 | JAR_DL=${JAR}.part
32 | if hash curl 2>/dev/null; then
33 | curl -f -L --progress-bar ${URL} -o ${JAR_DL} && mv ${JAR_DL} ${JAR}
34 | elif hash wget 2>/dev/null; then
35 | wget --progress=bar ${URL} -O ${JAR_DL} && mv ${JAR_DL} ${JAR}
36 | else
37 | printf "You do not have curl or wget installed, please install sbt manually from http://www.scala-sbt.org/\n"
38 | exit -1
39 | fi
40 | fi
41 | if [ ! -f ${JAR} ]; then
42 | # We failed to download
43 | printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n"
44 | exit -1
45 | fi
46 | printf "Launching sbt from ${JAR}\n"
47 | java \
48 | -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=256m \
49 | -jar ${JAR} \
50 | "$@"
51 |
--------------------------------------------------------------------------------
/connector/src/main/scala/com/datastax/spark/connector/util/SpanningIterator.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.util
20 |
21 | import scala.collection.mutable.ArrayBuffer
22 |
23 | /** An iterator that groups items having the same value of the given function (key).
24 | * To be included in the same group, items with the same key must be next to each other
25 | * in the original collection.
26 | *
27 | * `SpanningIterator` buffers internally one group at a time and the wrapped iterator
28 | * is consumed in a lazy way.
29 | *
30 | * Example:
31 | * {{{
32 | * val collection = Seq(1 -> "a", 1 -> "b", 1 -> "c", 2 -> "d", 2 -> "e")
33 | * val iterator = new SpanningIterator(collection.iterator, (x: (Int, String)) => x._1)
34 | * val result = iterator.toSeq // Seq(1 -> Seq("a", "b", "c"), 2 -> Seq("d", "e"))
35 | * }}}
36 | */
37 | class SpanningIterator[K, T](iterator: Iterator[T], f: T => K) extends Iterator[(K, Seq[T])] {
38 |
39 | private[this] val items = new BufferedIterator2(iterator)
40 |
41 | override def hasNext = items.hasNext
42 |
43 | override def next(): (K, Seq[T]) = {
44 | val key = f(items.head)
45 | val buffer = new ArrayBuffer[T]
46 | items.appendWhile(r => f(r) == key, buffer)
47 | (key, buffer.toSeq)
48 | }
49 | }
50 |
--------------------------------------------------------------------------------
/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} [T%X{TEST_GROUP_NO}] %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 | log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR
34 |
35 | # Suppress some warnings
36 | log4j.logger.com.datastax.driver.core.NettyUtil=ERROR
37 | log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR
38 | log4j.logger.org.apache.cassandra.utils.CLibrary=ERROR
39 | log4j.logger.org.apache.cassandra.service.StartupChecks=ERROR
40 | log4j.logger.org.spark-project.jetty.server.Server=ERROR
41 | log4j.logger.org.eclipse.jetty.server.Server=ERROR
42 |
43 | #See CCM Bridge INFO
44 | log4j.logger.com.datastax.spark.connector.ccm=INFO
45 |
--------------------------------------------------------------------------------
/driver/src/test/scala/com/datastax/spark/connector/types/CollectionColumnTypeSpec.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.types
20 |
21 | import org.scalatest.{FlatSpec, Matchers}
22 |
23 | class CollectionColumnTypeSpec extends FlatSpec with Matchers {
24 |
25 | val udt = UserDefinedType(
26 | name = "address",
27 | columns = IndexedSeq(
28 | UDTFieldDef("street", VarCharType),
29 | UDTFieldDef("number", IntType)
30 | )
31 | )
32 |
33 | "ListType" should "mark nested UDT types as frozen" in {
34 | ListType(udt).cqlTypeName shouldBe "list>"
35 | }
36 |
37 | it should "not mark non UDT types as frozen" in {
38 | ListType(IntType).cqlTypeName shouldBe "list"
39 | }
40 |
41 | "SetType" should "mark nested UDT types as frozen" in {
42 | SetType(udt).cqlTypeName shouldBe "set>"
43 | }
44 |
45 | it should "not mark non UDT types as frozen" in {
46 | SetType(IntType).cqlTypeName shouldBe "set"
47 | }
48 |
49 | "MapType" should "mark key UDT types as frozen" in {
50 | MapType(udt, IntType).cqlTypeName shouldBe "map, int>"
51 | }
52 |
53 | it should "mark value UDT types as frozen" in {
54 | MapType(IntType, udt).cqlTypeName shouldBe "map>"
55 | }
56 |
57 | }
58 |
--------------------------------------------------------------------------------
/connector/src/test/java/com/datastax/spark/connector/mapper/JavaTestUDTBean.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.mapper;
20 |
21 | import com.datastax.oss.driver.api.mapper.annotations.CqlName;
22 |
23 | import java.io.Serializable;
24 |
25 | /**
26 | * This Java POJO represents an UDT in Cassandra
27 | *
28 | * Created by soumya on 9/15/16.
29 | */
30 | public class JavaTestUDTBean implements Serializable {
31 | public Integer field;
32 | @CqlName(value = "cassandra_another_field")
33 | public Integer anotherField;
34 | @CqlName(value = "cassandra_yet_another_field")
35 | public Integer completelyUnrelatedField;
36 |
37 | public Integer getField() {
38 | return field;
39 | }
40 |
41 | public void setField(Integer field) {
42 | this.field = field;
43 | }
44 |
45 | public Integer getAnotherField() {
46 | return anotherField;
47 | }
48 |
49 | public void setAnotherField(Integer anotherField) {
50 | this.anotherField = anotherField;
51 | }
52 |
53 | public Integer getCompletelyUnrelatedField() {
54 | return completelyUnrelatedField;
55 | }
56 |
57 | public void setCompletelyUnrelatedField(Integer completelyUnrelatedField) {
58 | this.completelyUnrelatedField = completelyUnrelatedField;
59 | }
60 | }
61 |
--------------------------------------------------------------------------------
/driver/src/main/scala/com/datastax/spark/connector/mapper/PropertyExtractor.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.mapper
20 |
21 | import java.lang.reflect.Method
22 |
23 | import scala.util.Try
24 |
25 | /** Extracts values from fields of an object. */
26 | class PropertyExtractor[T](val cls: Class[T], val propertyNames: Seq[String]) extends Serializable {
27 |
28 | private def getter(name: String) =
29 | cls.getMethod(name)
30 |
31 | @transient
32 | private lazy val methods: Array[Method] =
33 | propertyNames.map(getter).toArray
34 |
35 | @transient
36 | private lazy val methodByName =
37 | methods.map(m => (m.getName, m)).toMap
38 |
39 | def extract(obj: T): Array[AnyRef] =
40 | extract(obj, Array.ofDim(methods.length))
41 |
42 | def extract(obj: T, target: Array[AnyRef]): Array[AnyRef] = {
43 | for (i <- methods.indices)
44 | target(i) = methods(i).invoke(obj)
45 | target
46 | }
47 |
48 | def extractProperty(obj: T, propertyName: String): AnyRef = {
49 | val m = methodByName(propertyName)
50 | m.invoke(obj)
51 | }
52 | }
53 |
54 | object PropertyExtractor {
55 |
56 | def availablePropertyNames(cls: Class[_], requestedPropertyNames: Seq[String]): Seq[String] =
57 | requestedPropertyNames.filter(name => Try(cls.getMethod(name)).isSuccess)
58 |
59 | }
60 |
--------------------------------------------------------------------------------
/test-support/src/main/scala/com/datastax/spark/connector/ccm/mode/DebugModeExecutor.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.ccm.mode
20 |
21 | import java.nio.file.{Files, Path, Paths}
22 |
23 | import com.datastax.spark.connector.ccm.CcmConfig
24 | import org.slf4j.{Logger, LoggerFactory}
25 |
26 | private[ccm] class DebugModeExecutor(val config: CcmConfig) extends DefaultExecutor {
27 |
28 | private val logger: Logger = LoggerFactory.getLogger(classOf[StandardModeExecutor])
29 |
30 | private val Cwd = Paths.get("").toAbsolutePath().toString();
31 |
32 | override val dir: Path = {
33 | sys.env.get("PRESERVE_LOGS") match {
34 | case Some(dir) =>
35 | val subPath = s"$Cwd/$dir/ccm_${config.ipPrefix
36 | .replace(".","_")
37 | .stripSuffix("_")}"
38 |
39 | val path = Files.createDirectories(Paths.get(subPath))
40 | logger.debug(s"Preserving CCM Install Directory at [$path]. It will not be removed")
41 | logger.debug(s"Checking directory exists [${Files.exists(path)}]")
42 | path
43 | case None =>
44 | val tmp = Files.createTempDirectory("ccm")
45 | tmp.toFile.deleteOnExit()
46 | tmp
47 | }
48 | }
49 |
50 | // stop nodes, don't remove logs
51 | override def remove(): Unit = {
52 | execute("stop")
53 | }
54 |
55 | }
56 |
--------------------------------------------------------------------------------
/connector/src/test/java/com/datastax/spark/connector/japi/SparkContextJavaFunctionsTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.japi;
20 |
21 | import com.datastax.oss.driver.api.core.DefaultConsistencyLevel;
22 | import com.datastax.spark.connector.rdd.ReadConf;
23 | import org.apache.spark.SparkConf;
24 | import org.apache.spark.SparkContext;
25 | import org.junit.Test;
26 | import org.mockito.Mockito;
27 |
28 | import static org.junit.Assert.assertEquals;
29 | import static org.mockito.Mockito.when;
30 |
31 | public class SparkContextJavaFunctionsTest
32 | {
33 | @Test
34 | public void testReadConfPopulating() {
35 | SparkConf conf = new SparkConf();
36 | conf.set("spark.cassandra.input.fetch.sizeInRows", "1234");
37 | conf.set("spark.cassandra.input.split.sizeInMB", "4321");
38 | conf.set("spark.cassandra.input.consistency.level", "THREE");
39 |
40 | SparkContext sc = Mockito.mock(SparkContext.class);
41 | when(sc.getConf()).thenReturn(conf);
42 |
43 | ReadConf readConf = CassandraJavaUtil.javaFunctions(sc).cassandraTable("a", "b").rdd().readConf();
44 |
45 | assertEquals(readConf.fetchSizeInRows(), 1234);
46 | assertEquals(readConf.splitSizeInMB(), 4321);
47 | assertEquals(readConf.consistencyLevel(), DefaultConsistencyLevel.THREE);
48 | }
49 | }
50 |
--------------------------------------------------------------------------------
/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/TimeTypeTest.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.rdd.typeTests
20 |
21 | import java.time.LocalTime
22 |
23 | import com.datastax.spark.connector._
24 |
25 | import com.datastax.oss.driver.api.core.DefaultProtocolVersion
26 | import com.datastax.oss.driver.api.core.cql.Row
27 | import com.datastax.spark.connector.cluster.DefaultCluster
28 |
29 | class TimeTypeTest extends AbstractTypeTest[LocalTime, LocalTime] with DefaultCluster {
30 |
31 | override val minPV = DefaultProtocolVersion.V4
32 |
33 | override def getDriverColumn(row: Row, colName: String): LocalTime = row.getLocalTime(colName)
34 |
35 | override protected val typeName: String = "time"
36 |
37 | override protected val typeData: Seq[LocalTime] = (1L to 5L).map(LocalTime.ofNanoOfDay)
38 | override protected val addData: Seq[LocalTime] = (6L to 10L).map(LocalTime.ofNanoOfDay)
39 |
40 | "Time Types" should "be writable as dates" in skipIfProtocolVersionLT(minPV) {
41 | val times = (100 to 500 by 100).map(LocalTime.ofNanoOfDay(_))
42 | sc.parallelize(times.map(x => (x, x, x, x))).saveToCassandra(keyspaceName, typeNormalTable)
43 | val results = sc.cassandraTable[(LocalTime, LocalTime, LocalTime, LocalTime)](keyspaceName, typeNormalTable).collect
44 | checkNormalRowConsistency(times, results)
45 | }
46 |
47 | }
48 |
--------------------------------------------------------------------------------
/connector/src/main/scala/com/datastax/spark/connector/cql/QueryUtils.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.cql
20 |
21 | import java.nio.ByteBuffer
22 |
23 | import com.datastax.oss.driver.api.core.cql.BoundStatement
24 | import com.datastax.spark.connector.writer.NullKeyColumnException
25 |
26 | import scala.jdk.CollectionConverters._
27 |
28 | object QueryUtils {
29 | /**
30 | * If a bound statement has all partition key components bound it will
31 | * return a routing key, but if all components are not bound it returns
32 | * null. When this is the case we want to let the user know which columns
33 | * were not correctly bound
34 | * @param bs a statement completely bound with all parameters
35 | * @return The routing key
36 | */
37 | def getRoutingKeyOrError(bs: BoundStatement): ByteBuffer = {
38 | val routingKey = bs.getRoutingKey
39 | if (routingKey == null) throw new NullKeyColumnException(nullPartitionKeyValues(bs))
40 | routingKey
41 | }
42 |
43 | private def nullPartitionKeyValues(bs: BoundStatement) = {
44 | val pkIndicies = bs.getPreparedStatement.getPartitionKeyIndices
45 | val boundValues = bs.getValues
46 | pkIndicies.asScala
47 | .filter(bs.isNull(_))
48 | .map(bs.getPreparedStatement.getVariableDefinitions.get(_))
49 | .map(_.getName)
50 | .mkString(", ")
51 | }
52 |
53 |
54 | }
55 |
--------------------------------------------------------------------------------
/connector/src/main/scala/com/datastax/spark/connector/rdd/SpannedRDD.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.rdd
20 |
21 | import org.apache.spark.{TaskContext, Partition}
22 | import org.apache.spark.annotation.DeveloperApi
23 | import org.apache.spark.rdd.RDD
24 |
25 | import com.datastax.spark.connector.util.SpanningIterator
26 |
27 | /**
28 | * Groups items with the same key, assuming items with the same key are next to each other in
29 | * the parent collection. Contrary to Spark GroupedRDD, it does not perform shuffle, therefore it
30 | * is much faster. A key for each item is obtained by calling a given function.
31 | *
32 | * This RDD is very useful for grouping data coming out from Cassandra, because they are already
33 | * coming in order of partitioning key i.e. it is not possible for two rows
34 | * with the same partition key to be in different Spark partitions.
35 | *
36 | * @param parent parent RDD
37 | * @tparam K type of keys
38 | * @tparam T type of elements to be grouped together
39 | */
40 | private[connector] class SpannedRDD[K, T](parent: RDD[T], f: T => K)
41 | extends RDD[(K, Iterable[T])](parent) {
42 |
43 | override protected def getPartitions = parent.partitions
44 |
45 | @DeveloperApi
46 | override def compute(split: Partition, context: TaskContext) =
47 | new SpanningIterator(parent.iterator(split, context), f)
48 |
49 | }
50 |
51 |
--------------------------------------------------------------------------------
/connector/src/test/scala/com/datastax/bdp/spark/DseAuthConfFactorySpec.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.bdp.spark
20 | /**
21 | TODO:
22 | import org.apache.commons.lang3.SerializationUtils
23 | import org.apache.spark.SparkConf
24 | import org.scalatest.{FlatSpec, Matchers}
25 |
26 | import com.datastax.bdp.config.{ClientConfigurationFactory, YamlClientConfiguration}
27 | import com.datastax.bdp.spark.DseAuthConfFactory.DsePasswordAuthConf
28 | import com.datastax.bdp.test.ng.{DataGenerator, DseScalaTestBase, ToString, YamlProvider}
29 |
30 | class DseAuthConfFactorySpec extends FlatSpec with Matchers with DseScalaTestBase {
31 |
32 | beforeClass {
33 | YamlProvider.provideDefaultYamls()
34 | YamlClientConfiguration.setAsClientConfigurationImpl()
35 | }
36 |
37 | it should "produce equivalent AuthConf instances for the same SparkConf" in {
38 | def genAuthConf = DseAuthConfFactory.authConf(new SparkConf())
39 |
40 | genAuthConf shouldBe genAuthConf
41 | }
42 |
43 | it should "produce comparable DsePasswordAuthConf instances" in {
44 | val gen = new DataGenerator()
45 | val cases = gen.generate[DsePasswordAuthConf]()
46 | for (c <- cases) {
47 | withClue(s"Comparing ${ToString.toStringWithNames(c)} failed") {
48 | val duplicate = SerializationUtils.roundtrip(c)
49 | duplicate shouldBe c
50 | }
51 | }
52 | }
53 | }
54 | **/
55 |
--------------------------------------------------------------------------------
/connector/src/test/java/com/datastax/spark/connector/mapper/JavaTestBean.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.mapper;
20 |
21 | import com.datastax.oss.driver.api.mapper.annotations.CqlName;
22 | import com.datastax.oss.driver.api.mapper.annotations.Entity;
23 |
24 | import java.io.Serializable;
25 |
26 | /**
27 | * This is a Java Bean style class with Java Driver style annotations built in.
28 | * This class also contains nested UDTs with its own mappings
29 | */
30 | @Entity
31 | public class JavaTestBean implements Serializable {
32 |
33 | @CqlName(value = "cassandra_property_1")
34 | public Integer property1;
35 | @CqlName(value = "cassandra_camel_case_property")
36 | public Integer camelCaseProperty;
37 | public JavaTestUDTBean nested;
38 |
39 | public int getProperty1() {
40 | return property1;
41 | }
42 |
43 | public void setProperty1(int property1) {
44 | this.property1 = property1;
45 | }
46 |
47 | public int getCamelCaseProperty() {
48 | return camelCaseProperty;
49 | }
50 |
51 | public void setCamelCaseProperty(int camelCaseProperty) {
52 | this.camelCaseProperty = camelCaseProperty;
53 | }
54 |
55 | public JavaTestUDTBean getNested() {
56 | return nested;
57 | }
58 |
59 | public void setNested(JavaTestUDTBean nested) {
60 | this.nested = nested;
61 | }
62 |
63 | }
64 |
--------------------------------------------------------------------------------
/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/RandomPartitionerTokenRangeSplitter.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.rdd.partitioner
20 |
21 | import com.datastax.spark.connector.rdd.partitioner.dht.BigIntToken
22 |
23 | /** Fast token range splitter assuming that data are spread out evenly in the whole range. */
24 | private[partitioner] class RandomPartitionerTokenRangeSplitter
25 | extends TokenRangeSplitter[BigInt, BigIntToken] {
26 |
27 | private type TokenRange = com.datastax.spark.connector.rdd.partitioner.dht.TokenRange[BigInt, BigIntToken]
28 |
29 | private def wrapWithMax(max: BigInt)(token: BigInt): BigInt = {
30 | if (token <= max) token else token - max
31 | }
32 |
33 | override def split(tokenRange: TokenRange, splitCount: Int): Seq[TokenRange] = {
34 | val rangeSize = tokenRange.rangeSize
35 | val wrap = wrapWithMax(tokenRange.tokenFactory.maxToken.value)(_)
36 |
37 | val splitPointsCount = if (rangeSize < splitCount) rangeSize.toInt else splitCount
38 | val splitPoints = (0 until splitPointsCount).map({ i =>
39 | val nextToken: BigInt = tokenRange.start.value + (rangeSize * i / splitPointsCount)
40 | new BigIntToken(wrap(nextToken))
41 | }) :+ tokenRange.end
42 |
43 | for (Seq(left, right) <- splitPoints.sliding(2).toSeq) yield
44 | new TokenRange(left, right, tokenRange.replicas, tokenRange.tokenFactory)
45 | }
46 | }
--------------------------------------------------------------------------------
/connector/src/main/scala/com/datastax/spark/connector/writer/DefaultRowWriter.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.writer
20 |
21 | import scala.reflect.runtime.universe._
22 | import scala.collection.Seq
23 | import com.datastax.spark.connector.ColumnRef
24 | import com.datastax.spark.connector.cql.TableDef
25 | import com.datastax.spark.connector.mapper.{ColumnMapper, MappedToGettableDataConverter}
26 |
27 | /** A `RowWriter` suitable for saving objects mappable by a [[com.datastax.spark.connector.mapper.ColumnMapper ColumnMapper]].
28 | * Can save case class objects, java beans and tuples. */
29 | class DefaultRowWriter[T : TypeTag : ColumnMapper](
30 | table: TableDef,
31 | selectedColumns: IndexedSeq[ColumnRef])
32 | extends RowWriter[T] {
33 |
34 | private val converter = MappedToGettableDataConverter[T](table, selectedColumns)
35 | override val columnNames = selectedColumns.map(_.columnName)
36 |
37 | override def readColumnValues(data: T, buffer: Array[Any]) = {
38 | val row = converter.convert(data)
39 | for (i <- columnNames.indices)
40 | buffer(i) = row.getRaw(i)
41 | }
42 | }
43 |
44 | object DefaultRowWriter {
45 |
46 | def factory[T : ColumnMapper : TypeTag] = new RowWriterFactory[T] {
47 | override def rowWriter(tableDef: TableDef, selectedColumns: IndexedSeq[ColumnRef]) = {
48 | new DefaultRowWriter[T](tableDef, selectedColumns)
49 | }
50 | }
51 | }
52 |
53 |
--------------------------------------------------------------------------------
/connector/src/test/scala/com/datastax/spark/connector/cql/DefaultConnectionFactoryTest.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.cql
20 |
21 | import java.io.IOException
22 |
23 | import org.apache.spark.SparkEnv
24 | import org.mockito.Mockito
25 | import org.mockito.invocation.InvocationOnMock
26 | import org.mockito.stubbing.Answer
27 | import org.scalatest.FlatSpec
28 | import org.scalatestplus.mockito.MockitoSugar
29 |
30 | class DefaultConnectionFactoryTest extends FlatSpec with MockitoSugar {
31 |
32 | /** DefaultConnectionFactory relies on a non-null SparkEnv */
33 | private def mockedSparkEnv[T](code: => T): T = {
34 | val original = SparkEnv.get
35 | val sparkEnv = Mockito.mock(classOf[SparkEnv], new Answer[Option[String]] {
36 | override def answer(invocation: InvocationOnMock): Option[String] = None
37 | })
38 | SparkEnv.set(sparkEnv)
39 | try {
40 | code
41 | } finally {
42 | SparkEnv.set(original)
43 | }
44 | }
45 |
46 | it should "complain when a malformed URL is provided" in mockedSparkEnv {
47 | intercept[IOException] {
48 | DefaultConnectionFactory.maybeGetLocalFile("secure-bundle.zip")
49 | }
50 | }
51 |
52 | it should "complain when an URL with unrecognized scheme is provided" in mockedSparkEnv {
53 | intercept[IOException] {
54 | DefaultConnectionFactory.maybeGetLocalFile("hdfs:///secure-bundle.zip")
55 | }
56 | }
57 |
58 | }
59 |
--------------------------------------------------------------------------------
/driver/src/test/scala/com/datastax/spark/connector/types/TypeSerializationTest.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector.types
20 |
21 | import org.apache.commons.lang3.SerializationUtils
22 | import org.junit.Assert._
23 | import org.junit.Test
24 |
25 | class TypeSerializationTest {
26 |
27 | private def testSerialization(t: ColumnType[_]) {
28 | assertEquals(t, SerializationUtils.roundtrip(t))
29 | }
30 |
31 | @Test
32 | def testSerializationOfPrimitiveTypes() {
33 | testSerialization(AsciiType)
34 | testSerialization(TextType)
35 | testSerialization(IntType)
36 | testSerialization(BigIntType)
37 | testSerialization(DoubleType)
38 | testSerialization(FloatType)
39 | testSerialization(BooleanType)
40 | testSerialization(UUIDType)
41 | testSerialization(TimeUUIDType)
42 | testSerialization(TimestampType)
43 | testSerialization(DecimalType)
44 | testSerialization(BigIntType)
45 | testSerialization(InetType)
46 | testSerialization(CounterType)
47 | testSerialization(SmallIntType)
48 | testSerialization(TinyIntType)
49 | testSerialization(DateType)
50 | }
51 |
52 | @Test
53 | def testSerializationOfCollectionTypes() {
54 | testSerialization(ListType(IntType))
55 | testSerialization(ListType(ListType(IntType)))
56 | testSerialization(SetType(TextType))
57 | testSerialization(MapType(BigIntType, TimestampType))
58 | }
59 |
60 |
61 | }
62 |
--------------------------------------------------------------------------------
/connector/src/test/java/com/datastax/spark/connector/SampleWithDeeplyNestedJavaBean.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package com.datastax.spark.connector;
20 |
21 | import java.io.Serializable;
22 |
23 | /**
24 | * This is a sample JavaBean style class. In order to test JavaAPI correctly, we cannot implement this in Scala because
25 | * Scala adds some additional accessors and mutators.
26 | */
27 | public class SampleWithDeeplyNestedJavaBean implements Serializable {
28 | public class IntermediateClass implements Serializable {
29 | public class InnerClass implements Serializable {
30 | private Integer key;
31 | private String value;
32 |
33 | public InnerClass(Integer key) {
34 | this.key = key;
35 | }
36 |
37 | public InnerClass() {
38 | }
39 |
40 | public InnerClass(Integer key, String value) {
41 | this.key = key;
42 | this.value = value;
43 | }
44 |
45 | public Integer getKey() {
46 | return key;
47 | }
48 |
49 | public void setKey(Integer key) {
50 | this.key = key;
51 | }
52 |
53 | public String getValue() {
54 | return value;
55 | }
56 |
57 | public void setValue(String value) {
58 | this.value = value;
59 | }
60 | }
61 | }
62 | }
63 |
--------------------------------------------------------------------------------