From 22a852eceead8141c8c4944a7f49de50babb07fb Mon Sep 17 00:00:00 2001 From: SamTheisens <1911436+SamTheisens@users.noreply.github.com> Date: Tue, 11 Jul 2023 11:08:58 +0800 Subject: [PATCH 01/11] [SPARKC-686] Add support for scala 2.13 while keeping compatibility with scala 2.12 - Applied automatated scala collection migration via [scalafix] (https://github.com/scala/scala-collection-compat) - Conditionally add dependencies on `org.scala-lang.modules.scala-collection-compat` and `org.scala-lang.modules.scala-parallel-collections` for scala 2.13 - Scala version specific implementations of `SparkILoop`, `ParIterable`, `GenericJavaRowReaderFactory` and `CanBuildFrom`. branch: feature/SPARKC-686-scala-213-support --- .github/workflows/main.yml | 2 +- build.sbt | 29 ++++- .../connector/CassandraJavaUtilSpec.scala | 72 ++++++------ .../SparkCassandraITFlatSpecBase.scala | 7 +- .../CassandraAuthenticatedConnectorSpec.scala | 2 +- .../rdd/CassandraJavaPairRDDSpec.scala | 16 +-- .../connector/rdd/CassandraJavaRDDSpec.scala | 85 +++++++------- .../connector/rdd/CassandraRDDSpec.scala | 12 +- .../rdd/CustomTableScanMethodSpec.scala | 3 +- .../spark/connector/rdd/RDDSpec.scala | 5 +- .../rdd/partitioner/TokenGeneratorSpec.scala | 5 +- .../rdd/typeTests/AbstractTypeTest.scala | 4 +- .../rdd/typeTests/BlobTypeTest.scala | 4 +- .../sql/CassandraDataFrameSpec.scala | 14 +-- .../connector/util/MultiThreadedSpec.scala | 2 +- .../writer/GroupingBatchBuilderSpec.scala | 18 +-- .../connector/writer/TableWriterSpec.scala | 107 +++++++++--------- .../execution/CassandraDirectJoinSpec.scala | 2 +- .../connector/japi/PairRDDJavaFunctions.java | 2 +- .../japi/rdd/CassandraJavaPairRDD.java | 4 +- .../connector/japi/rdd/CassandraJavaRDD.java | 4 +- .../japi/rdd/CassandraTableScanJavaRDD.java | 4 +- .../japi/GenericJavaRowReaderFactory.java | 0 .../spark/connector/util/RuntimeUtil.scala | 25 ++++ .../japi/GenericJavaRowReaderFactory.java | 47 ++++++++ .../spark/connector/util/RuntimeUtil.scala | 19 ++++ .../datasource/CassandraCatalog.scala | 4 +- .../datasource/CassandraSourceUtil.scala | 5 +- .../connector/rdd/CassandraCoGroupedRDD.scala | 4 +- .../connector/rdd/CassandraMergeJoinRDD.scala | 4 +- .../connector/rdd/CassandraTableScanRDD.scala | 4 +- .../connector/rdd/DseGraphUnionedRDD.scala | 10 +- .../rdd/partitioner/BucketingRangeIndex.scala | 2 +- .../CassandraPartitionGenerator.scala | 6 +- .../rdd/partitioner/DataSizeEstimates.scala | 28 ++--- .../rdd/partitioner/NodeAddresses.scala | 4 +- .../rdd/partitioner/ReplicaPartitioner.scala | 4 +- .../rdd/partitioner/TokenRangeSplitter.scala | 8 +- .../spark/connector/util/JavaApiHelper.scala | 10 +- .../connector/util/MergeJoinIterator.scala | 2 +- .../util/MultiMergeJoinIterator.scala | 2 +- .../connector/util/SpanningIterator.scala | 2 +- .../spark/connector/writer/Batch.scala | 2 +- .../writer/ObjectSizeEstimator.scala | 8 +- .../connector/writer/ReplicaLocator.scala | 4 +- .../spark/connector/writer/TableWriter.scala | 2 +- .../spark/connector/writer/WriteOption.scala | 2 +- .../apache/spark/metrics/CassandraSink.scala | 4 +- .../BasicCassandraPredicatePushDown.scala | 4 +- .../sql/cassandra/DsePredicateRules.scala | 3 +- .../CassandraDirectJoinStrategy.scala | 2 +- .../spark/connector/embedded/SparkRepl.scala | 7 +- .../metrics/InputMetricsUpdaterSpec.scala | 9 +- .../types/CanBuildFrom.scala | 0 .../spark/connector/types/CanBuildFrom.scala | 90 +++++++++++++++ .../spark/connector/GettableData.scala | 8 +- .../datastax/spark/connector/UDTValue.scala | 4 +- .../datastax/spark/connector/cql/Schema.scala | 2 +- .../GettableDataToMappedTypeConverter.scala | 2 +- .../spark/connector/types/ColumnType.scala | 2 +- .../spark/connector/types/TupleType.scala | 8 +- .../spark/connector/types/TypeConverter.scala | 10 +- .../connector/types/UserDefinedType.scala | 8 +- .../spark/connector/util/NameTools.scala | 8 +- .../connector/types/TypeConverterTest.scala | 2 +- project/Publishing.scala | 2 +- .../spark/connector/ccm/CcmBridge.scala | 2 +- 67 files changed, 503 insertions(+), 285 deletions(-) rename connector/src/main/{java => scala-2.12}/com/datastax/spark/connector/japi/GenericJavaRowReaderFactory.java (100%) create mode 100644 connector/src/main/scala-2.12/com/datastax/spark/connector/util/RuntimeUtil.scala create mode 100644 connector/src/main/scala-2.13/com/datastax/spark/connector/japi/GenericJavaRowReaderFactory.java create mode 100644 connector/src/main/scala-2.13/com/datastax/spark/connector/util/RuntimeUtil.scala rename driver/src/main/{scala/com/datastax/spark/connector => scala-2.12}/types/CanBuildFrom.scala (100%) create mode 100644 driver/src/main/scala-2.13/com/datastax/spark/connector/types/CanBuildFrom.scala diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 00de87fc1..dff6cb917 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -14,7 +14,7 @@ jobs: runs-on: ubuntu-latest strategy: matrix: - scala: [2.12.10] + scala: [2.12.11, 2.13.11] db-version: [6.8.13, 5.1.24, 3.11.10, 4.0-rc2] steps: diff --git a/build.sbt b/build.sbt index 5b7455036..9d79e02ae 100644 --- a/build.sbt +++ b/build.sbt @@ -4,7 +4,8 @@ import sbt.{Compile, moduleFilter, _} import sbtassembly.AssemblyPlugin.autoImport.assembly lazy val scala212 = "2.12.11" -lazy val supportedScalaVersions = List(scala212) +lazy val scala213 = "2.13.11" +lazy val supportedScalaVersions = List(scala212, scala213) // factor out common settings ThisBuild / scalaVersion := scala212 @@ -53,6 +54,17 @@ lazy val assemblySettings = Seq( }, ) +val scalaReleaseVersion = SettingKey[Int]("scalaReleaseVersion") +scalaReleaseVersion := { + val v = scalaVersion.value + CrossVersion.partialVersion(v).map(_._1.toInt).getOrElse { + throw new RuntimeException(s"could not get Scala release version from $v") + } +} + + + + lazy val commonSettings = Seq( // dependency updates check dependencyUpdatesFailBuild := true, @@ -60,9 +72,20 @@ lazy val commonSettings = Seq( fork := true, parallelExecution := true, testForkedParallel := false, - testOptions += Tests.Argument(TestFrameworks.JUnit, "-v") + testOptions += Tests.Argument(TestFrameworks.JUnit, "-v"), + libraryDependencies ++= { + CrossVersion.partialVersion(scalaVersion.value) match { + case Some((2, scalaMajor)) if scalaMajor == 13 => + Seq( + "org.scala-lang.modules" %% "scala-collection-compat" % "2.11.0", + "org.scala-lang.modules" %% "scala-parallel-collections" % "1.0.4" + ) + case _ => Seq() + } + } ) + val annotationProcessor = Seq( "-processor", "com.datastax.oss.driver.internal.mapper.processor.MapperProcessor" ) @@ -70,6 +93,7 @@ val annotationProcessor = Seq( def scalacVersionDependantOptions(scalaBinary: String): Seq[String] = scalaBinary match { case "2.11" => Seq() case "2.12" => Seq("-no-java-comments") //Scala Bug on inner classes, CassandraJavaUtil, + case "2.13" => Seq("-no-java-comments") //Scala Bug on inner classes, CassandraJavaUtil, } lazy val root = (project in file(".")) @@ -81,6 +105,7 @@ lazy val root = (project in file(".")) publish / skip := true ) + lazy val connector = (project in file("connector")) .configs(IntegrationTest) .settings(Defaults.itSettings: _*) //This and above enables the "it" suite diff --git a/connector/src/it/scala/com/datastax/spark/connector/CassandraJavaUtilSpec.scala b/connector/src/it/scala/com/datastax/spark/connector/CassandraJavaUtilSpec.scala index 0777c3e98..1466426aa 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/CassandraJavaUtilSpec.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/CassandraJavaUtilSpec.scala @@ -3,7 +3,7 @@ package com.datastax.spark.connector import com.datastax.spark.connector.ccm.CcmBridge import com.datastax.spark.connector.cluster.DefaultCluster -import scala.collection.JavaConversions._ +import scala.jdk.CollectionConverters._ import scala.concurrent.Future import org.apache.spark.rdd.RDD import com.datastax.spark.connector.cql.CassandraConnector @@ -117,9 +117,9 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu val rows = results.all() assert(rows.size() == 3) - assert(rows.exists(row ⇒ row.getString("value") == "one" && row.getInt("key") == 1)) - assert(rows.exists(row ⇒ row.getString("value") == "two" && row.getInt("key") == 2)) - assert(rows.exists(row ⇒ row.getString("value") == "three" && row.getInt("key") == 3)) + assert(rows.asScala.exists(row ⇒ row.getString("value") == "one" && row.getInt("key") == 1)) + assert(rows.asScala.exists(row ⇒ row.getString("value") == "two" && row.getInt("key") == 2)) + assert(rows.asScala.exists(row ⇒ row.getString("value") == "three" && row.getInt("key") == 3)) } @@ -140,9 +140,9 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu val rows = results.all() assert(rows.size() == 3) - assert(rows.exists(row ⇒ row.getString("value") == "one" && row.getInt("key") == 1)) - assert(rows.exists(row ⇒ row.getString("value") == "two" && row.getInt("key") == 2)) - assert(rows.exists(row ⇒ row.getString("value") == "three" && row.getInt("key") == 3)) + assert(rows.asScala.exists(row ⇒ row.getString("value") == "one" && row.getInt("key") == 1)) + assert(rows.asScala.exists(row ⇒ row.getString("value") == "two" && row.getInt("key") == 2)) + assert(rows.asScala.exists(row ⇒ row.getString("value") == "three" && row.getInt("key") == 3)) } it should "allow to save beans with transient fields to Cassandra" in { @@ -162,9 +162,9 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu val rows = results.all() assert(rows.size() == 3) - assert(rows.exists(row ⇒ row.getString("value") == "one" && row.getInt("key") == 1)) - assert(rows.exists(row ⇒ row.getString("value") == "two" && row.getInt("key") == 2)) - assert(rows.exists(row ⇒ row.getString("value") == "three" && row.getInt("key") == 3)) + assert(rows.asScala.exists(row ⇒ row.getString("value") == "one" && row.getInt("key") == 1)) + assert(rows.asScala.exists(row ⇒ row.getString("value") == "two" && row.getInt("key") == 2)) + assert(rows.asScala.exists(row ⇒ row.getString("value") == "three" && row.getInt("key") == 3)) } it should "allow to save beans with inherited fields to Cassandra" in { @@ -184,7 +184,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu val rows = results.all() rows should have size 3 - rows.map(row => (row.getString("value"), row.getInt("key"), row.getString("sub_class_field"))).toSet shouldBe Set( + rows.asScala.map(row => (row.getString("value"), row.getInt("key"), row.getString("sub_class_field"))).toSet shouldBe Set( ("one", 1, "a"), ("two", 2, "b"), ("three", 3, "c") @@ -210,9 +210,9 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu val rows = results.all() assert(rows.size() == 3) - assert(rows.exists(row ⇒ row.getString("value") == "one" && row.getInt("key") == 1)) - assert(rows.exists(row ⇒ row.getString("value") == "two" && row.getInt("key") == 2)) - assert(rows.exists(row ⇒ row.getString("value") == "three" && row.getInt("key") == 3)) + assert(rows.asScala.exists(row ⇒ row.getString("value") == "one" && row.getInt("key") == 1)) + assert(rows.asScala.exists(row ⇒ row.getString("value") == "two" && row.getInt("key") == 2)) + assert(rows.asScala.exists(row ⇒ row.getString("value") == "three" && row.getInt("key") == 3)) } it should "allow to read rows as Tuple1" in { @@ -222,7 +222,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu )).select( "c1" ) - .collect().head + .collect().asScala.head tuple shouldBe Tuple1( 1: Integer ) @@ -237,7 +237,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu )).select( "c1", "c2" ) - .collect().head + .collect().asScala.head tuple shouldBe Tuple2( 1: Integer, "2" @@ -254,7 +254,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu )).select( "c1", "c2", "c3" ) - .collect().head + .collect().asScala.head tuple shouldBe Tuple3( 1: Integer, "2", @@ -273,7 +273,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu )).select( "c1", "c2", "c3", "c4" ) - .collect().head + .collect().asScala.head tuple shouldBe Tuple4( 1: Integer, "2", @@ -294,7 +294,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu )).select( "c1", "c2", "c3", "c4", "c5" ) - .collect().head + .collect().asScala.head tuple shouldBe Tuple5( 1: Integer, "2", @@ -317,7 +317,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu )).select( "c1", "c2", "c3", "c4", "c5", "c6" ) - .collect().head + .collect().asScala.head tuple shouldBe Tuple6( 1: Integer, "2", @@ -342,7 +342,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu )).select( "c1", "c2", "c3", "c4", "c5", "c6", "c7" ) - .collect().head + .collect().asScala.head tuple shouldBe Tuple7( 1: Integer, "2", @@ -369,7 +369,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu )).select( "c1", "c2", "c3", "c4", "c5", "c6", "c7", "c8" ) - .collect().head + .collect().asScala.head tuple shouldBe Tuple8( 1: Integer, "2", @@ -398,7 +398,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu )).select( "c1", "c2", "c3", "c4", "c5", "c6", "c7", "c8", "c9" ) - .collect().head + .collect().asScala.head tuple shouldBe Tuple9( 1: Integer, "2", @@ -429,7 +429,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu )).select( "c1", "c2", "c3", "c4", "c5", "c6", "c7", "c8", "c9", "c10" ) - .collect().head + .collect().asScala.head tuple shouldBe Tuple10( 1: Integer, "2", @@ -462,7 +462,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu )).select( "c1", "c2", "c3", "c4", "c5", "c6", "c7", "c8", "c9", "c10", "c11" ) - .collect().head + .collect().asScala.head tuple shouldBe Tuple11( 1: Integer, "2", @@ -497,7 +497,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu )).select( "c1", "c2", "c3", "c4", "c5", "c6", "c7", "c8", "c9", "c10", "c11", "c12" ) - .collect().head + .collect().asScala.head tuple shouldBe Tuple12( 1: Integer, "2", @@ -534,7 +534,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu )).select( "c1", "c2", "c3", "c4", "c5", "c6", "c7", "c8", "c9", "c10", "c11", "c12", "c13" ) - .collect().head + .collect().asScala.head tuple shouldBe Tuple13( 1: Integer, "2", @@ -573,7 +573,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu )).select( "c1", "c2", "c3", "c4", "c5", "c6", "c7", "c8", "c9", "c10", "c11", "c12", "c13", "c14" ) - .collect().head + .collect().asScala.head tuple shouldBe Tuple14( 1: Integer, "2", @@ -614,7 +614,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu )).select( "c1", "c2", "c3", "c4", "c5", "c6", "c7", "c8", "c9", "c10", "c11", "c12", "c13", "c14", "c15" ) - .collect().head + .collect().asScala.head tuple shouldBe Tuple15( 1: Integer, "2", @@ -657,7 +657,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu )).select( "c1", "c2", "c3", "c4", "c5", "c6", "c7", "c8", "c9", "c10", "c11", "c12", "c13", "c14", "c15", "c16" ) - .collect().head + .collect().asScala.head tuple shouldBe Tuple16( 1: Integer, "2", @@ -702,7 +702,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu )).select( "c1", "c2", "c3", "c4", "c5", "c6", "c7", "c8", "c9", "c10", "c11", "c12", "c13", "c14", "c15", "c16", "c17" ) - .collect().head + .collect().asScala.head tuple shouldBe Tuple17( 1: Integer, "2", @@ -749,7 +749,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu )).select( "c1", "c2", "c3", "c4", "c5", "c6", "c7", "c8", "c9", "c10", "c11", "c12", "c13", "c14", "c15", "c16", "c17", "c18" ) - .collect().head + .collect().asScala.head tuple shouldBe Tuple18( 1: Integer, "2", @@ -798,7 +798,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu )).select( "c1", "c2", "c3", "c4", "c5", "c6", "c7", "c8", "c9", "c10", "c11", "c12", "c13", "c14", "c15", "c16", "c17", "c18", "c19" ) - .collect().head + .collect().asScala.head tuple shouldBe Tuple19( 1: Integer, "2", @@ -849,7 +849,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu )).select( "c1", "c2", "c3", "c4", "c5", "c6", "c7", "c8", "c9", "c10", "c11", "c12", "c13", "c14", "c15", "c16", "c17", "c18", "c19", "c20" ) - .collect().head + .collect().asScala.head tuple shouldBe Tuple20( 1: Integer, "2", @@ -902,7 +902,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu )).select( "c1", "c2", "c3", "c4", "c5", "c6", "c7", "c8", "c9", "c10", "c11", "c12", "c13", "c14", "c15", "c16", "c17", "c18", "c19", "c20", "c21" ) - .collect().head + .collect().asScala.head tuple shouldBe Tuple21( 1: Integer, "2", @@ -957,7 +957,7 @@ class CassandraJavaUtilSpec extends SparkCassandraITFlatSpecBase with DefaultClu )).select( "c1", "c2", "c3", "c4", "c5", "c6", "c7", "c8", "c9", "c10", "c11", "c12", "c13", "c14", "c15", "c16", "c17", "c18", "c19", "c20", "c21", "c22" ) - .collect().head + .collect().asScala.head tuple shouldBe Tuple22( 1: Integer, "2", diff --git a/connector/src/it/scala/com/datastax/spark/connector/SparkCassandraITFlatSpecBase.scala b/connector/src/it/scala/com/datastax/spark/connector/SparkCassandraITFlatSpecBase.scala index 0c11c28c1..bdcc8a7fc 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/SparkCassandraITFlatSpecBase.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/SparkCassandraITFlatSpecBase.scala @@ -26,7 +26,8 @@ import org.scalatest._ import org.scalatest.concurrent.Eventually._ import org.scalatest.time.{Seconds, Span} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ +import scala.collection.compat._ import scala.concurrent.duration.Duration import scala.concurrent.{Await, ExecutionContext, Future} import scala.util.Try @@ -200,8 +201,8 @@ trait SparkCassandraITSpecBase Await.result(Future.sequence(units), Duration.Inf) } - def awaitAll[T](units: TraversableOnce[Future[T]]): TraversableOnce[T] = { - Await.result(Future.sequence(units), Duration.Inf) + def awaitAll[T](units: IterableOnce[Future[T]]): IterableOnce[T] = { + Await.result(Future.sequence(units.iterator), Duration.Inf) } def keyspaceCql(name: String = ks) = diff --git a/connector/src/it/scala/com/datastax/spark/connector/cql/CassandraAuthenticatedConnectorSpec.scala b/connector/src/it/scala/com/datastax/spark/connector/cql/CassandraAuthenticatedConnectorSpec.scala index b7561071e..1c382b4da 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/cql/CassandraAuthenticatedConnectorSpec.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/cql/CassandraAuthenticatedConnectorSpec.scala @@ -66,7 +66,7 @@ class CassandraAuthenticatedConnectorSpec extends SparkCassandraITFlatSpecBase w "spark.cassandra.auth.password" -> "cassandra", "keyspace" -> ks, "table" -> "authtest") - personDF1.write.format("org.apache.spark.sql.cassandra").options(options).mode("append")save() + personDF1.write.format("org.apache.spark.sql.cassandra").options(options).mode("append").save() val personDF2 = spark.read.format("org.apache.spark.sql.cassandra").options(options).load() personDF2.count should be(4) diff --git a/connector/src/it/scala/com/datastax/spark/connector/rdd/CassandraJavaPairRDDSpec.scala b/connector/src/it/scala/com/datastax/spark/connector/rdd/CassandraJavaPairRDDSpec.scala index a94fc6811..2b209373f 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/rdd/CassandraJavaPairRDDSpec.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/rdd/CassandraJavaPairRDDSpec.scala @@ -8,7 +8,7 @@ import com.datastax.spark.connector.cql.CassandraConnector import com.datastax.spark.connector.japi.CassandraJavaUtil._ import org.apache.spark.api.java.function.{Function2, Function => JFunction} -import scala.collection.JavaConversions._ +import scala.jdk.CollectionConverters._ import scala.concurrent.Future case class SimpleClass(value: Integer) @@ -103,15 +103,16 @@ class CassandraJavaPairRDDSpec extends SparkCassandraITFlatSpecBase with Default "key") .spanBy(f, classOf[Integer]) .collect() + .asScala .toMap results should have size 2 results should contain key 10 results should contain key 20 - results(10).size should be(3) - results(10).map(_._2).toSeq should be(Seq(10, 11, 12)) - results(20).size should be(3) - results(20).map(_._2).toSeq should be(Seq(20, 21, 22)) + results(10).asScala.size should be(3) + results(10).asScala.map(_._2).toSeq should be(Seq(10, 11, 12)) + results(20).asScala.size should be(3) + results(20).asScala.map(_._2).toSeq should be(Seq(20, 21, 22)) } it should "allow to use spanByKey method" in { @@ -129,15 +130,16 @@ class CassandraJavaPairRDDSpec extends SparkCassandraITFlatSpecBase with Default "key") .spanByKey() .collect() + .asScala .toMap results should have size 2 results should contain key 10 results should contain key 20 results(10).size should be(3) - results(10).toSeq should be(Seq(10, 11, 12)) + results(10).asScala.toSeq should be(Seq(10, 11, 12)) results(20).size should be(3) - results(20).toSeq should be(Seq(20, 21, 22)) + results(20).asScala.toSeq should be(Seq(20, 21, 22)) } it should "allow to use of keyByAndApplyPartitioner" in { diff --git a/connector/src/it/scala/com/datastax/spark/connector/rdd/CassandraJavaRDDSpec.scala b/connector/src/it/scala/com/datastax/spark/connector/rdd/CassandraJavaRDDSpec.scala index a0279d057..ead546eb5 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/rdd/CassandraJavaRDDSpec.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/rdd/CassandraJavaRDDSpec.scala @@ -12,7 +12,7 @@ import com.datastax.spark.connector.types.TypeConverter import org.apache.commons.lang3.tuple import org.apache.spark.api.java.function.{Function => JFunction} -import scala.collection.JavaConversions._ +import scala.jdk.CollectionConverters._ import scala.concurrent.Future class CassandraJavaRDDSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { @@ -91,25 +91,25 @@ class CassandraJavaRDDSpec extends SparkCassandraITFlatSpecBase with DefaultClus "CassandraJavaRDD" should "allow to read data as CassandraRows " in { val rows = javaFunctions(sc).cassandraTable(ks, "test_table").collect() assert(rows.size == 3) - assert(rows.exists(row ⇒ row.getString("value") == "one" && row.getInt("key") == 1)) - assert(rows.exists(row ⇒ row.getString("value") == "two" && row.getInt("key") == 2)) - assert(rows.exists(row ⇒ row.getString("value") == null && row.getInt("key") == 3)) + assert(rows.asScala.exists(row ⇒ row.getString("value") == "one" && row.getInt("key") == 1)) + assert(rows.asScala.exists(row ⇒ row.getString("value") == "two" && row.getInt("key") == 2)) + assert(rows.asScala.exists(row ⇒ row.getString("value") == null && row.getInt("key") == 3)) } it should "allow to read data as Java beans " in { val beans = javaFunctions(sc).cassandraTable(ks, "test_table", mapRowTo(classOf[SampleJavaBean])).collect() assert(beans.size == 3) - assert(beans.exists(bean ⇒ bean.getValue == "one" && bean.getKey == 1)) - assert(beans.exists(bean ⇒ bean.getValue == "two" && bean.getKey == 2)) - assert(beans.exists(bean ⇒ bean.getValue == null && bean.getKey == 3)) + assert(beans.asScala.exists(bean ⇒ bean.getValue == "one" && bean.getKey == 1)) + assert(beans.asScala.exists(bean ⇒ bean.getValue == "two" && bean.getKey == 2)) + assert(beans.asScala.exists(bean ⇒ bean.getValue == null && bean.getKey == 3)) } it should "allow to read data as Java beans with inherited fields" in { val beans = javaFunctions(sc).cassandraTable(ks, "test_table3", mapRowTo(classOf[SampleJavaBeanSubClass])).collect() assert(beans.size == 3) - assert(beans.exists(bean ⇒ bean.getValue == "one" && bean.getKey == 1 && bean.getSubClassField == "a")) - assert(beans.exists(bean ⇒ bean.getValue == "two" && bean.getKey == 2 && bean.getSubClassField == "b")) - assert(beans.exists(bean ⇒ bean.getValue == null && bean.getKey == 3 && bean.getSubClassField == "c")) + assert(beans.asScala.exists(bean ⇒ bean.getValue == "one" && bean.getKey == 1 && bean.getSubClassField == "a")) + assert(beans.asScala.exists(bean ⇒ bean.getValue == "two" && bean.getKey == 2 && bean.getSubClassField == "b")) + assert(beans.asScala.exists(bean ⇒ bean.getValue == null && bean.getKey == 3 && bean.getSubClassField == "c")) } it should "allow to read data as Java beans with custom mapping defined by aliases" in { @@ -118,17 +118,17 @@ class CassandraJavaRDDSpec extends SparkCassandraITFlatSpecBase with DefaultClus .select(column("key").as("devil"), column("value").as("cat")) .collect() assert(beans.size == 3) - assert(beans.exists(bean ⇒ bean.getCat == "one" && bean.getDevil == 1)) - assert(beans.exists(bean ⇒ bean.getCat == "two" && bean.getDevil == 2)) - assert(beans.exists(bean ⇒ bean.getCat == null && bean.getDevil == 3)) + assert(beans.asScala.exists(bean ⇒ bean.getCat == "one" && bean.getDevil == 1)) + assert(beans.asScala.exists(bean ⇒ bean.getCat == "two" && bean.getDevil == 2)) + assert(beans.asScala.exists(bean ⇒ bean.getCat == null && bean.getDevil == 3)) } it should "allow to read data as Java beans (with multiple constructors)" in { val beans = javaFunctions(sc).cassandraTable(ks, "test_table", mapRowTo(classOf[SampleJavaBeanWithMultipleCtors])).collect() assert(beans.size == 3) - assert(beans.exists(bean ⇒ bean.getValue == "one" && bean.getKey == 1)) - assert(beans.exists(bean ⇒ bean.getValue == "two" && bean.getKey == 2)) - assert(beans.exists(bean ⇒ bean.getValue == null && bean.getKey == 3)) + assert(beans.asScala.exists(bean ⇒ bean.getValue == "one" && bean.getKey == 1)) + assert(beans.asScala.exists(bean ⇒ bean.getValue == "two" && bean.getKey == 2)) + assert(beans.asScala.exists(bean ⇒ bean.getValue == null && bean.getKey == 3)) } it should "throw NoSuchMethodException when trying to read data as Java beans (without no-args constructor)" in { @@ -139,18 +139,18 @@ class CassandraJavaRDDSpec extends SparkCassandraITFlatSpecBase with DefaultClus it should "allow to read data as nested Java beans" in { val beans = javaFunctions(sc).cassandraTable(ks, "test_table", mapRowTo(classOf[SampleWithNestedJavaBean#InnerClass])).collect() assert(beans.size == 3) - assert(beans.exists(bean ⇒ bean.getValue == "one" && bean.getKey == 1)) - assert(beans.exists(bean ⇒ bean.getValue == "two" && bean.getKey == 2)) - assert(beans.exists(bean ⇒ bean.getValue == null && bean.getKey == 3)) + assert(beans.asScala.exists(bean ⇒ bean.getValue == "one" && bean.getKey == 1)) + assert(beans.asScala.exists(bean ⇒ bean.getValue == "two" && bean.getKey == 2)) + assert(beans.asScala.exists(bean ⇒ bean.getValue == null && bean.getKey == 3)) } it should "allow to read data as deeply nested Java beans" in { val beans = javaFunctions(sc).cassandraTable(ks, "test_table", mapRowTo(classOf[SampleWithDeeplyNestedJavaBean#IntermediateClass#InnerClass])).collect() assert(beans.size == 3) - assert(beans.exists(bean ⇒ bean.getValue == "one" && bean.getKey == 1)) - assert(beans.exists(bean ⇒ bean.getValue == "two" && bean.getKey == 2)) - assert(beans.exists(bean ⇒ bean.getValue == null && bean.getKey == 3)) + assert(beans.asScala.exists(bean ⇒ bean.getValue == "one" && bean.getKey == 1)) + assert(beans.asScala.exists(bean ⇒ bean.getValue == "two" && bean.getKey == 2)) + assert(beans.asScala.exists(bean ⇒ bean.getValue == null && bean.getKey == 3)) } @@ -158,9 +158,9 @@ class CassandraJavaRDDSpec extends SparkCassandraITFlatSpecBase with DefaultClus val rows = javaFunctions(sc).cassandraTable(ks, "test_table") .select("key").collect() assert(rows.size == 3) - assert(rows.exists(row ⇒ !row.contains("value") && row.getInt("key") == 1)) - assert(rows.exists(row ⇒ !row.contains("value") && row.getInt("key") == 2)) - assert(rows.exists(row ⇒ !row.contains("value") && row.getInt("key") == 3)) + assert(rows.asScala.exists(row ⇒ !row.contains("value") && row.getInt("key") == 1)) + assert(rows.asScala.exists(row ⇒ !row.contains("value") && row.getInt("key") == 2)) + assert(rows.asScala.exists(row ⇒ !row.contains("value") && row.getInt("key") == 3)) } it should "return selected columns" in { @@ -174,7 +174,7 @@ class CassandraJavaRDDSpec extends SparkCassandraITFlatSpecBase with DefaultClus val rows = javaFunctions(sc).cassandraTable(ks, "test_table") .where("value = ?", "two").collect() assert(rows.size === 1) - assert(rows.exists(row => row.getString("value") == "two" && row.getInt("key") == 2)) + assert(rows.asScala.exists(row => row.getString("value") == "two" && row.getInt("key") == 2)) } it should "allow to read rows as an array of a single-column type supported by TypeConverter" in { @@ -212,7 +212,7 @@ class CassandraJavaRDDSpec extends SparkCassandraITFlatSpecBase with DefaultClus val rows = javaFunctions(sc) .cassandraTable(ks, "collections", mapColumnToListOf(classOf[String])) .select("l") - .collect().map(_.toList) + .collect().asScala.map(_.asScala.toList) rows should have size 2 rows should contain(List("item1", "item2")) @@ -223,7 +223,7 @@ class CassandraJavaRDDSpec extends SparkCassandraITFlatSpecBase with DefaultClus val rows = javaFunctions(sc) .cassandraTable(ks, "collections", mapColumnToSetOf(classOf[String])) .select("s") - .collect().map(_.toSet) + .collect().asScala.map(_.asScala.toSet) rows should have size 2 rows should contain(Set("item1", "item2")) @@ -234,7 +234,7 @@ class CassandraJavaRDDSpec extends SparkCassandraITFlatSpecBase with DefaultClus val rows = javaFunctions(sc) .cassandraTable(ks, "collections", mapColumnToMapOf(classOf[String], classOf[String])) .select("m") - .collect().map(_.toMap) + .collect().asScala.map(_.asScala.toMap) rows should have size 2 rows should contain(Map("key1" → "value1", "key2" → "value2")) @@ -244,7 +244,7 @@ class CassandraJavaRDDSpec extends SparkCassandraITFlatSpecBase with DefaultClus it should "allow to read rows as an array of multi-column type" in { val rows = javaFunctions(sc) .cassandraTable(ks, "test_table", mapRowTo(classOf[SampleJavaBean])) - .collect().map(x => (x.getKey, x.getValue)) + .collect().asScala.map(x => (x.getKey, x.getValue)) rows should have size 3 rows should contain((1, "one")) @@ -256,7 +256,7 @@ class CassandraJavaRDDSpec extends SparkCassandraITFlatSpecBase with DefaultClus val rows = javaFunctions(sc) .cassandraTable(ks, "test_table2", mapRowTo(classOf[SampleJavaBean], tuple.Pair.of("key", "some_key"), tuple.Pair.of("value", "some_value"))) - .collect().map(x => (x.getKey, x.getValue)) + .collect().asScala.map(x => (x.getKey, x.getValue)) rows should have size 3 rows should contain((1, "one")) @@ -288,7 +288,7 @@ class CassandraJavaRDDSpec extends SparkCassandraITFlatSpecBase with DefaultClus mapToRow(classOf[java.lang.Integer]), classOf[Integer], "key") - .collect().map { case (i, x) ⇒ (i, (x.getKey, x.getValue))} + .collect().asScala.map { case (i, x) ⇒ (i, (x.getKey, x.getValue))} rows should have size 3 rows should contain((1, (1, "one"))) @@ -304,7 +304,7 @@ class CassandraJavaRDDSpec extends SparkCassandraITFlatSpecBase with DefaultClus mapRowTo(classOf[SampleJavaBean]), mapToRow(classOf[SampleJavaBean]), classOf[SampleJavaBean]) - .collect().map { case (x, i) ⇒ ((x.getKey, x.getValue), i)} + .collect().asScala.map { case (x, i) ⇒ ((x.getKey, x.getValue), i)} rows should have size 3 rows should contain(((1, "one"), 1)) @@ -319,7 +319,7 @@ class CassandraJavaRDDSpec extends SparkCassandraITFlatSpecBase with DefaultClus mapRowTo(classOf[SampleJavaBean]), mapToRow(classOf[SampleJavaBean]), classOf[SampleJavaBean]) - .collect().map { case (x, y) ⇒ ((x.getKey, x.getValue), (y.getKey, y.getValue))} + .collect().asScala.map { case (x, y) ⇒ ((x.getKey, x.getValue), (y.getKey, y.getValue))} rows should have size 3 rows should contain(((1, "one"), (1, "one"))) @@ -368,7 +368,7 @@ class CassandraJavaRDDSpec extends SparkCassandraITFlatSpecBase with DefaultClus .select("key", "name", "addr").collect() result should have length 1 - val row = result.head + val row = result.asScala.head row.getInt(0) should be(1) row.getString(1) should be("name") @@ -385,7 +385,7 @@ class CassandraJavaRDDSpec extends SparkCassandraITFlatSpecBase with DefaultClus .select("key", "value").collect() result should have length 1 - val row = result.head + val row = result.asScala.head row.getInt(0) should be(1) val tupleValue = row.getTupleValue(1) @@ -405,15 +405,16 @@ class CassandraJavaRDDSpec extends SparkCassandraITFlatSpecBase with DefaultClus .select("key", "group", "value") .spanBy[Int](f, classOf[Int]) .collect() + .asScala .toMap results should have size 2 results should contain key 10 results should contain key 20 - results(10).size should be(3) - results(10).map(_.getInt("group")).toSeq should be(Seq(10, 11, 12)) - results(20).size should be(3) - results(20).map(_.getInt("group")).toSeq should be(Seq(20, 21, 22)) + results(10).asScala.size should be(3) + results(10).asScala.map(_.getInt("group")).toSeq should be(Seq(10, 11, 12)) + results(20).asScala.size should be(3) + results(20).asScala.map(_.getInt("group")).toSeq should be(Seq(20, 21, 22)) } it should "allow to set limit" in { @@ -425,13 +426,13 @@ class CassandraJavaRDDSpec extends SparkCassandraITFlatSpecBase with DefaultClus it should "allow to set ascending ordering" in { val rdd = javaFunctions(sc).cassandraTable(ks, "wide_rows").where("key=10").withAscOrder - val result = rdd.collect() + val result = rdd.collect().asScala result(0).getInt("group") should be(10) } it should "allow to set descending ordering" in { val rdd = javaFunctions(sc).cassandraTable(ks, "wide_rows").where("key=20").withDescOrder - val result = rdd.collect() + val result = rdd.collect().asScala result(0).getInt("group") should be(22) } } diff --git a/connector/src/it/scala/com/datastax/spark/connector/rdd/CassandraRDDSpec.scala b/connector/src/it/scala/com/datastax/spark/connector/rdd/CassandraRDDSpec.scala index dc3e6d6fe..7fffee04a 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/rdd/CassandraRDDSpec.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/rdd/CassandraRDDSpec.scala @@ -3,7 +3,6 @@ package com.datastax.spark.connector.rdd import java.io.IOException import java.time.{Instant, LocalDate, ZoneId, ZonedDateTime} import java.util.Date - import com.datastax.oss.driver.api.core.DefaultProtocolVersion._ import com.datastax.oss.driver.api.core.config.DefaultDriverOption import com.datastax.oss.driver.api.core.cql.SimpleStatement @@ -15,8 +14,9 @@ import com.datastax.spark.connector.cql.{CassandraConnector, CassandraConnectorC import com.datastax.spark.connector.mapper.{DefaultColumnMapper, JavaBeanColumnMapper, JavaTestBean, JavaTestUDTBean} import com.datastax.spark.connector.rdd.partitioner.dht.TokenFactory import com.datastax.spark.connector.types.{CassandraOption, TypeConverter} +import com.datastax.spark.connector.util.RuntimeUtil -import scala.collection.JavaConversions._ +import scala.jdk.CollectionConverters._ import scala.concurrent.Future import scala.reflect.runtime.universe.typeTag @@ -730,19 +730,19 @@ class CassandraRDDSpec extends SparkCassandraITFlatSpecBase with DefaultCluster // compute a few RDDs so the thread pools get initialized // using parallel range, to initialize parallel collections fork-join-pools val iterationCount = 256 - for (i <- (1 to iterationCount).par) + for (i <- RuntimeUtil.toParallelIterable(1 to iterationCount).par) sc.cassandraTable(ks, "key_value").collect() // subsequent computations of RDD should reuse already created thread pools, // not instantiate new ones val startThreadCount = threadCount() - val oldThreads = Thread.getAllStackTraces.keySet().toSet + val oldThreads = Thread.getAllStackTraces.keySet().asScala.toSet - for (i <- (1 to iterationCount).par) + for (i <- RuntimeUtil.toParallelIterable(1 to iterationCount).par) sc.cassandraTable(ks, "key_value").collect() val endThreadCount = threadCount() - val newThreads = Thread.getAllStackTraces.keySet().toSet + val newThreads = Thread.getAllStackTraces.keySet().asScala.toSet val createdThreads = newThreads -- oldThreads println("Start thread count: " + startThreadCount) println("End thread count: " + endThreadCount) diff --git a/connector/src/it/scala/com/datastax/spark/connector/rdd/CustomTableScanMethodSpec.scala b/connector/src/it/scala/com/datastax/spark/connector/rdd/CustomTableScanMethodSpec.scala index 25acb8d3b..c2ba1527b 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/rdd/CustomTableScanMethodSpec.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/rdd/CustomTableScanMethodSpec.scala @@ -5,6 +5,7 @@ import com.datastax.spark.connector.cluster.{DefaultCluster, SeparateJVM} import com.datastax.spark.connector.cql._ import com.datastax.spark.connector.embedded.SparkTemplate._ import com.datastax.spark.connector.rdd.partitioner.dht.TokenFactory +import com.datastax.spark.connector.util.RuntimeUtil import com.datastax.spark.connector.{SparkCassandraITFlatSpecBase, _} import org.apache.spark.{SparkContext, SparkException} import org.scalatest.Inspectors @@ -33,7 +34,7 @@ class CustomTableScanMethodSpec extends SparkCassandraITFlatSpecBase with Defaul session.execute(s"CREATE TABLE $ks.$tableName(key int primary key, value text)") val st = session.prepare(s"INSERT INTO $ks.$tableName(key, value) VALUES(?, ?)") // 1M rows x 64 bytes of payload = 64 MB of data + overhead - for (i <- (1 to 100).par) { + for (i <- RuntimeUtil.toParallelIterable(1 to 100).par) { val key = i.asInstanceOf[AnyRef] val value = "123456789.123456789.123456789.123456789.123456789.123456789." session.execute(st.bind(key, value)) diff --git a/connector/src/it/scala/com/datastax/spark/connector/rdd/RDDSpec.scala b/connector/src/it/scala/com/datastax/spark/connector/rdd/RDDSpec.scala index ef9eb4152..3d03f5a6a 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/rdd/RDDSpec.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/rdd/RDDSpec.scala @@ -1,7 +1,6 @@ package com.datastax.spark.connector.rdd import java.lang.{Long => JLong} - import com.datastax.oss.driver.api.core.config.DefaultDriverOption._ import com.datastax.oss.driver.api.core.cql.{AsyncResultSet, BoundStatement} import com.datastax.oss.driver.api.core.{DefaultConsistencyLevel, DefaultProtocolVersion} @@ -12,7 +11,9 @@ import com.datastax.spark.connector.cql.CassandraConnector import com.datastax.spark.connector.embedded.SparkTemplate._ import com.datastax.spark.connector.rdd.partitioner.EndpointPartition import com.datastax.spark.connector.writer.AsyncExecutor +import spire.ClassTag +import scala.jdk.CollectionConverters._ import scala.concurrent.Future case class KVRow(key: Int) @@ -139,7 +140,7 @@ class RDDSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { } } - def checkLeftSide[T, S](leftSideSource: Array[T], result: Array[(T, S)]) = { + def checkLeftSide[T, S](leftSideSource: Array[T], result: Array[(T, S)])(implicit classTag: ClassTag[T]) = { markup("Checking LeftSide") val leftSideResults = result.map(_._1) for (element <- leftSideSource) { diff --git a/connector/src/it/scala/com/datastax/spark/connector/rdd/partitioner/TokenGeneratorSpec.scala b/connector/src/it/scala/com/datastax/spark/connector/rdd/partitioner/TokenGeneratorSpec.scala index 5074270ab..e0d754a0d 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/rdd/partitioner/TokenGeneratorSpec.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/rdd/partitioner/TokenGeneratorSpec.scala @@ -9,7 +9,7 @@ import com.datastax.spark.connector.util.schemaFromCassandra import com.datastax.spark.connector.writer.RowWriterFactory import com.datastax.spark.connector.{PartitionKeyColumns, SparkCassandraITFlatSpecBase} -import scala.collection.JavaConversions._ +import scala.jdk.CollectionConverters._ import scala.concurrent.Future class TokenGeneratorSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { @@ -44,7 +44,7 @@ class TokenGeneratorSpec extends SparkCassandraITFlatSpecBase with DefaultCluste } val simpleTokenMap: Map[SimpleKey, Token] = conn.withSessionDo { session => - val resultSet = session.execute(s"SELECT key, TOKEN(key) FROM $ks.simple").all() + val resultSet = session.execute(s"SELECT key, TOKEN(key) FROM $ks.simple").all().asScala resultSet.map(row => SimpleKey(row.getInt("key")) -> row.getToken(1)).toMap } @@ -52,6 +52,7 @@ class TokenGeneratorSpec extends SparkCassandraITFlatSpecBase with DefaultCluste val resultSet = session .execute(s"SELECT key1, key2, key3, TOKEN(key1, key2, key3) FROM $ks.complex") .all() + .asScala resultSet.map(row => ComplexKey(row.getInt("key1"), row.getInt("key2"), row.getString("key3")) -> diff --git a/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/AbstractTypeTest.scala b/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/AbstractTypeTest.scala index 5e49de99c..8ebd2ae0e 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/AbstractTypeTest.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/AbstractTypeTest.scala @@ -354,8 +354,8 @@ abstract class AbstractTypeTest[TestType: ClassTag, DriverType <: AnyRef : Class val javaSet: java.util.Set[DriverType] = scalaSet.asJava val javaList: java.util.List[DriverType] = scalaSet.toList.asJava - val javaMap1: java.util.Map[String, DriverType] = typeMap.mapValues(value => - convertToDriverInsertable(value)).asJava + val javaMap1: java.util.Map[String, DriverType] = typeMap.map{ case (key ,value) => + (key, convertToDriverInsertable(value))}.toMap.asJava val javaMap2: java.util.Map[DriverType, String] = typeReverseMap.map { case (key, value) => (convertToDriverInsertable(key), value) }.asJava diff --git a/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/BlobTypeTest.scala b/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/BlobTypeTest.scala index 84b167089..3c71ccefa 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/BlobTypeTest.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/BlobTypeTest.scala @@ -44,13 +44,13 @@ class BlobTypeTest extends AbstractTypeTest[Array[Byte], ByteBuffer] with Defaul val resSet = row.getSet("set1", classOf[ByteBuffer]).asScala.map(Bytes.toHexString(_)) val resList = row.getList("list1", classOf[ByteBuffer]).asScala.map(Bytes.toHexString(_)) val resMap1 = row.getMap("map1", classOf[String], classOf[ByteBuffer]).asScala - .mapValues(Bytes.toHexString(_)) + .map{ case (k,v) => (k, Bytes.toHexString(v)) } val resMap2 = row.getMap("map2", classOf[ByteBuffer], classOf[String]).asScala .map{ case (k,v) => (Bytes.toHexString(k), v)} resSet should contain theSameElementsAs bufferTypeSet.map(Bytes.toHexString(_)) resList should contain theSameElementsAs bufferTypeSet.toList.map(Bytes.toHexString(_)) - resMap1 should contain theSameElementsAs bufferTypeMap1.mapValues( Bytes.toHexString(_)) + resMap1 should contain theSameElementsAs bufferTypeMap1.map{ case (k, v) => (k, Bytes.toHexString(v))} resMap2 should contain theSameElementsAs bufferTypeMap2.map{ case (k,v) => (Bytes.toHexString(k), v)} } } diff --git a/connector/src/it/scala/com/datastax/spark/connector/sql/CassandraDataFrameSpec.scala b/connector/src/it/scala/com/datastax/spark/connector/sql/CassandraDataFrameSpec.scala index 7d21a2f8b..48e63edc3 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/sql/CassandraDataFrameSpec.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/sql/CassandraDataFrameSpec.scala @@ -17,7 +17,7 @@ import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructT import org.joda.time.LocalDate import org.scalatest.concurrent.Eventually -import scala.collection.JavaConversions._ +import scala.jdk.CollectionConverters._ import scala.concurrent.Future case class RowWithV4Types(key: Int, a: Byte, b: Short, c: java.sql.Date) @@ -180,8 +180,8 @@ class CassandraDataFrameSpec extends SparkCassandraITFlatSpecBase with DefaultCl val meta = conn.withSessionDo(_.getMetadata) val autoTableMeta = meta.getKeyspace(ks).get().getTable("kv_auto").get() - autoTableMeta.getPartitionKey.map(k => toName(k.getName)) should contain ("v") - autoTableMeta.getClusteringColumns.map(c => toName(c._1.getName)) should contain ("k") + autoTableMeta.getPartitionKey.asScala.map(k => toName(k.getName)) should contain ("v") + autoTableMeta.getClusteringColumns.asScala.map(c => toName(c._1.getName)) should contain ("k") } @@ -202,9 +202,9 @@ class CassandraDataFrameSpec extends SparkCassandraITFlatSpecBase with DefaultCl val meta = conn.withSessionDo(_.getMetadata) val autoTableMeta = meta.getKeyspace(ks).get().getTable("kv_auto2").get() - autoTableMeta.getPartitionKey.map(k => toName(k.getName)) should contain ("v") - autoTableMeta.getClusteringColumns.map(c => toName(c._1.getName)) should contain ("k") - autoTableMeta.getClusteringColumns.map(_._2) should contain (ClusteringOrder.DESC) + autoTableMeta.getPartitionKey.asScala.map(k => toName(k.getName)) should contain ("v") + autoTableMeta.getClusteringColumns.asScala.map(c => toName(c._1.getName)) should contain ("k") + autoTableMeta.getClusteringColumns.asScala.map(_._2) should contain (ClusteringOrder.DESC) autoTableMeta.getOptions.getOrDefault(CqlIdentifier.fromCql("gc_grace_seconds"), "0").toString should equal ("1000") } @@ -319,7 +319,7 @@ class CassandraDataFrameSpec extends SparkCassandraITFlatSpecBase with DefaultCl df.createCassandraTable(ks, table) val tableColumns = eventually( - conn.withSessionDo(_.getMetadata.getKeyspace(ks).get().getTable(table)).get().getColumns.map(_._2.getType)) + conn.withSessionDo(_.getMetadata.getKeyspace(ks).get().getTable(table)).get().getColumns.asScala.map(_._2.getType)) tableColumns should contain theSameElementsAs ( Seq(DataTypes.INT, DataTypes.INT, DataTypes.INT, DataTypes.TIMESTAMP) diff --git a/connector/src/it/scala/com/datastax/spark/connector/util/MultiThreadedSpec.scala b/connector/src/it/scala/com/datastax/spark/connector/util/MultiThreadedSpec.scala index 2eefa72a0..d3b650682 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/util/MultiThreadedSpec.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/util/MultiThreadedSpec.scala @@ -46,7 +46,7 @@ class MultiThreadedSpec extends SparkCassandraITFlatSpecBase with DefaultCluster for (thread <- threads) thread.start() import org.scalatest.time.SpanSugar._ - w.await(timeout(90 seconds), dismissals(5)) + w.await(timeout(90.seconds), dismissals(5)) for (thread <- threads) thread.join() } diff --git a/connector/src/it/scala/com/datastax/spark/connector/writer/GroupingBatchBuilderSpec.scala b/connector/src/it/scala/com/datastax/spark/connector/writer/GroupingBatchBuilderSpec.scala index 0d65fce37..360215fc1 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/writer/GroupingBatchBuilderSpec.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/writer/GroupingBatchBuilderSpec.scala @@ -7,7 +7,7 @@ import com.datastax.spark.connector.cql.{CassandraConnector, Schema} import com.datastax.spark.connector.util.schemaFromCassandra import com.datastax.spark.connector.{BatchSize, BytesInBatch, RowsInBatch, SparkCassandraITFlatSpecBase} -import scala.collection.JavaConversions._ +import scala.jdk.CollectionConverters._ import scala.util.Random class GroupingBatchBuilderSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { @@ -76,7 +76,7 @@ class GroupingBatchBuilderSpec extends SparkCassandraITFlatSpecBase with Default statements(1) shouldBe a[RichBoundStatementWrapper] statements.flatMap { case s: RichBoundStatementWrapper => List(s.stmt) - case s: RichBatchStatementWrapper => s.stmt.collect{ case b: BoundStatement => b } + case s: RichBatchStatementWrapper => s.stmt.asScala.collect{ case b: BoundStatement => b } }.map(s => (s.getInt(0), s.getString(1))) should contain theSameElementsAs data } } @@ -93,7 +93,7 @@ class GroupingBatchBuilderSpec extends SparkCassandraITFlatSpecBase with Default statements.map(_.stmt).flatMap { case s: BatchStatement => s.size() should be(2) - s.collect{ case b: BoundStatement => b } + s.asScala.collect{ case b: BoundStatement => b } }.map(s => (s.getInt(0), s.getString(1))) should contain theSameElementsAs data } } @@ -121,7 +121,7 @@ class GroupingBatchBuilderSpec extends SparkCassandraITFlatSpecBase with Default val stmtss = statements.map(_.stmt).map { case s: BoundStatement => List(s) - case s: BatchStatement => s.collect{ case b: BoundStatement => b } + case s: BatchStatement => s.asScala.collect{ case b: BoundStatement => b } } stmtss.foreach(stmts => stmts.size should be > 0) @@ -180,7 +180,7 @@ class GroupingBatchBuilderSpec extends SparkCassandraITFlatSpecBase with Default statements(1) shouldBe a[RichBoundStatementWrapper] statements.flatMap { case s: RichBoundStatementWrapper => List(s.stmt) - case s: RichBatchStatementWrapper => s.stmt.collect{ case b: BoundStatement => b } + case s: RichBatchStatementWrapper => s.stmt.asScala.collect{ case b: BoundStatement => b } }.map(s => (s.getInt(0), s.getString(1))) should contain theSameElementsAs data } } @@ -208,7 +208,7 @@ class GroupingBatchBuilderSpec extends SparkCassandraITFlatSpecBase with Default statements.flatMap { case s: RichBatchStatementWrapper => s.stmt.size() should be(2) - s.stmt.collect{ case b: BoundStatement => b } + s.stmt.asScala.collect{ case b: BoundStatement => b } }.map(s => (s.getInt(0), s.getString(1))) should contain theSameElementsAs data } } @@ -239,14 +239,14 @@ class GroupingBatchBuilderSpec extends SparkCassandraITFlatSpecBase with Default boundStatements should contain theSameElementsAs Seq(5, 6, 7) val batchStatements = statements collect { - case s: RichBatchStatementWrapper if s.stmt.size() == 2 => s.stmt.collect{ case b: BoundStatement => b.getInt(0) } + case s: RichBatchStatementWrapper if s.stmt.size() == 2 => s.stmt.asScala.collect{ case b: BoundStatement => b.getInt(0) } } batchStatements should have size 2 batchStatements should contain theSameElementsAs Seq(List(1, 3), List(2, 4)) val stmtss = statements.map { case s: RichBoundStatementWrapper => List(s.stmt) - case s: RichBatchStatementWrapper => s.stmt.collect{ case b: BoundStatement => b } + case s: RichBatchStatementWrapper => s.stmt.asScala.collect{ case b: BoundStatement => b } } stmtss.foreach(stmts => stmts.size should be > 0) stmtss.foreach(stmts => if (stmts.size > 1) stmts.map(BoundStatementBuilder.calculateDataSize).sum should be <= 15) @@ -283,7 +283,7 @@ class GroupingBatchBuilderSpec extends SparkCassandraITFlatSpecBase with Default case s: RichBoundStatementWrapper => List(s.stmt) case s: RichBatchStatementWrapper => s.stmt.size() should be <= 10 - s.stmt.collect { case b: BoundStatement => b } + s.stmt.asScala.collect { case b: BoundStatement => b } }.map(s => (s.getInt(0), s.getString(1))).sortBy(_.toString()) should contain theSameElementsInOrderAs data.sortBy(_.toString()) } diff --git a/connector/src/it/scala/com/datastax/spark/connector/writer/TableWriterSpec.scala b/connector/src/it/scala/com/datastax/spark/connector/writer/TableWriterSpec.scala index 5ed855be6..69cb1afea 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/writer/TableWriterSpec.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/writer/TableWriterSpec.scala @@ -4,7 +4,7 @@ import java.io.IOException import com.datastax.oss.driver.api.core.DefaultProtocolVersion -import scala.collection.JavaConversions._ +import scala.jdk.CollectionConverters._ import scala.concurrent.Future import com.datastax.spark.connector.cluster.DefaultCluster import com.datastax.spark.connector.{SomeColumns, _} @@ -107,7 +107,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { private def verifyKeyValueTable(tableName: String) { conn.withSessionDo { session => - val result = session.execute(s"""SELECT * FROM $ks.""" + tableName).all() + val result = session.execute(s"""SELECT * FROM $ks.""" + tableName).all().asScala result should have size 3 for (row <- result) { Some(row.getInt("key")) should contain oneOf(1, 2, 3) @@ -279,7 +279,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { val col = Seq((1, "value1"), (2, "value2"), (3, "value3")) sc.parallelize(col).saveToCassandra(ks, "camelCase", SomeColumns("primaryKey", "textValue")) conn.withSessionDo { session => - val result = session.execute(s"""SELECT * FROM $ks."camelCase"""").all() + val result = session.execute(s"""SELECT * FROM $ks."camelCase"""").all().asScala result should have size 3 for (row <- result) { Some(row.getInt(0)) should contain oneOf(1, 2, 3) @@ -293,7 +293,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { val col = Seq((1, 1L, None)) sc.parallelize(col).saveToCassandra(ks, "key_value", SomeColumns("key", "group", "value")) conn.withSessionDo { session => - val result = session.execute(s"""SELECT * FROM $ks.key_value""").all() + val result = session.execute(s"""SELECT * FROM $ks.key_value""").all().asScala result should have size 1 for (row <- result) { row.getString(2) should be (null) @@ -309,7 +309,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { sc.parallelize(Seq(row)).saveToCassandra(ks, "nulls") conn.withSessionDo { session => - val result = session.execute(s"""SELECT * FROM $ks.nulls""").all() + val result = session.execute(s"""SELECT * FROM $ks.nulls""").all().asScala result should have size 1 for (r <- result) { r.getInt(0) shouldBe key @@ -324,7 +324,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { val col = Seq((1, 1L, None)) sc.parallelize(col).saveToCassandra(ks, "key_value", SomeColumns("key", "group")) conn.withSessionDo { session => - val result = session.execute(s"""SELECT * FROM $ks.key_value""").all() + val result = session.execute(s"""SELECT * FROM $ks.key_value""").all().asScala result should have size 1 for (row <- result) { row.getInt(0) should be (1) @@ -338,7 +338,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { val col = Seq((2, 1L, None)) sc.parallelize(col).saveToCassandra(ks, "key_value", SomeColumns("key", "group")) conn.withSessionDo { session => - val result = session.execute(s"""SELECT * FROM $ks.key_value""").all() + val result = session.execute(s"""SELECT * FROM $ks.key_value""").all().asScala result should have size 1 for (row <- result) { row.getInt(0) should be (2) @@ -354,14 +354,14 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { sc.parallelize(col).saveToCassandra(ks, "collections", SomeColumns("key", "l", "s", "m")) conn.withSessionDo { session => - val result = session.execute(s"""SELECT * FROM $ks.collections""").all() + val result = session.execute(s"""SELECT * FROM $ks.collections""").all().asScala result should have size 2 val rows = result.groupBy(_.getInt(0)).mapValues(_.head) val row0 = rows(1) val row1 = rows(2) - row0.getList("l", classOf[String]).toSeq shouldEqual Seq("item1", "item2") - row0.getSet("s", classOf[String]).toSeq shouldEqual Seq("item1", "item2") - row0.getMap("m", classOf[String], classOf[String]).toMap shouldEqual Map("key1" -> "value1", "key2" -> "value2") + row0.getList("l", classOf[String]).asScala.toSeq shouldEqual Seq("item1", "item2") + row0.getSet("s", classOf[String]).asScala.toSeq shouldEqual Seq("item1", "item2") + row0.getMap("m", classOf[String], classOf[String]).asScala.toMap shouldEqual Map("key1" -> "value1", "key2" -> "value2") row1.isNull("l") shouldEqual true row1.isNull("m") shouldEqual true row1.isNull("s") shouldEqual true @@ -372,7 +372,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { val col = Seq((1, Some(Array[Byte](0, 1, 2, 3))), (2, None)) sc.parallelize(col).saveToCassandra(ks, "blobs", SomeColumns("key", "b")) conn.withSessionDo { session => - val result = session.execute(s"""SELECT * FROM $ks.blobs""").all() + val result = session.execute(s"""SELECT * FROM $ks.blobs""").all().asScala result should have size 2 val rows = result.groupBy(_.getInt(0)).mapValues(_.head) val row0 = rows(1) @@ -414,7 +414,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { sc.parallelize(col).saveToCassandra(ks, "key_value", SomeColumns("key", "group", "value")) conn.withSessionDo { session => - val result = session.execute(s"""SELECT * FROM $ks.key_value""").all() + val result = session.execute(s"""SELECT * FROM $ks.key_value""").all().asScala result should have size 1 for (row <- result) row.getString(2) shouldEqual "foo" @@ -430,7 +430,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { sc.parallelize(col).saveToCassandra(ks, "udts", SomeColumns("key", "name", "addr")) conn.withSessionDo { session => - val result = session.execute(s"""SELECT key, name, addr FROM $ks.udts""").all() + val result = session.execute(s"""SELECT key, name, addr FROM $ks.udts""").all().asScala result should have size 1 for (row <- result) { row.getInt(0) shouldEqual 1 @@ -455,7 +455,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { sc.parallelize(col).saveToCassandra(ks, "udts", SomeColumns("key", "name", "addr")) conn.withSessionDo { session => - val result = session.execute(s"""SELECT key, name, addr FROM $ks.udts""").all() + val result = session.execute(s"""SELECT key, name, addr FROM $ks.udts""").all().asScala result should have size 1 for (row <- result) { row.getInt(0) shouldEqual 1 @@ -472,7 +472,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { sc.parallelize(col).saveToCassandra(ks, "udts", SomeColumns("key", "name", "addr")) conn.withSessionDo { session => - val result = session.execute(s"""SELECT key, name, addr FROM "$ks".udts""").all() + val result = session.execute(s"""SELECT key, name, addr FROM "$ks".udts""").all().asScala result should have size 1 for (row <- result) { row.getInt(0) shouldEqual 1 @@ -488,7 +488,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { sc.parallelize(col).saveToCassandra(ks, "udts", SomeColumns("key", "name", "addr")) conn.withSessionDo { session => - val result = session.execute(s"""SELECT key, name, addr FROM "$ks".udts""").all() + val result = session.execute(s"""SELECT key, name, addr FROM "$ks".udts""").all().asScala result should have size 1 for (row <- result) { row.getInt(0) shouldEqual 1 @@ -507,7 +507,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { sc.parallelize(col).saveToCassandra(ks, "tuples", SomeColumns("key", "value")) conn.withSessionDo { session => - val result = session.execute(s"""SELECT key, value FROM $ks.tuples""").all() + val result = session.execute(s"""SELECT key, value FROM $ks.tuples""").all().asScala result should have size 1 for (row <- result) { row.getInt(0) shouldEqual 1 @@ -524,7 +524,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { sc.parallelize(col).saveToCassandra(ks, "tuples", SomeColumns("key", "value")) conn.withSessionDo { session => - val result = session.execute(s"""SELECT key, value FROM $ks.tuples""").all() + val result = session.execute(s"""SELECT key, value FROM $ks.tuples""").all().asScala result should have size 1 for (row <- result) { row.getInt(0) shouldEqual 1 @@ -542,7 +542,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { sc.parallelize(col).saveToCassandra(ks, "nested_tuples", SomeColumns("key", "addr")) conn.withSessionDo { session => - val result = session.execute(s"""SELECT key, addr FROM $ks.nested_tuples""").all() + val result = session.execute(s"""SELECT key, addr FROM $ks.nested_tuples""").all().asScala result should have size 1 for (row <- result) { row.getInt(0) shouldEqual 1 @@ -560,7 +560,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { sc.parallelize(col).saveToCassandra(ks, "nested_tuples", SomeColumns("key", "addr")) conn.withSessionDo { session => - val result = session.execute(s"""SELECT key, addr FROM $ks.nested_tuples""").all() + val result = session.execute(s"""SELECT key, addr FROM $ks.nested_tuples""").all().asScala for (row <- result) { row.getUdtValue(1).getTupleValue(1).getInt(0) shouldEqual 1 row.getUdtValue(1).getTupleValue(1).getInt(1) shouldEqual 2 @@ -572,7 +572,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { val col = Seq(1, 2, 3, 4, 5).map(Tuple1.apply) sc.parallelize(col).saveToCassandra(ks, "single_column", SomeColumns("pk")) conn.withSessionDo { session => - val result = session.execute(s"""SELECT * FROM $ks.single_column""").all() + val result = session.execute(s"""SELECT * FROM $ks.single_column""").all().asScala result should have size 5 result.map(_.getInt(0)).toSet should be (Set(1, 2, 3, 4, 5)) } @@ -593,7 +593,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { verifyKeyValueTable("key_value") conn.withSessionDo { session => - val result = session.execute(s"""SELECT TTL(value) FROM $ks.key_value""").all() + val result = session.execute(s"""SELECT TTL(value) FROM $ks.key_value""").all().asScala result should have size 3 result.foreach(_.getInt(0) should be > 50) result.foreach(_.getInt(0) should be <= 100) @@ -609,7 +609,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { verifyKeyValueTable("key_value") conn.withSessionDo { session => - val result = session.execute(s"""SELECT WRITETIME(value) FROM $ks.key_value""").all() + val result = session.execute(s"""SELECT WRITETIME(value) FROM $ks.key_value""").all().asScala result should have size 3 result.foreach(_.getLong(0) should be (ts * 1000L)) } @@ -623,7 +623,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { verifyKeyValueTable("key_value") conn.withSessionDo { session => - val result = session.execute(s"""SELECT key, TTL(value) FROM $ks.key_value""").all() + val result = session.execute(s"""SELECT key, TTL(value) FROM $ks.key_value""").all().asScala result should have size 3 result.foreach(row => { row.getInt(1) should be > (100 * row.getInt(0) - 50) @@ -641,7 +641,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { verifyKeyValueTable("key_value") conn.withSessionDo { session => - val result = session.execute(s"""SELECT key, WRITETIME(value) FROM $ks.key_value""").all() + val result = session.execute(s"""SELECT key, WRITETIME(value) FROM $ks.key_value""").all().asScala result should have size 3 result.foreach(row => { row.getLong(1) should be (ts * 1000L + row.getInt(0) * 100L) @@ -660,7 +660,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { verifyKeyValueTable("key_value") conn.withSessionDo { session => - val result = session.execute(s"""SELECT key, TTL(value) FROM $ks.key_value""").all() + val result = session.execute(s"""SELECT key, TTL(value) FROM $ks.key_value""").all().asScala result should have size 3 result.foreach(row => { row.getInt(1) should be > (100 * row.getInt(0) - 50) @@ -683,7 +683,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { verifyKeyValueTable("key_value") conn.withSessionDo { session => - val result = session.execute(s"""SELECT key, WRITETIME(value) FROM $ks.key_value""").all() + val result = session.execute(s"""SELECT key, WRITETIME(value) FROM $ks.key_value""").all().asScala result should have size 3 result.foreach(row => { row.getLong(1) should be (ts * 1000L + row.getInt(0) * 100L) @@ -711,7 +711,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { "map_tuple", SomeColumns(("a" as "_2"), ("c" as "_1"))) conn.withSessionDo { session => - val result = session.execute(s"""SELECT * FROM $ks.map_tuple""").all() + val result = session.execute(s"""SELECT * FROM $ks.map_tuple""").all().asScala result should have size 1 val row = result(0) row.getString("a") should be ("a") @@ -726,7 +726,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { "map_tuple", SomeColumns(("a" as "_2"),("b" as "_3"), ("c" as "_1"))) conn.withSessionDo { session => - val result = session.execute(s"""SELECT * FROM $ks.map_tuple""").all() + val result = session.execute(s"""SELECT * FROM $ks.map_tuple""").all().asScala result should have size 1 val row = result(0) row.getString("a") should be ("a") @@ -771,7 +771,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { verifyKeyValueTable("key_value") } - it should "be able to append and prepend elements to a C* list" in { + it should "be able to.append and.prepend elements to a C* list" in { val listElements = sc.parallelize(Seq( (1, Vector("One")), @@ -783,8 +783,8 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { (1, Vector("PrependTwo")), (1, Vector("PrependThree")))) - listElements.saveToCassandra(ks, "collections_mod", SomeColumns("key", "lcol" append)) - prependElements.saveToCassandra(ks, "collections_mod", SomeColumns("key", "lcol" prepend)) + listElements.saveToCassandra(ks, "collections_mod", SomeColumns("key", "lcol".append)) + prependElements.saveToCassandra(ks, "collections_mod", SomeColumns("key", "lcol".prepend)) val testList = sc.cassandraTable[(Seq[String])](ks, "collections_mod") .where("key = 1") @@ -798,12 +798,12 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { (2, Vector("One")), (2, Vector("Two")), (2, Vector("Three")))) - listElements.saveToCassandra(ks, "collections_mod", SomeColumns("key", "lcol" append)) + listElements.saveToCassandra(ks, "collections_mod", SomeColumns("key", "lcol".append)) sc.parallelize(Seq( (2, Vector("Two")), (2, Vector("Three")))) - .saveToCassandra(ks, "collections_mod", SomeColumns("key", "lcol" remove)) + .saveToCassandra(ks, "collections_mod", SomeColumns("key", "lcol".remove)) val testList = sc.cassandraTable[(Seq[String])](ks, "collections_mod") .where("key = 2") @@ -817,7 +817,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { (3, Set("One")), (3, Set("Two")), (3, Set("Three")))) - setElements.saveToCassandra(ks, "collections_mod", SomeColumns("key", "scol" append)) + setElements.saveToCassandra(ks, "collections_mod", SomeColumns("key", "scol".append)) val testSet = sc.cassandraTable[(Set[String])](ks, "collections_mod") .where("key = 3") .select("scol").take(1)(0) @@ -825,15 +825,15 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { testSet should contain allOf("One", "Two", "Three") } - it should "be able to remove elements from a C* set" in { + it should "be able to.remove elements from a C* set" in { val setElements = sc.parallelize(Seq( (4, Set("One")), (4, Set("Two")), (4, Set("Three")))) - setElements.saveToCassandra(ks, "collections_mod", SomeColumns("key", "scol" append)) + setElements.saveToCassandra(ks, "collections_mod", SomeColumns("key", "scol".append)) sc.parallelize(Seq((4, Set("Two")), (4, Set("Three")))) - .saveToCassandra(ks, "collections_mod", SomeColumns("key", "scol" remove)) + .saveToCassandra(ks, "collections_mod", SomeColumns("key", "scol".remove)) val testSet = sc.cassandraTable[(Set[String])](ks, "collections_mod") .where("key = 4") @@ -848,7 +848,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { (5, Map("One" -> "One")), (5, Map("Two" -> "Two")), (5, Map("Three" -> "Three")))) - setElements.saveToCassandra(ks, "collections_mod", SomeColumns("key", "mcol" append)) + setElements.saveToCassandra(ks, "collections_mod", SomeColumns("key", "mcol".append)) val testMap = sc.cassandraTable[(Map[String, String])](ks, "collections_mod") .where("key = 5") @@ -861,29 +861,28 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { conn.withSessionDo(_.execute(s"""TRUNCATE $ks.key_value""")) val col = Seq((1, 1L, "value1"), (2, 2L, "value2"), (3, 3L, "value3")) val e = intercept[IllegalArgumentException] { - sc.parallelize(col).saveToCassandra(ks, "key_value", SomeColumns("key", "group" - overwrite, "value")) + sc.parallelize(col).saveToCassandra(ks, "key_value", SomeColumns("key", "group".overwrite, "value")) } e.getMessage should include("group") } - it should "throw an exception if you try to remove values from a map" in { + it should "throw an exception if you try to.remove values from a map" in { val setElements = sc.parallelize(Seq( (5, Map("One" -> "One")), (5, Map("Two" -> "Two")), (5, Map("Three" -> "Three")))) val e = intercept[IllegalArgumentException] { - setElements.saveToCassandra(ks, "collections_mod", SomeColumns("key", "mcol" remove)) + setElements.saveToCassandra(ks, "collections_mod", SomeColumns("key", "mcol".remove)) } e.getMessage should include("mcol") } - it should "throw an exception if you prepend anything but a list" in { + it should "throw an exception if you.prepend anything but a list" in { val setElements = sc.parallelize(Seq( (5, Map("One" -> "One"), Set("One")))) val e = intercept[IllegalArgumentException] { - setElements.saveToCassandra(ks, "collections_mod", SomeColumns("key", "mcol" prepend, - "scol" prepend)) + setElements.saveToCassandra(ks, "collections_mod", SomeColumns("key", "mcol".prepend, + "scol".prepend)) } e.getMessage should include("mcol") e.getMessage should include("scol") @@ -916,13 +915,13 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { results should contain theSameElementsAs Seq((1, "new"), (2, "new")) } - "Idempotent Queries" should "not be used with list append" in { - val listAppend = TableWriter(conn, ks, "collections_mod", SomeColumns("key", "lcol" append), WriteConf.fromSparkConf(sc.getConf)) + "Idempotent Queries" should "not be used with list.append" in { + val listAppend = TableWriter(conn, ks, "collections_mod", SomeColumns("key", "lcol".append), WriteConf.fromSparkConf(sc.getConf)) listAppend.isIdempotent should be (false) } - it should "not be used with list prepend" in { - val listPrepend = TableWriter(conn, ks, "collections_mod", SomeColumns("key", "lcol" prepend), WriteConf.fromSparkConf(sc.getConf)) + it should "not be used with list.prepend" in { + val listPrepend = TableWriter(conn, ks, "collections_mod", SomeColumns("key", "lcol".prepend), WriteConf.fromSparkConf(sc.getConf)) listPrepend.isIdempotent should be (false) } @@ -942,11 +941,11 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { } it should "be used with collections that aren't lists" in { - val listOverwrite = TableWriter(conn, ks, "collections_mod", SomeColumns("key", "lcol" overwrite), WriteConf.fromSparkConf(sc.getConf)) + val listOverwrite = TableWriter(conn, ks, "collections_mod", SomeColumns("key", "lcol".overwrite), WriteConf.fromSparkConf(sc.getConf)) listOverwrite.isIdempotent should be (true) - val setOverwrite = TableWriter(conn, ks, "collections_mod", SomeColumns("key", "scol" overwrite), WriteConf.fromSparkConf(sc.getConf)) + val setOverwrite = TableWriter(conn, ks, "collections_mod", SomeColumns("key", "scol".overwrite), WriteConf.fromSparkConf(sc.getConf)) setOverwrite.isIdempotent should be (true) - val mapOverwrite = TableWriter(conn, ks, "collections_mod", SomeColumns("key", "mcol" overwrite), WriteConf.fromSparkConf(sc.getConf)) + val mapOverwrite = TableWriter(conn, ks, "collections_mod", SomeColumns("key", "mcol".overwrite), WriteConf.fromSparkConf(sc.getConf)) mapOverwrite.isIdempotent should be (true) } diff --git a/connector/src/it/scala/org/apache/spark/sql/cassandra/execution/CassandraDirectJoinSpec.scala b/connector/src/it/scala/org/apache/spark/sql/cassandra/execution/CassandraDirectJoinSpec.scala index e727561e8..affd0918f 100644 --- a/connector/src/it/scala/org/apache/spark/sql/cassandra/execution/CassandraDirectJoinSpec.scala +++ b/connector/src/it/scala/org/apache/spark/sql/cassandra/execution/CassandraDirectJoinSpec.scala @@ -336,7 +336,7 @@ class CassandraDirectJoinSpec extends SparkCassandraITFlatSpecBase with DefaultC //Need to wait for a real batch to occur try { - eventually(timeout(scaled(10 seconds))) { + eventually(timeout(scaled(10.seconds))) { getDirectJoin(stream) shouldBe defined } } finally { diff --git a/connector/src/main/java/com/datastax/spark/connector/japi/PairRDDJavaFunctions.java b/connector/src/main/java/com/datastax/spark/connector/japi/PairRDDJavaFunctions.java index 95f139f2a..e9c7ab618 100644 --- a/connector/src/main/java/com/datastax/spark/connector/japi/PairRDDJavaFunctions.java +++ b/connector/src/main/java/com/datastax/spark/connector/japi/PairRDDJavaFunctions.java @@ -32,7 +32,7 @@ public JavaPairRDD> spanByKey(ClassTag keyClassTag) { ClassTag>> tupleClassTag = classTag(Tuple2.class); ClassTag> vClassTag = classTag(Collection.class); RDD>> newRDD = pairRDDFunctions.spanByKey() - .map(JavaApiHelper.>valuesAsJavaCollection(), tupleClassTag); + .map(JavaApiHelper.valuesAsJavaCollection(), tupleClassTag); return new JavaPairRDD<>(newRDD, keyClassTag, vClassTag); } diff --git a/connector/src/main/java/com/datastax/spark/connector/japi/rdd/CassandraJavaPairRDD.java b/connector/src/main/java/com/datastax/spark/connector/japi/rdd/CassandraJavaPairRDD.java index 15960e7f0..c80885405 100644 --- a/connector/src/main/java/com/datastax/spark/connector/japi/rdd/CassandraJavaPairRDD.java +++ b/connector/src/main/java/com/datastax/spark/connector/japi/rdd/CassandraJavaPairRDD.java @@ -57,7 +57,7 @@ public CassandraRDD> rdd() { @SuppressWarnings("unchecked") public CassandraJavaPairRDD select(String... columnNames) { Seq columnRefs = toScalaSeq(toSelectableColumnRefs(columnNames)); - CassandraRDD> newRDD = rdd().select(columnRefs); + CassandraRDD> newRDD = rdd().select(columnRefs.toSeq()); return wrap(newRDD); } @@ -71,7 +71,7 @@ public CassandraJavaPairRDD select(String... columnNames) { @SuppressWarnings("unchecked") public CassandraJavaPairRDD select(ColumnRef... selectionColumns) { Seq columnRefs = JavaApiHelper.toScalaSeq(selectionColumns); - CassandraRDD> newRDD = rdd().select(columnRefs); + CassandraRDD> newRDD = rdd().select(columnRefs.toSeq()); return wrap(newRDD); } diff --git a/connector/src/main/java/com/datastax/spark/connector/japi/rdd/CassandraJavaRDD.java b/connector/src/main/java/com/datastax/spark/connector/japi/rdd/CassandraJavaRDD.java index 86acfde4a..67b60e152 100644 --- a/connector/src/main/java/com/datastax/spark/connector/japi/rdd/CassandraJavaRDD.java +++ b/connector/src/main/java/com/datastax/spark/connector/japi/rdd/CassandraJavaRDD.java @@ -55,7 +55,7 @@ protected CassandraJavaRDD wrap(CassandraRDD newRDD) { */ public CassandraJavaRDD select(String... columnNames) { Seq columnRefs = toScalaSeq(toSelectableColumnRefs(columnNames)); - CassandraRDD newRDD = rdd().select(columnRefs); + CassandraRDD newRDD = rdd().select(columnRefs.toSeq()); return wrap(newRDD); } @@ -68,7 +68,7 @@ public CassandraJavaRDD select(String... columnNames) { */ public CassandraJavaRDD select(ColumnRef... columns) { Seq columnRefs = JavaApiHelper.toScalaSeq(columns); - CassandraRDD newRDD = rdd().select(columnRefs); + CassandraRDD newRDD = rdd().select(columnRefs.toSeq()); return wrap(newRDD); } diff --git a/connector/src/main/java/com/datastax/spark/connector/japi/rdd/CassandraTableScanJavaRDD.java b/connector/src/main/java/com/datastax/spark/connector/japi/rdd/CassandraTableScanJavaRDD.java index f00d7cd3d..73345bfc0 100644 --- a/connector/src/main/java/com/datastax/spark/connector/japi/rdd/CassandraTableScanJavaRDD.java +++ b/connector/src/main/java/com/datastax/spark/connector/japi/rdd/CassandraTableScanJavaRDD.java @@ -2,7 +2,7 @@ import com.datastax.spark.connector.writer.RowWriterFactory; import scala.Tuple2; -import scala.collection.Seq; +import scala.collection.immutable.Seq; import scala.reflect.ClassTag; import org.apache.spark.rdd.RDD; @@ -94,7 +94,7 @@ public CassandraJavaPairRDD keyBy( RowWriterFactory rwf, ColumnRef... columns) { - Seq columnRefs = JavaApiHelper.toScalaSeq(columns); + Seq columnRefs = JavaApiHelper.toScalaImmutableSeq(columns); CassandraRDD> resultRDD = columns.length == 0 ? rdd().keyBy(keyClassTag, rrf, rwf) diff --git a/connector/src/main/java/com/datastax/spark/connector/japi/GenericJavaRowReaderFactory.java b/connector/src/main/scala-2.12/com/datastax/spark/connector/japi/GenericJavaRowReaderFactory.java similarity index 100% rename from connector/src/main/java/com/datastax/spark/connector/japi/GenericJavaRowReaderFactory.java rename to connector/src/main/scala-2.12/com/datastax/spark/connector/japi/GenericJavaRowReaderFactory.java diff --git a/connector/src/main/scala-2.12/com/datastax/spark/connector/util/RuntimeUtil.scala b/connector/src/main/scala-2.12/com/datastax/spark/connector/util/RuntimeUtil.scala new file mode 100644 index 000000000..eff8cd62c --- /dev/null +++ b/connector/src/main/scala-2.12/com/datastax/spark/connector/util/RuntimeUtil.scala @@ -0,0 +1,25 @@ +package com.datastax.spark.connector.util + +import org.apache.spark.repl.SparkILoop +import scala.tools.nsc.Settings +import java.io.{BufferedReader, PrintWriter} +import scala.collection.parallel.ParIterable + +class Scala213SparkILoop(in: BufferedReader, out: PrintWriter) extends SparkILoop(in, out) { + + def run(interpreterSettings: Settings): Boolean = { + super.process(interpreterSettings) + } +} + + +object RuntimeUtil { + + def toParallelIterable[A](iterable: Iterable[A]): ParIterable[A] = { + iterable.par + } + + def createSparkILoop(in: BufferedReader, out: PrintWriter): Scala213SparkILoop = { + new Scala213SparkILoop(in, out) + } +} diff --git a/connector/src/main/scala-2.13/com/datastax/spark/connector/japi/GenericJavaRowReaderFactory.java b/connector/src/main/scala-2.13/com/datastax/spark/connector/japi/GenericJavaRowReaderFactory.java new file mode 100644 index 000000000..41e25d620 --- /dev/null +++ b/connector/src/main/scala-2.13/com/datastax/spark/connector/japi/GenericJavaRowReaderFactory.java @@ -0,0 +1,47 @@ +package com.datastax.spark.connector.japi; + +import com.datastax.oss.driver.api.core.cql.Row; +import com.datastax.spark.connector.CassandraRowMetadata; +import com.datastax.spark.connector.ColumnRef; +import com.datastax.spark.connector.cql.TableDef; +import com.datastax.spark.connector.rdd.reader.RowReader; +import com.datastax.spark.connector.rdd.reader.RowReaderFactory; +import scala.Option; +import scala.collection.immutable.IndexedSeq; +import scala.collection.immutable.Seq; + +public class GenericJavaRowReaderFactory { + public final static RowReaderFactory instance = new RowReaderFactory() { + + @Override + public RowReader rowReader(TableDef table, IndexedSeq selectedColumns) { + return JavaRowReader.instance; + } + + @Override + public Class targetClass() { + return CassandraRow.class; + } + }; + + + public static class JavaRowReader implements RowReader { + public final static JavaRowReader instance = new JavaRowReader(); + + private JavaRowReader() { + } + + @Override + public CassandraRow read(Row row, CassandraRowMetadata metaData) { + assert row.getColumnDefinitions().size() == metaData.columnNames().size() : + "Number of columns in a row must match the number of columns in the table metadata"; + return CassandraRow$.MODULE$.fromJavaDriverRow(row, metaData); + } + + @Override + public Option> neededColumns() { + return Option.empty(); + } + } + +} diff --git a/connector/src/main/scala-2.13/com/datastax/spark/connector/util/RuntimeUtil.scala b/connector/src/main/scala-2.13/com/datastax/spark/connector/util/RuntimeUtil.scala new file mode 100644 index 000000000..ff9995e25 --- /dev/null +++ b/connector/src/main/scala-2.13/com/datastax/spark/connector/util/RuntimeUtil.scala @@ -0,0 +1,19 @@ +package com.datastax.spark.connector.util + +import org.apache.spark.repl.SparkILoop + +import java.io.{BufferedReader, PrintWriter} +import scala.collection.parallel.ParIterable + + +object RuntimeUtil { + + def toParallelIterable[A](iterable: Iterable[A]): ParIterable[A] = { + import scala.collection.parallel.CollectionConverters._ + iterable.par + } + + def createSparkILoop(in: BufferedReader, out: PrintWriter): SparkILoop = { + new SparkILoop(in, out) + } +} diff --git a/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraCatalog.scala b/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraCatalog.scala index ea4941878..8909fb389 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraCatalog.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraCatalog.scala @@ -155,7 +155,7 @@ class CassandraCatalog extends CatalogPlugin val ksMeta: mutable.Map[String, String] = changes.foldLeft(loadNamespaceMetadata(namespace).asScala) { case (metadata: mutable.Map[String, String], setProperty: SetProperty) => - metadata + (setProperty.property() -> setProperty.value) + metadata.clone() += (setProperty.property() -> setProperty.value) case (metadata: mutable.Map[String, String], removeProperty: RemoveProperty) => metadata - removeProperty.property() case (_, other) => throw new CassandraCatalogException(s"Unable to handle alter namespace operation: ${other.getClass.getSimpleName}") @@ -186,7 +186,7 @@ class CassandraCatalog extends CatalogPlugin override def loadNamespaceMetadata(namespace: Array[String]): java.util.Map[String, String] = { val ksMetadata = getKeyspaceMeta(connector, namespace) - (ksMetadata.getReplication.asScala + (DurableWrites -> ksMetadata.isDurableWrites.toString)) + (ksMetadata.getReplication.asScala.clone() += (DurableWrites -> ksMetadata.isDurableWrites.toString)) .asJava } diff --git a/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraSourceUtil.scala b/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraSourceUtil.scala index d798c123f..3ad335972 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraSourceUtil.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraSourceUtil.scala @@ -13,6 +13,7 @@ import org.apache.spark.sql.types.{DataType => CatalystType} import org.apache.spark.sql.types.{BooleanType => SparkSqlBooleanType, DataType => SparkSqlDataType, DateType => SparkSqlDateType, DecimalType => SparkSqlDecimalType, DoubleType => SparkSqlDoubleType, FloatType => SparkSqlFloatType, MapType => SparkSqlMapType, TimestampType => SparkSqlTimestampType, UserDefinedType => SparkSqlUserDefinedType, _} import scala.collection.JavaConverters._ +import scala.language.postfixOps import scala.util.Try object CassandraSourceUtil extends Logging { @@ -147,7 +148,7 @@ object CassandraSourceUtil extends Logging { catalystDataType(dataType, nullable = true), nullable = true) } - StructType(structFields) + StructType(structFields.asJava) } def fromTuple(t: TupleType): StructType = { @@ -157,7 +158,7 @@ object CassandraSourceUtil extends Logging { catalystDataType(dataType, nullable = true), nullable = true) } - StructType(structFields) + StructType(structFields.asJava) } cassandraType match { diff --git a/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraCoGroupedRDD.scala b/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraCoGroupedRDD.scala index dfae0b836..6dd9ada5b 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraCoGroupedRDD.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraCoGroupedRDD.scala @@ -12,7 +12,6 @@ import com.datastax.oss.driver.api.core.CqlSession import com.datastax.oss.driver.api.core.cql.{BoundStatement, Row} import com.datastax.spark.connector.util._ -import scala.collection.JavaConversions._ import scala.language.existentials import scala.reflect.ClassTag import org.apache.spark.annotation.DeveloperApi @@ -32,6 +31,7 @@ import com.datastax.spark.connector.util.{CountingIterator, MultiMergeJoinIterat import scala.concurrent.Await import scala.concurrent.duration.Duration +import scala.collection.JavaConverters._ /** * A RDD which pulls from provided separate CassandraTableScanRDDs which share partition keys type and @@ -127,7 +127,7 @@ class CassandraCoGroupedRDD[T]( try { val stmt = session.prepare(cql) - val converters = stmt.getVariableDefinitions + val converters = stmt.getVariableDefinitions.asScala .map(v => ColumnType.converterToCassandra(v.getType)) .toArray val convertedValues = diff --git a/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraMergeJoinRDD.scala b/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraMergeJoinRDD.scala index 862f3f8bd..fe0bdb10c 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraMergeJoinRDD.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraMergeJoinRDD.scala @@ -9,7 +9,6 @@ import java.io.IOException import com.datastax.bdp.util.ScalaJavaUtil.asScalaFuture -import scala.collection.JavaConversions._ import scala.language.existentials import scala.reflect.ClassTag import org.apache.spark.annotation.DeveloperApi @@ -30,6 +29,7 @@ import com.datastax.spark.connector.util.{CountingIterator, MergeJoinIterator, N import scala.concurrent.Await import scala.concurrent.duration.Duration +import scala.collection.JavaConverters._ /** * A RDD which pulls from two separate CassandraTableScanRDDs which share partition keys and @@ -115,7 +115,7 @@ class CassandraMergeJoinRDD[L,R]( try { val stmt = session.prepare(cql) - val converters = stmt.getVariableDefinitions + val converters = stmt.getVariableDefinitions.asScala .map(v => ColumnType.converterToCassandra(v.getType)) .toArray val convertedValues = diff --git a/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraTableScanRDD.scala b/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraTableScanRDD.scala index 3f02f5058..57f97f133 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraTableScanRDD.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraTableScanRDD.scala @@ -15,9 +15,9 @@ import org.apache.spark.rdd.{PartitionCoalescer, RDD} import org.apache.spark.{Partition, Partitioner, SparkContext, TaskContext} import java.io.IOException -import scala.collection.JavaConversions._ import scala.language.existentials import scala.reflect.ClassTag +import scala.collection.JavaConverters._ /** RDD representing a Table Scan of A Cassandra table. @@ -186,7 +186,7 @@ class CassandraTableScanRDD[R] private[connector]( val selectedColumnNames = columns.selectFrom(tableDef).map(_.columnName).toSet val partitionKeyColumnNames = PartitionKeyColumns.selectFrom(tableDef).map(_.columnName).toSet - if (selectedColumnNames.containsAll(partitionKeyColumnNames)) { + if (selectedColumnNames.asJava.containsAll(partitionKeyColumnNames.asJava)) { val partitioner = partitionGenerator.partitioner[K](columns) logDebug( s"""Made partitioner ${partitioner} for $this""".stripMargin) diff --git a/connector/src/main/scala/com/datastax/spark/connector/rdd/DseGraphUnionedRDD.scala b/connector/src/main/scala/com/datastax/spark/connector/rdd/DseGraphUnionedRDD.scala index d40399698..bb1b3fb57 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/rdd/DseGraphUnionedRDD.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/rdd/DseGraphUnionedRDD.scala @@ -10,7 +10,6 @@ import java.lang.{String => JString} import java.util.{Map => JMap} import scala.annotation.meta.param -import scala.collection.JavaConversions._ import scala.language.existentials import scala.reflect.ClassTag import org.apache.spark.rdd.{RDD, UnionRDD} @@ -20,6 +19,7 @@ import com.datastax.spark.connector.rdd.partitioner.dht.{Token, TokenFactory} import com.datastax.spark.connector.rdd.partitioner.{BucketingRangeIndex, CassandraPartition, TokenGenerator, TokenRangeWithPartitionIndex} import com.datastax.spark.connector.util.{Logging, schemaFromCassandra} import com.datastax.spark.connector.writer.RowWriter +import scala.collection.JavaConverters._ /** * A Java Friendly api for DseGraphUnionedRDD to make it easier to call @@ -33,8 +33,8 @@ object DseGraphUnionedRDD { graphLabels: java.util.List[String])( implicit connector: CassandraConnector): DseGraphUnionedRDD[R] = { - val rddSeq: Seq[RDD[R]] = rdds - val labelSeq: Seq[String] = graphLabels + val rddSeq: Seq[RDD[R]] = rdds.asScala.toSeq + val labelSeq: Seq[String] = graphLabels.asScala.toSeq new DseGraphUnionedRDD(sc, rddSeq, keyspace, labelSeq) } } @@ -188,7 +188,7 @@ class DseGraphPartitioner[V, T <: Token[V]]( */ override def getPartition(key: Any): Int = key match { case vertexId: JMap[JString, AnyRef]@unchecked => { - val label: String = vertexId.getOrElse( + val label: String = vertexId.asScala.getOrElse( LabelAccessor, throw new IllegalArgumentException(s"Couldn't find $LabelAccessor in key $key")) .asInstanceOf[String] @@ -217,7 +217,7 @@ class DseGraphPartitioner[V, T <: Token[V]]( class MapRowWriter(override val columnNames: Seq[String]) extends RowWriter[JMap[JString, AnyRef]] { override def readColumnValues(data: JMap[JString, AnyRef], buffer: Array[Any]): Unit = columnNames.zipWithIndex.foreach { case (columnName, index) => - buffer(index) = data.getOrElse(columnName, + buffer(index) = data.asScala.getOrElse(columnName, throw new IllegalArgumentException(s"""Couldn't find $columnName in $data, unable to generate token""")) } } diff --git a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/BucketingRangeIndex.scala b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/BucketingRangeIndex.scala index 45cfa3e0d..8a654acbc 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/BucketingRangeIndex.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/BucketingRangeIndex.scala @@ -74,5 +74,5 @@ class BucketingRangeIndex[R, T](ranges: Seq[R]) /** Finds rangesContaining containing given point in O(1) time. */ def rangesContaining(point: T): IndexedSeq[R] = - table(bucket(point)).filter(bounds.contains(_, point)) + table(bucket(point)).filter(bounds.contains(_, point)).toIndexedSeq } diff --git a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/CassandraPartitionGenerator.scala b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/CassandraPartitionGenerator.scala index f7fadeb5a..eaf9d853f 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/CassandraPartitionGenerator.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/CassandraPartitionGenerator.scala @@ -4,7 +4,6 @@ import com.datastax.oss.driver.api.core.CqlIdentifier import com.datastax.oss.driver.api.core.metadata.TokenMap import com.datastax.oss.driver.api.core.metadata.token.{TokenRange => DriverTokenRange} -import scala.collection.JavaConversions._ import scala.language.existentials import scala.reflect.ClassTag import scala.util.Try @@ -15,6 +14,7 @@ import com.datastax.spark.connector.cql.{CassandraConnector, TableDef} import com.datastax.spark.connector.rdd.partitioner.dht.{Token, TokenFactory} import com.datastax.spark.connector.writer.RowWriterFactory import org.apache.spark.sql.connector.read.InputPartition +import scala.collection.JavaConverters._ /** Creates CassandraPartitions for given Cassandra table */ @@ -36,7 +36,7 @@ private[connector] class CassandraPartitionGenerator[V, T <: Token[V]]( val startToken = tokenFactory.tokenFromString(metadata.format(range.getStart)) val endToken = tokenFactory.tokenFromString(metadata.format(range.getEnd)) val replicas = metadata - .getReplicas(keyspaceName, range) + .getReplicas(keyspaceName, range).asScala .map(node => DriverUtil.toAddress(node) .getOrElse(throw new IllegalStateException(s"Unable to determine Node Broadcast Address of $node"))) @@ -49,7 +49,7 @@ private[connector] class CassandraPartitionGenerator[V, T <: Token[V]]( val ranges = connector.withSessionDo { session => val tokenMap = Option(session.getMetadata.getTokenMap.get) .getOrElse(throw new IllegalStateException("Unable to determine Token Range Metadata")) - for (tr <- tokenMap.getTokenRanges()) yield tokenRange(tr, tokenMap) + for (tr <- tokenMap.getTokenRanges().asScala) yield tokenRange(tr, tokenMap) } /** diff --git a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/DataSizeEstimates.scala b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/DataSizeEstimates.scala index bd2cd524c..a03df8d66 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/DataSizeEstimates.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/DataSizeEstimates.scala @@ -1,11 +1,11 @@ package com.datastax.spark.connector.rdd.partitioner -import scala.collection.JavaConversions._ import com.datastax.spark.connector.util.Logging import com.datastax.oss.driver.api.core.cql.SimpleStatementBuilder import com.datastax.oss.driver.api.core.servererrors.InvalidQueryException import com.datastax.spark.connector.cql.CassandraConnector import com.datastax.spark.connector.rdd.partitioner.dht.{Token, TokenFactory} +import scala.collection.JavaConverters._ /** Estimates amount of data in the Cassandra table. @@ -36,17 +36,19 @@ class DataSizeEstimates[V, T <: Token[V]]( private lazy val tokenRanges: Seq[TokenRangeSizeEstimate] = conn.withSessionDo { session => try { - val rs = session.execute(new SimpleStatementBuilder( - "SELECT range_start, range_end, partitions_count, mean_partition_size " + - "FROM system.size_estimates " + - "WHERE keyspace_name = ? AND table_name = ?").addPositionalValues(keyspaceName, tableName).build()) - - for (row <- rs.all()) yield TokenRangeSizeEstimate( - rangeStart = tokenFactory.tokenFromString(row.getString("range_start")), - rangeEnd = tokenFactory.tokenFromString(row.getString("range_end")), - partitionsCount = row.getLong("partitions_count"), - meanPartitionSize = row.getLong("mean_partition_size") - ) + { + val rs = session.execute(new SimpleStatementBuilder( + "SELECT range_start, range_end, partitions_count, mean_partition_size " + + "FROM system.size_estimates " + + "WHERE keyspace_name = ? AND table_name = ?").addPositionalValues(keyspaceName, tableName).build()) + + for (row <- rs.all().asScala) yield TokenRangeSizeEstimate( + rangeStart = tokenFactory.tokenFromString(row.getString("range_start")), + rangeEnd = tokenFactory.tokenFromString(row.getString("range_end")), + partitionsCount = row.getLong("partitions_count"), + meanPartitionSize = row.getLong("mean_partition_size") + ) + }.toSeq // The table may not contain the estimates yet if the data was just inserted and the // amount of data in the table was small. This is very common situation during tests, @@ -103,7 +105,7 @@ object DataSizeEstimates { def hasSizeEstimates: Boolean = { session.execute( s"SELECT * FROM system.size_estimates " + - s"WHERE keyspace_name = '$keyspaceName' AND table_name = '$tableName'").all().nonEmpty + s"WHERE keyspace_name = '$keyspaceName' AND table_name = '$tableName'").all().asScala.nonEmpty } val startTime = System.currentTimeMillis() diff --git a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/NodeAddresses.scala b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/NodeAddresses.scala index e068943af..2e9e838a2 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/NodeAddresses.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/NodeAddresses.scala @@ -2,9 +2,9 @@ package com.datastax.spark.connector.rdd.partitioner import java.net.InetAddress -import scala.collection.JavaConversions._ import com.datastax.spark.connector.cql.CassandraConnector import com.datastax.spark.connector.util.DriverUtil.{toName, toOption} +import scala.collection.JavaConverters._ /** Looks up listen address of a cluster node given its Native Transport address. * Uses system.peers table as the source of information. @@ -28,7 +28,7 @@ class NodeAddresses(conn: CassandraConnector) extends Serializable { // TODO: fetch information about the local node from system.local, when CASSANDRA-9436 is done val rs = session.execute(s"SELECT $nativeTransportAddressColumnName, $listenAddressColumnName FROM $table") for { - row <- rs.all() + row <- rs.all().asScala nativeTransportAddress <- Option(row.getInetAddress(nativeTransportAddressColumnName)) listenAddress = row.getInetAddress(listenAddressColumnName) } yield (nativeTransportAddress, listenAddress) diff --git a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/ReplicaPartitioner.scala b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/ReplicaPartitioner.scala index 10eb90955..be6cfdf94 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/ReplicaPartitioner.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/ReplicaPartitioner.scala @@ -9,8 +9,8 @@ import com.datastax.spark.connector.util._ import com.datastax.spark.connector.writer.RowWriterFactory import org.apache.spark.{Partition, Partitioner} -import scala.collection.JavaConversions._ import scala.reflect.ClassTag +import scala.collection.JavaConverters._ case class ReplicaPartition(index: Int, endpoints: Array[String]) extends EndpointPartition @@ -73,7 +73,7 @@ implicit val token = tokenGenerator.getTokenFor(key) val tokenHash = Math.abs(token.hashCode()) val replicas = tokenMap - .getReplicas(_keyspace, token) + .getReplicas(_keyspace, token).asScala .map(n => DriverUtil.toAddress(n).get.getAddress) val replicaSetInDC = (hostSet & replicas).toVector diff --git a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/TokenRangeSplitter.scala b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/TokenRangeSplitter.scala index 280f723f3..b32e76601 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/TokenRangeSplitter.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/TokenRangeSplitter.scala @@ -1,10 +1,10 @@ package com.datastax.spark.connector.rdd.partitioner -import scala.collection.parallel.ForkJoinTaskSupport -import scala.concurrent.forkjoin.ForkJoinPool - +import scala.collection.parallel.{ForkJoinTaskSupport, ParIterable} +import java.util.concurrent.ForkJoinPool import com.datastax.spark.connector.rdd.partitioner.TokenRangeSplitter.WholeRing import com.datastax.spark.connector.rdd.partitioner.dht.{Token, TokenRange} +import com.datastax.spark.connector.util.RuntimeUtil /** Splits a token ranges into smaller sub-ranges, @@ -14,7 +14,7 @@ private[partitioner] trait TokenRangeSplitter[V, T <: Token[V]] { def split(tokenRanges: Iterable[TokenRange[V, T]], splitCount: Int): Iterable[TokenRange[V, T]] = { val ringFractionPerSplit = WholeRing / splitCount.toDouble - val parTokenRanges = tokenRanges.par + val parTokenRanges: ParIterable[TokenRange[V, T]] = RuntimeUtil.toParallelIterable(tokenRanges) parTokenRanges.tasksupport = new ForkJoinTaskSupport(TokenRangeSplitter.pool) parTokenRanges.flatMap(tokenRange => { diff --git a/connector/src/main/scala/com/datastax/spark/connector/util/JavaApiHelper.scala b/connector/src/main/scala/com/datastax/spark/connector/util/JavaApiHelper.scala index 6f6725b68..41a1c924b 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/util/JavaApiHelper.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/util/JavaApiHelper.scala @@ -4,7 +4,6 @@ import java.lang.{Iterable => JIterable} import java.util.{Collection => JCollection} import java.util.{Map => JMap} -import scala.collection.JavaConversions._ import scala.reflect._ import scala.reflect.api.{Mirror, TypeCreator, _} import scala.reflect.runtime.universe._ @@ -15,6 +14,7 @@ import com.datastax.spark.connector.CassandraRow import com.datastax.spark.connector.mapper.{ColumnMapper, JavaBeanColumnMapper} import com.datastax.spark.connector.rdd.reader.RowReaderFactory import com.datastax.spark.connector.writer.RowWriterFactory +import scala.collection.JavaConverters._ /** A helper class to make it possible to access components written in Scala from Java code. * INTERNAL API @@ -55,11 +55,11 @@ object JavaApiHelper { def toScalaFunction1[T1, R](f: JFunction[T1, R]): T1 => R = f.call def valuesAsJavaIterable[K, V, IV <: Iterable[V]]: ((K, IV)) => (K, JIterable[V]) = { - case (k, iterable) => (k, asJavaIterable(iterable)) + case (k, iterable) => (k, iterable.asJava) } def valuesAsJavaCollection[K, V, IV <: Iterable[V]]: ((K, IV)) => (K, JCollection[V]) = { - case (k, iterable) => (k, asJavaCollection(iterable)) + case (k, iterable) => (k, iterable.asJavaCollection) } /** Returns a runtime class of a given `TypeTag`. */ @@ -71,7 +71,7 @@ object JavaApiHelper { classTag.runtimeClass.asInstanceOf[Class[T]] /** Converts a Java `Map` to a Scala immutable `Map`. */ - def toScalaMap[K, V](map: JMap[K, V]): Map[K, V] = Map(map.toSeq: _*) + def toScalaMap[K, V](map: JMap[K, V]): Map[K, V] = Map(map.asScala.toSeq: _*) /** Converts an array to a Scala `Seq`. */ def toScalaSeq[T](array: Array[T]): Seq[T] = array @@ -80,7 +80,7 @@ object JavaApiHelper { def toScalaImmutableSeq[T](array: Array[T]): scala.collection.immutable.Seq[T] = array.toIndexedSeq /** Converts a Java `Iterable` to Scala `Seq`. */ - def toScalaSeq[T](iterable: java.lang.Iterable[T]): Seq[T] = iterable.toSeq + def toScalaSeq[T](iterable: java.lang.Iterable[T]): Seq[T] = iterable.asScala.toSeq /** Returns the default `RowWriterFactory` initialized with the given `ColumnMapper`. */ def defaultRowWriterFactory[T](typeTag: TypeTag[T], mapper: ColumnMapper[T]): RowWriterFactory[T] = { diff --git a/connector/src/main/scala/com/datastax/spark/connector/util/MergeJoinIterator.scala b/connector/src/main/scala/com/datastax/spark/connector/util/MergeJoinIterator.scala index bdee74312..07d2c88fc 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/util/MergeJoinIterator.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/util/MergeJoinIterator.scala @@ -63,6 +63,6 @@ extends Iterator[(K, Seq[L], Seq[R])] { val bufferRight = new ArrayBuffer[R] itemsLeft.appendWhile(l => keyExtractLeft(l) == key, bufferLeft) itemsRight.appendWhile(r => keyExtractRight(r) == key, bufferRight) - (key, bufferLeft, bufferRight) + (key, bufferLeft.toSeq, bufferRight.toSeq) } } diff --git a/connector/src/main/scala/com/datastax/spark/connector/util/MultiMergeJoinIterator.scala b/connector/src/main/scala/com/datastax/spark/connector/util/MultiMergeJoinIterator.scala index 2690cd274..02debe048 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/util/MultiMergeJoinIterator.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/util/MultiMergeJoinIterator.scala @@ -48,7 +48,7 @@ extends Iterator[Seq[Seq[T]]] { items.map (i => { var buffer = new ArrayBuffer[T] i.appendWhile(l => keyExtract(l) == key, buffer) - buffer + buffer.toSeq }) } } diff --git a/connector/src/main/scala/com/datastax/spark/connector/util/SpanningIterator.scala b/connector/src/main/scala/com/datastax/spark/connector/util/SpanningIterator.scala index 2d32cdc3b..8e5103540 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/util/SpanningIterator.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/util/SpanningIterator.scala @@ -26,6 +26,6 @@ class SpanningIterator[K, T](iterator: Iterator[T], f: T => K) extends Iterator[ val key = f(items.head) val buffer = new ArrayBuffer[T] items.appendWhile(r => f(r) == key, buffer) - (key, buffer) + (key, buffer.toSeq) } } diff --git a/connector/src/main/scala/com/datastax/spark/connector/writer/Batch.scala b/connector/src/main/scala/com/datastax/spark/connector/writer/Batch.scala index 9efa2fee1..7f2856a0d 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/writer/Batch.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/writer/Batch.scala @@ -15,7 +15,7 @@ private[writer] sealed trait Batch extends Ordered[Batch] { def add(stmt: RichBoundStatementWrapper, force: Boolean = false): Boolean /** Collected statements */ - def statements: Seq[RichBoundStatementWrapper] = buf + def statements: Seq[RichBoundStatementWrapper] = buf.toSeq /** Only for internal use - batches are compared by this value. */ protected[Batch] def size: Int diff --git a/connector/src/main/scala/com/datastax/spark/connector/writer/ObjectSizeEstimator.scala b/connector/src/main/scala/com/datastax/spark/connector/writer/ObjectSizeEstimator.scala index fa471ffa0..8d1d91140 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/writer/ObjectSizeEstimator.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/writer/ObjectSizeEstimator.scala @@ -3,9 +3,9 @@ package com.datastax.spark.connector.writer import java.io.{OutputStream, ObjectOutputStream} import java.nio.ByteBuffer -import scala.collection.JavaConversions._ import com.datastax.spark.connector.util.ByteBufferUtil +import scala.collection.JavaConverters._ /** Estimates amount of memory required to serialize Java/Scala objects */ @@ -14,11 +14,11 @@ object ObjectSizeEstimator { private def makeSerializable(obj: Any): AnyRef = { obj match { case bb: ByteBuffer => ByteBufferUtil.toArray(bb) - case list: java.util.List[_] => list.map(makeSerializable) + case list: java.util.List[_] => list.asScala.map(makeSerializable) case list: List[_] => list.map(makeSerializable) - case set: java.util.Set[_] => set.map(makeSerializable) + case set: java.util.Set[_] => set.asScala.map(makeSerializable) case set: Set[_] => set.map(makeSerializable) - case map: java.util.Map[_, _] => map.map { case (k, v) => (makeSerializable(k), makeSerializable(v)) } + case map: java.util.Map[_, _] => map.asScala.map { case (k, v) => (makeSerializable(k), makeSerializable(v)) } case map: Map[_, _] => map.map { case (k, v) => (makeSerializable(k), makeSerializable(v)) } case other => other.asInstanceOf[AnyRef] } diff --git a/connector/src/main/scala/com/datastax/spark/connector/writer/ReplicaLocator.scala b/connector/src/main/scala/com/datastax/spark/connector/writer/ReplicaLocator.scala index eaa6c3d1b..5b3411f5f 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/writer/ReplicaLocator.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/writer/ReplicaLocator.scala @@ -10,8 +10,8 @@ import com.datastax.spark.connector.util.DriverUtil._ import com.datastax.spark.connector.util.{DriverUtil, Logging, tableFromCassandra} import com.datastax.spark.connector.util.PatitionKeyTools._ -import scala.collection.JavaConversions._ import scala.collection._ +import scala.collection.JavaConverters._ /** * A utility class for determining the Replica Set (Ip Addresses) of a particular Cassandra Row. Used @@ -42,7 +42,7 @@ class ReplicaLocator[T] private( val clusterMetadata = session.getMetadata data.map { row => val hosts = tokenMap - .getReplicas(CqlIdentifier.fromInternal(keyspaceName), QueryUtils.getRoutingKeyOrError(boundStmtBuilder.bind(row).stmt)) + .getReplicas(CqlIdentifier.fromInternal(keyspaceName), QueryUtils.getRoutingKeyOrError(boundStmtBuilder.bind(row).stmt)).asScala .map(node => DriverUtil.toAddress(node) .getOrElse(throw new IllegalStateException(s"Unable to determine Node Broadcast Address of $node"))) .map(_.getAddress) diff --git a/connector/src/main/scala/com/datastax/spark/connector/writer/TableWriter.scala b/connector/src/main/scala/com/datastax/spark/connector/writer/TableWriter.scala index 5c4c255d5..35f1abb54 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/writer/TableWriter.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/writer/TableWriter.scala @@ -212,7 +212,7 @@ class TableWriter[T] private ( 1024 * 1024) (stmt: RichStatement) => rateLimiter.maybeSleep(stmt.bytesCount) case None => - (stmt: RichStatement) => Unit + (stmt: RichStatement) => () } AsyncStatementWriter(connector, writeConf, tableDef, stmt, batchBuilder, maybeRateLimit) diff --git a/connector/src/main/scala/com/datastax/spark/connector/writer/WriteOption.scala b/connector/src/main/scala/com/datastax/spark/connector/writer/WriteOption.scala index 83ab25818..6aa07d772 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/writer/WriteOption.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/writer/WriteOption.scala @@ -42,7 +42,7 @@ object TTLOption { def constant(ttl: JodaDuration): TTLOption = constant(ttl.getStandardSeconds.toInt) - def constant(ttl: ScalaDuration): TTLOption = if (ttl.isFinite()) constant(ttl.toSeconds.toInt) else forever + def constant(ttl: ScalaDuration): TTLOption = if (ttl.isFinite) constant(ttl.toSeconds.toInt) else forever def perRow(placeholder: String): TTLOption = TTLOption(PerRowWriteOptionValue[Int](placeholder)) diff --git a/connector/src/main/scala/org/apache/spark/metrics/CassandraSink.scala b/connector/src/main/scala/org/apache/spark/metrics/CassandraSink.scala index 6ba0a177e..1ad24bbbf 100644 --- a/connector/src/main/scala/org/apache/spark/metrics/CassandraSink.scala +++ b/connector/src/main/scala/org/apache/spark/metrics/CassandraSink.scala @@ -10,13 +10,13 @@ import java.util.Properties import java.util.concurrent.{Executors, TimeUnit} import java.util.function.BiConsumer -import scala.collection.JavaConversions._ import com.codahale.metrics.{Counting, Gauge, Metered, Metric, MetricRegistry, Sampling} import org.apache.spark.metrics.sink.Sink import org.apache.spark.{SecurityManager, SparkConf, SparkEnv} import com.datastax.oss.driver.api.core.cql.{AsyncResultSet, ResultSet} import com.datastax.spark.connector.cql.CassandraConnector import com.datastax.spark.connector.util.Logging +import scala.collection.JavaConverters._ class CassandraSink(val properties: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink with Runnable with Logging { @@ -55,7 +55,7 @@ class CassandraSink(val properties: Properties, val registry: MetricRegistry, se connector.withSessionDo { session => val stmt = session.prepare(writer.insertStatement) - for ((MetricName(appId, componentId, metricId), metric) <- registry.getMetrics.iterator) { + for ((MetricName(appId, componentId, metricId), metric) <- registry.getMetrics.asScala.iterator) { val bndStmt = stmt.bind(writer.build(componentId, metricId, metric): _*) session.executeAsync(bndStmt).whenComplete(warnOnError) } diff --git a/connector/src/main/scala/org/apache/spark/sql/cassandra/BasicCassandraPredicatePushDown.scala b/connector/src/main/scala/org/apache/spark/sql/cassandra/BasicCassandraPredicatePushDown.scala index f6769890a..5e1f27214 100644 --- a/connector/src/main/scala/org/apache/spark/sql/cassandra/BasicCassandraPredicatePushDown.scala +++ b/connector/src/main/scala/org/apache/spark/sql/cassandra/BasicCassandraPredicatePushDown.scala @@ -49,14 +49,14 @@ class BasicCassandraPredicatePushDown[Predicate : PredicateOps]( private val eqPredicatesByName = eqPredicates .groupBy(Predicates.columnName) - .mapValues(_.take(1)) // don't push down more than one EQ predicate for the same column + .map{ case (k, v) => (k, v.take(1))} .withDefaultValue(Set.empty) private val inPredicates = singleColumnPredicates.filter(Predicates.isInPredicate) private val inPredicatesByName = inPredicates .groupBy(Predicates.columnName) - .mapValues(_.take(1)) // don't push down more than one IN predicate for the same column + .map{ case (k, v) => (k, v.take(1))} // don't push down more than one IN predicate for the same column .withDefaultValue(Set.empty) private val rangePredicates = singleColumnPredicates.filter(Predicates.isRangePredicate) diff --git a/connector/src/main/scala/org/apache/spark/sql/cassandra/DsePredicateRules.scala b/connector/src/main/scala/org/apache/spark/sql/cassandra/DsePredicateRules.scala index 5710c3057..fbd2df1da 100644 --- a/connector/src/main/scala/org/apache/spark/sql/cassandra/DsePredicateRules.scala +++ b/connector/src/main/scala/org/apache/spark/sql/cassandra/DsePredicateRules.scala @@ -11,6 +11,7 @@ import com.datastax.spark.connector.util.Logging import org.apache.spark.SparkConf import org.apache.spark.sql.cassandra.PredicateOps.FilterOps import org.apache.spark.sql.sources.{EqualTo, Filter, IsNotNull} +import scala.collection.compat._ /** @@ -76,7 +77,7 @@ object DsePredicateRules extends CassandraPredicateRules with Logging { val indexColumns = saiIndexes(table).map(_.targetColumn) val pushedEqualityPredicates = predicates.handledByCassandra.collect { case f if FilterOps.isEqualToPredicate(f) => FilterOps.columnName(f) - }.to[collection.mutable.Set] + }.to(collection.mutable.Set) val (handledByCassandra, handledBySpark) = predicates.handledBySpark.partition { filter => lazy val columnName = FilterOps.columnName(filter) diff --git a/connector/src/main/scala/org/apache/spark/sql/cassandra/execution/CassandraDirectJoinStrategy.scala b/connector/src/main/scala/org/apache/spark/sql/cassandra/execution/CassandraDirectJoinStrategy.scala index 159976475..e0ee2d322 100644 --- a/connector/src/main/scala/org/apache/spark/sql/cassandra/execution/CassandraDirectJoinStrategy.scala +++ b/connector/src/main/scala/org/apache/spark/sql/cassandra/execution/CassandraDirectJoinStrategy.scala @@ -103,7 +103,7 @@ case class CassandraDirectJoinStrategy(spark: SparkSession) extends Strategy wit .get(DirectJoinSizeRatioParam.name, DirectJoinSizeRatioParam.default.toString)) val cassandraSize = BigDecimal(cassandraPlan.stats.sizeInBytes) - val keySize = BigDecimal(keyPlan.stats.sizeInBytes.doubleValue()) + val keySize = BigDecimal(keyPlan.stats.sizeInBytes.doubleValue) logDebug(s"Checking if size ratio is good: $cassandraSize * $ratio > $keySize") diff --git a/connector/src/test/scala/com/datastax/spark/connector/embedded/SparkRepl.scala b/connector/src/test/scala/com/datastax/spark/connector/embedded/SparkRepl.scala index e3d255028..13c6e00b1 100644 --- a/connector/src/test/scala/com/datastax/spark/connector/embedded/SparkRepl.scala +++ b/connector/src/test/scala/com/datastax/spark/connector/embedded/SparkRepl.scala @@ -1,8 +1,9 @@ package com.datastax.spark.connector.embedded +import com.datastax.spark.connector.util.RuntimeUtil + import java.io._ import java.net.URLClassLoader - import org.apache.spark.SparkConf import org.apache.spark.repl.{Main, SparkILoop} @@ -27,12 +28,12 @@ object SparkRepl { } Main.conf.setAll(conf.getAll) - val interp = new SparkILoop(Some(in), new PrintWriter(out)) + val interp = RuntimeUtil.createSparkILoop(in, new PrintWriter(out)) Main.interp = interp val separator = System.getProperty("path.separator") val settings = new GenericRunnerSettings(s => throw new RuntimeException(s"Scala options error: $s")) settings.processArguments(List("-classpath", paths.mkString(separator)), processAll = true) - interp.process(settings) // Repl starts and goes in loop of R.E.P.L + interp.run(settings) // Repl starts and goes in loop of R.E.P.L Main.interp = null Option(Main.sparkContext).foreach(_.stop()) System.clearProperty("spark.driver.port") diff --git a/connector/src/test/scala/org/apache/spark/metrics/InputMetricsUpdaterSpec.scala b/connector/src/test/scala/org/apache/spark/metrics/InputMetricsUpdaterSpec.scala index 6667781ee..b33aeee05 100644 --- a/connector/src/test/scala/org/apache/spark/metrics/InputMetricsUpdaterSpec.scala +++ b/connector/src/test/scala/org/apache/spark/metrics/InputMetricsUpdaterSpec.scala @@ -10,6 +10,7 @@ import org.scalatest.concurrent.Eventually import org.scalatest.{FlatSpec, Matchers} import com.datastax.driver.core.RowMock import com.datastax.spark.connector.rdd.ReadConf +import com.datastax.spark.connector.util.RuntimeUtil.toParallelIterable import org.scalatestplus.mockito.MockitoSugar class InputMetricsUpdaterSpec extends FlatSpec with Matchers with MockitoSugar { @@ -55,8 +56,8 @@ class InputMetricsUpdaterSpec extends FlatSpec with Matchers with MockitoSugar { val updater = InputMetricsUpdater(tc, ReadConf.fromSparkConf(conf)) val row = new RowMock(Some(1), Some(2), Some(3), None, Some(4)) - val range = (1 to 1000).par - range.tasksupport = new ForkJoinTaskSupport(new scala.concurrent.forkjoin.ForkJoinPool(10)) + val range = toParallelIterable(1 to 1000).par + range.tasksupport = new ForkJoinTaskSupport(new java.util.concurrent.ForkJoinPool(10)) for (i <- range) updater.updateMetrics(row) updater.finish() tc.taskMetrics().inputMetrics.bytesRead shouldBe 10000L @@ -111,8 +112,8 @@ class InputMetricsUpdaterSpec extends FlatSpec with Matchers with MockitoSugar { ccs.readRowMeter.getCount shouldBe 0 ccs.readByteMeter.getCount shouldBe 0 - val range = (1 to 1000).par - range.tasksupport = new ForkJoinTaskSupport(new scala.concurrent.forkjoin.ForkJoinPool(10)) + val range = toParallelIterable(1 to 1000).par + range.tasksupport = new ForkJoinTaskSupport(new java.util.concurrent.ForkJoinPool(10)) for (i <- range) updater.updateMetrics(row) updater.finish() diff --git a/driver/src/main/scala/com/datastax/spark/connector/types/CanBuildFrom.scala b/driver/src/main/scala-2.12/types/CanBuildFrom.scala similarity index 100% rename from driver/src/main/scala/com/datastax/spark/connector/types/CanBuildFrom.scala rename to driver/src/main/scala-2.12/types/CanBuildFrom.scala diff --git a/driver/src/main/scala-2.13/com/datastax/spark/connector/types/CanBuildFrom.scala b/driver/src/main/scala-2.13/com/datastax/spark/connector/types/CanBuildFrom.scala new file mode 100644 index 000000000..98ff49b5f --- /dev/null +++ b/driver/src/main/scala-2.13/com/datastax/spark/connector/types/CanBuildFrom.scala @@ -0,0 +1,90 @@ +package com.datastax.spark.connector.types + +import scala.collection.immutable.{TreeMap, TreeSet} +import scala.collection.mutable + +/** Serves the same purpose as standard Scala `CanBuildFrom`, however this one is `Serializable`. */ +trait CanBuildFrom[-Elem, To] extends Serializable { + + def apply(): mutable.Builder[Elem, To] + + // it is safe to upcast. because CanBuildFrom is in fact covariant in To. However we cannot mark it + // covariant, because that would trigger implicit resolution ambiguities and + // the workaround for them is much more complex than this "asInstanceOf" hack. + def upcast[T >: To] = this.asInstanceOf[CanBuildFrom[Elem, T]] +} + +object CanBuildFrom { + + implicit def iterableCanBuildFrom[T] = new CanBuildFrom[T, Iterable[T]] { + override def apply() = Iterable.newBuilder[T] + } + + implicit def seqCanBuildFrom[T] = new CanBuildFrom[T, Seq[T]] { + override def apply() = Seq.newBuilder[T] + } + + implicit def indexedSeqCanBuildFrom[T] = new CanBuildFrom[T, IndexedSeq[T]] { + override def apply() = IndexedSeq.newBuilder[T] + } + + implicit def vectorCanBuildFrom[T] = new CanBuildFrom[T, Vector[T]] { + override def apply() = Vector.newBuilder[T] + } + + implicit def listCanBuildFrom[T] = new CanBuildFrom[T, List[T]] { + override def apply() = List.newBuilder[T] + } + + implicit def setCanBuildFrom[T] = new CanBuildFrom[T, Set[T]] { + override def apply() = Set.newBuilder[T] + } + + implicit def treeSetCanBuildFrom[T : Ordering] = new CanBuildFrom[T, TreeSet[T]] { + override def apply() = TreeSet.newBuilder[T](implicitly[Ordering[T]]) + } + + implicit def mapCanBuildFrom[K, V] = new CanBuildFrom[(K, V), Map[K, V]] { + override def apply() = Map.newBuilder[K, V] + } + + implicit def treeMapCanBuildFrom[K : Ordering, V] = new CanBuildFrom[(K, V), TreeMap[K, V]] { + override def apply() = TreeMap.newBuilder[K, V] + } + + implicit def javaArrayListCanBuildFrom[T] = new CanBuildFrom[T, java.util.ArrayList[T]] { + override def apply() = new scala.collection.mutable.Builder[T, java.util.ArrayList[T]]() { + val list = new java.util.ArrayList[T]() + override def addOne(elem: T) = { list.add(elem); this } + override def result() = list + override def clear() = list.clear() + } + } + + implicit def javaListCanBuildFrom[T] = + javaArrayListCanBuildFrom[T].upcast[java.util.List[T]] + + implicit def javaHashSetCanBuildFrom[T] = new CanBuildFrom[T, java.util.HashSet[T]] { + override def apply() = new scala.collection.mutable.Builder[T, java.util.HashSet[T]]() { + val set = new java.util.HashSet[T]() + override def addOne(elem: T) = { set.add(elem); this } + override def result() = set + override def clear() = set.clear() + } + } + + implicit def javaSetCanBuildFrom[T] = + javaHashSetCanBuildFrom[T].upcast[java.util.Set[T]] + + implicit def javaHashMapCanBuildFrom[K, V] = new CanBuildFrom[(K, V), java.util.HashMap[K, V]] { + override def apply() = new scala.collection.mutable.Builder[(K, V), java.util.HashMap[K, V]]() { + val map = new java.util.HashMap[K, V]() + override def addOne(elem: (K, V)) = { map.put(elem._1, elem._2); this } + override def result() = map + override def clear() = map.clear() + } + } + + implicit def javaMapCanBuildFrom[K, V] = + javaHashMapCanBuildFrom[K, V].upcast[java.util.Map[K, V]] +} \ No newline at end of file diff --git a/driver/src/main/scala/com/datastax/spark/connector/GettableData.scala b/driver/src/main/scala/com/datastax/spark/connector/GettableData.scala index 935c528c1..d100b9838 100644 --- a/driver/src/main/scala/com/datastax/spark/connector/GettableData.scala +++ b/driver/src/main/scala/com/datastax/spark/connector/GettableData.scala @@ -2,13 +2,13 @@ package com.datastax.spark.connector import java.nio.ByteBuffer -import scala.collection.JavaConversions._ import com.datastax.oss.driver.api.core.cql.Row import com.datastax.oss.driver.api.core.`type`.codec.TypeCodec import com.datastax.oss.driver.api.core.data.{TupleValue => DriverTupleValue, UdtValue => DriverUDTValue} import com.datastax.spark.connector.types.TypeConverter.StringConverter import com.datastax.spark.connector.util.ByteBufferUtil +import scala.collection.JavaConverters._ trait GettableData extends GettableByIndexData { @@ -74,9 +74,9 @@ object GettableData { private[connector] def convert(obj: Any): AnyRef = { obj match { case bb: ByteBuffer => ByteBufferUtil.toArray(bb) - case list: java.util.List[_] => list.view.map(convert).toList - case set: java.util.Set[_] => set.view.map(convert).toSet - case map: java.util.Map[_, _] => map.view.map { case (k, v) => (convert(k), convert(v))}.toMap + case list: java.util.List[_] => list.asScala.view.map(convert).toList + case set: java.util.Set[_] => set.asScala.view.map(convert).toSet + case map: java.util.Map[_, _] => map.asScala.view.map { case (k, v) => (convert(k), convert(v))}.toMap case udtValue: DriverUDTValue => UDTValue.fromJavaDriverUDTValue(udtValue) case tupleValue: DriverTupleValue => TupleValue.fromJavaDriverTupleValue(tupleValue) case other => other.asInstanceOf[AnyRef] diff --git a/driver/src/main/scala/com/datastax/spark/connector/UDTValue.scala b/driver/src/main/scala/com/datastax/spark/connector/UDTValue.scala index b4ac8bba2..5d67b9d73 100644 --- a/driver/src/main/scala/com/datastax/spark/connector/UDTValue.scala +++ b/driver/src/main/scala/com/datastax/spark/connector/UDTValue.scala @@ -4,8 +4,8 @@ import com.datastax.oss.driver.api.core.data.{UdtValue => DriverUDTValue} import com.datastax.spark.connector.types.NullableTypeConverter import com.datastax.spark.connector.util.DriverUtil.toName -import scala.collection.JavaConversions._ import scala.reflect.runtime.universe._ +import scala.collection.JavaConverters._ final case class UDTValue(metaData: CassandraRowMetadata, columnValues: IndexedSeq[AnyRef]) extends ScalaGettableData { @@ -25,7 +25,7 @@ final case class UDTValue(metaData: CassandraRowMetadata, columnValues: IndexedS object UDTValue { def fromJavaDriverUDTValue(value: DriverUDTValue): UDTValue = { - val fields = value.getType.getFieldNames.map(f => toName(f)).toIndexedSeq + val fields = value.getType.getFieldNames.asScala.map(f => toName(f)).toIndexedSeq val values = fields.map(GettableData.get(value, _)) UDTValue(fields, values) } diff --git a/driver/src/main/scala/com/datastax/spark/connector/cql/Schema.scala b/driver/src/main/scala/com/datastax/spark/connector/cql/Schema.scala index 82e36d70a..3cb5e987f 100644 --- a/driver/src/main/scala/com/datastax/spark/connector/cql/Schema.scala +++ b/driver/src/main/scala/com/datastax/spark/connector/cql/Schema.scala @@ -308,7 +308,7 @@ object Schema extends Logging { private def fetchPartitionKey(table: RelationMetadata): Seq[ColumnDef] = { for (column <- table.getPartitionKey.asScala) yield ColumnDef(column, PartitionKeyColumn) - } + }.toSeq private def fetchClusteringColumns(table: RelationMetadata): Seq[ColumnDef] = { for ((column, index) <- table.getClusteringColumns.asScala.toSeq.zipWithIndex) yield { diff --git a/driver/src/main/scala/com/datastax/spark/connector/mapper/GettableDataToMappedTypeConverter.scala b/driver/src/main/scala/com/datastax/spark/connector/mapper/GettableDataToMappedTypeConverter.scala index 33f42c801..bc8aad1c5 100644 --- a/driver/src/main/scala/com/datastax/spark/connector/mapper/GettableDataToMappedTypeConverter.scala +++ b/driver/src/main/scala/com/datastax/spark/connector/mapper/GettableDataToMappedTypeConverter.scala @@ -175,7 +175,7 @@ class GettableDataToMappedTypeConverter[T : TypeTag : ColumnMapper]( for ((s, _) <- columnMap.setters) yield (s, ReflectionUtil.methodParamTypes(targetType, s).head) val setterColumnTypes: Map[String, ColumnType[_]] = - columnMap.setters.mapValues(columnType) + columnMap.setters.map{case (k, v) => (k, columnType(v))}.toMap for (setterName <- setterParamTypes.keys) yield { val ct = setterColumnTypes(setterName) val pt = setterParamTypes(setterName) diff --git a/driver/src/main/scala/com/datastax/spark/connector/types/ColumnType.scala b/driver/src/main/scala/com/datastax/spark/connector/types/ColumnType.scala index 203e9a60e..768b4e1dc 100644 --- a/driver/src/main/scala/com/datastax/spark/connector/types/ColumnType.scala +++ b/driver/src/main/scala/com/datastax/spark/connector/types/ColumnType.scala @@ -11,8 +11,8 @@ import com.datastax.oss.driver.api.core.`type`.{DataType, DataTypes => DriverDat import com.datastax.spark.connector.util._ -import scala.collection.JavaConversions._ import scala.reflect.runtime.universe._ +import scala.collection.JavaConverters._ /** Serializable representation of column data type. */ trait ColumnType[T] extends Serializable { diff --git a/driver/src/main/scala/com/datastax/spark/connector/types/TupleType.scala b/driver/src/main/scala/com/datastax/spark/connector/types/TupleType.scala index 0a4a0e9f1..d9e6445bf 100644 --- a/driver/src/main/scala/com/datastax/spark/connector/types/TupleType.scala +++ b/driver/src/main/scala/com/datastax/spark/connector/types/TupleType.scala @@ -12,8 +12,8 @@ import com.datastax.spark.connector.types.TypeAdapters.ValuesSeqAdapter import com.datastax.spark.connector.{ColumnName, TupleValue} import org.apache.commons.lang3.tuple.{Pair, Triple} -import scala.collection.JavaConversions._ import scala.reflect.runtime.universe._ +import scala.collection.JavaConverters._ case class TupleFieldDef(index: Int, columnType: ColumnType[_]) extends FieldDef { override def columnName = index.toString @@ -107,7 +107,7 @@ object TupleType { extends TypeConverter[DriverTupleValue] { val fieldTypes = dataType.getComponentTypes - val fieldConverters = fieldTypes.map(ColumnType.converterToCassandra) + val fieldConverters = fieldTypes.asScala.map(ColumnType.converterToCassandra) override def targetTypeTag = typeTag[DriverTupleValue] @@ -120,7 +120,7 @@ object TupleType { if (fieldValue == null) { toSave.setToNull(i) } else { - toSave.set(i, fieldValue, CodecRegistry.DEFAULT.codecFor(fieldTypes(i), fieldValue)) + toSave.set(i, fieldValue, CodecRegistry.DEFAULT.codecFor(fieldTypes.asScala(i), fieldValue)) } } toSave @@ -143,7 +143,7 @@ object TupleType { } private def fields(dataType: DriverTupleType): IndexedSeq[TupleFieldDef] = unlazify { - for ((field, index) <- dataType.getComponentTypes.toIndexedSeq.zipWithIndex) yield + for ((field, index) <- dataType.getComponentTypes.asScala.toIndexedSeq.zipWithIndex) yield TupleFieldDef(index, fromDriverType(field)) } diff --git a/driver/src/main/scala/com/datastax/spark/connector/types/TypeConverter.scala b/driver/src/main/scala/com/datastax/spark/connector/types/TypeConverter.scala index 081020d62..75577cc26 100644 --- a/driver/src/main/scala/com/datastax/spark/connector/types/TypeConverter.scala +++ b/driver/src/main/scala/com/datastax/spark/connector/types/TypeConverter.scala @@ -16,9 +16,9 @@ import com.datastax.spark.connector.util.ByteBufferUtil import com.datastax.spark.connector.util.Symbols._ import org.apache.commons.lang3.tuple -import scala.collection.JavaConversions._ import scala.collection.immutable.{TreeMap, TreeSet} import scala.reflect.runtime.universe._ +import scala.collection.JavaConverters._ class TypeConversionException(val message: String, cause: Exception = null) extends Exception(message, cause) @@ -336,7 +336,7 @@ object TypeConverter { c } def targetTypeTag = GregorianCalendarTypeTag - def convertPF = DateConverter.convertPF.andThen(calendar) + def convertPF = DateConverter.convertPF.andThen(calendar _) } private val TimestampTypeTag = implicitly[TypeTag[Timestamp]] @@ -697,9 +697,9 @@ object TypeConverter { def convertPF = { case null => bf.apply().result() - case x: java.util.List[_] => newCollection(x) - case x: java.util.Set[_] => newCollection(x) - case x: java.util.Map[_, _] => newCollection(x) + case x: java.util.List[_] => newCollection(x.asScala) + case x: java.util.Set[_] => newCollection(x.asScala) + case x: java.util.Map[_, _] => newCollection(x.asScala) case x: Iterable[_] => newCollection(x) } } diff --git a/driver/src/main/scala/com/datastax/spark/connector/types/UserDefinedType.scala b/driver/src/main/scala/com/datastax/spark/connector/types/UserDefinedType.scala index ff3fc865d..5efc9890e 100644 --- a/driver/src/main/scala/com/datastax/spark/connector/types/UserDefinedType.scala +++ b/driver/src/main/scala/com/datastax/spark/connector/types/UserDefinedType.scala @@ -10,8 +10,8 @@ import com.datastax.spark.connector.types.ColumnType.fromDriverType import com.datastax.spark.connector.types.TypeAdapters.ValueByNameAdapter import com.datastax.spark.connector.{CassandraRowMetadata, ColumnName, UDTValue} -import scala.collection.JavaConversions._ import scala.reflect.runtime.universe._ +import scala.collection.JavaConverters._ /** A Cassandra user defined type field metadata. It consists of a name and an associated column type. * The word `column` instead of `field` is used in member names because we want to treat UDT field @@ -81,8 +81,8 @@ object UserDefinedType { class DriverUDTValueConverter(dataType: DriverUserDefinedType) extends TypeConverter[DriverUDTValue] { - val fieldNames = dataType.getFieldNames.toIndexedSeq - val fieldTypes = dataType.getFieldTypes.toIndexedSeq + val fieldNames = dataType.getFieldNames.asScala.toIndexedSeq + val fieldTypes = dataType.getFieldTypes.asScala.toIndexedSeq val fieldConverters = fieldTypes.map(ColumnType.converterToCassandra) override def targetTypeTag = implicitly[TypeTag[DriverUDTValue]] @@ -113,7 +113,7 @@ object UserDefinedType { } private def fields(dataType: DriverUserDefinedType): IndexedSeq[UDTFieldDef] = unlazify { - for ((fieldName, fieldType) <- dataType.getFieldNames.zip(dataType.getFieldTypes).toIndexedSeq) yield + for ((fieldName, fieldType) <- dataType.getFieldNames.asScala.zip(dataType.getFieldTypes.asScala).toIndexedSeq) yield UDTFieldDef(fieldName.asInternal(), fromDriverType(fieldType)) } diff --git a/driver/src/main/scala/com/datastax/spark/connector/util/NameTools.scala b/driver/src/main/scala/com/datastax/spark/connector/util/NameTools.scala index 2a2c022c8..0826d167d 100644 --- a/driver/src/main/scala/com/datastax/spark/connector/util/NameTools.scala +++ b/driver/src/main/scala/com/datastax/spark/connector/util/NameTools.scala @@ -6,7 +6,7 @@ import com.datastax.oss.driver.api.core.metadata.Metadata import com.datastax.spark.connector.util.DriverUtil.toName import org.apache.commons.lang3.StringUtils -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ object NameTools { @@ -44,7 +44,7 @@ object NameTools { def getSuggestions(clusterMetadata: Metadata, keyspace: String): Option[Suggestions] = { val keyspaceScores = clusterMetadata .getKeyspaces - .values() + .values().asScala .toSeq .map(ks => (toName(ks.getName), StringUtils.getJaroWinklerDistance(toName(ks.getName).toLowerCase(Locale.ROOT), keyspace.toLowerCase(Locale.ROOT)))) @@ -74,12 +74,12 @@ object NameTools { val keyspaceScores = clusterMetadata .getKeyspaces - .values() + .values().asScala .toSeq .map(ks => (ks, StringUtils.getJaroWinklerDistance(toName(ks.getName).toLowerCase(Locale.ROOT), keyspace.toLowerCase(Locale.ROOT)))) - val ktScores = for ((ks, ksScore) <- keyspaceScores; (_, t) <- (ks.getTables ++ ks.getViews)) yield { + val ktScores = for ((ks, ksScore) <- keyspaceScores; (_, t) <- (ks.getTables.asScala ++ ks.getViews.asScala)) yield { val tScore = StringUtils.getJaroWinklerDistance(toName(t.getName).toLowerCase(Locale.ROOT), table.toLowerCase(Locale.ROOT)) (toName(ks.getName), toName(t.getName), ksScore, tScore) } diff --git a/driver/src/test/scala/com/datastax/spark/connector/types/TypeConverterTest.scala b/driver/src/test/scala/com/datastax/spark/connector/types/TypeConverterTest.scala index 03f87b8b6..2a5515f1a 100644 --- a/driver/src/test/scala/com/datastax/spark/connector/types/TypeConverterTest.scala +++ b/driver/src/test/scala/com/datastax/spark/connector/types/TypeConverterTest.scala @@ -271,7 +271,7 @@ class TypeConverterTest { buf.put(array) buf.rewind() assertSame(array, c.convert(array)) - assertEquals(array.deep, c.convert(buf).deep) + assertEquals(array.toList, c.convert(buf).toList) } @Test diff --git a/project/Publishing.scala b/project/Publishing.scala index 8fa02fe25..0f27104c6 100644 --- a/project/Publishing.scala +++ b/project/Publishing.scala @@ -10,7 +10,7 @@ import scala.sys.process._ object Publishing extends sbt.librarymanagement.DependencyBuilders { val Version: String = { - sys.props.get("publish.version").orElse(sys.env.get("PUBLISH_VERSION")).getOrElse("git describe --tags" !!).stripLineEnd.stripPrefix("v") + sys.props.get("publish.version").orElse(sys.env.get("PUBLISH_VERSION")).getOrElse("git describe --tags").stripLineEnd.stripPrefix("v") } val altReleaseDeploymentRepository = sys.props.get("publish.repository.name") diff --git a/test-support/src/main/scala/com/datastax/spark/connector/ccm/CcmBridge.scala b/test-support/src/main/scala/com/datastax/spark/connector/ccm/CcmBridge.scala index 21b967451..7312c6600 100644 --- a/test-support/src/main/scala/com/datastax/spark/connector/ccm/CcmBridge.scala +++ b/test-support/src/main/scala/com/datastax/spark/connector/ccm/CcmBridge.scala @@ -104,7 +104,7 @@ object CcmBridge { logger.error( "Non-zero exit code ({}) returned from executing ccm command: {}", retValue, cli) } - outStream.lines + outStream.lines.toSeq } catch { case _: IOException if watchDog.killedProcess() => throw new RuntimeException(s"The command $cli was killed after 10 minutes") From 40f1cff2287d4fe3cb4050de1ccf283bb7f91604 Mon Sep 17 00:00:00 2001 From: SamTheisens <1911436+SamTheisens@users.noreply.github.com> Date: Tue, 11 Jul 2023 15:01:28 +0800 Subject: [PATCH 02/11] [SPARKC-686] Replace deprecated mapvalues with canonical way to map values branch: feature/SPARKC-686-scala-213-support --- build.sbt | 2 +- .../datastax/spark/connector/cluster/ClusterHolder.scala | 2 +- .../cql/CassandraAuthenticatedConnectorSpec.scala | 2 +- .../datasource/CassandraCatalogNamespaceSpec.scala | 2 +- .../connector/datasource/CassandraCatalogTableSpec.scala | 2 +- .../spark/connector/rdd/DseGraphUnionRDDSpec.scala | 2 +- .../spark/connector/rdd/typeTests/AbstractTypeTest.scala | 2 +- .../spark/connector/rdd/typeTests/BlobTypeTest.scala | 8 ++++---- .../spark/connector/rdd/typeTests/InetTypeTest.scala | 2 +- .../com/datastax/bdp/spark/ContinuousPagingScanner.scala | 4 ++-- .../spark/connector/cql/CassandraConnectionFactory.scala | 2 +- .../datastax/spark/connector/cql/CassandraConnector.scala | 2 +- .../com/datastax/spark/connector/cql/QueryUtils.scala | 2 +- .../spark/connector/datasource/CassandraCatalog.scala | 2 +- .../spark/connector/datasource/CassandraScanBuilder.scala | 2 +- .../spark/connector/datasource/CassandraSourceUtil.scala | 2 +- .../spark/connector/datasource/CassandraTable.scala | 2 +- .../connector/datasource/CassandraWriteBuilder.scala | 2 +- .../datastax/spark/connector/datasource/ScanHelper.scala | 2 +- .../spark/connector/rdd/AbstractCassandraJoin.scala | 2 +- .../spark/connector/rdd/CassandraCoGroupedRDD.scala | 2 +- .../spark/connector/rdd/CassandraMergeJoinRDD.scala | 2 +- .../spark/connector/rdd/CassandraTableScanRDD.scala | 2 +- .../datastax/spark/connector/rdd/DseGraphUnionedRDD.scala | 2 +- .../rdd/partitioner/CassandraPartitionGenerator.scala | 2 +- .../connector/rdd/partitioner/DataSizeEstimates.scala | 2 +- .../spark/connector/rdd/partitioner/NodeAddresses.scala | 2 +- .../connector/rdd/partitioner/ReplicaPartitioner.scala | 2 +- .../spark/connector/rdd/partitioner/TokenGenerator.scala | 2 +- .../com/datastax/spark/connector/util/JavaApiHelper.scala | 2 +- .../datastax/spark/connector/writer/AsyncExecutor.scala | 2 +- .../spark/connector/writer/ObjectSizeEstimator.scala | 2 +- .../datastax/spark/connector/writer/ReplicaLocator.scala | 2 +- .../scala/org/apache/spark/metrics/CassandraSink.scala | 2 +- .../spark/sql/cassandra/CassandraSourceRelation.scala | 2 +- .../org/apache/spark/sql/cassandra/DefaultSource.scala | 2 +- .../apache/spark/sql/cassandra/SolrPredicateRules.scala | 2 +- .../datastax/bdp/spark/DseByosAuthConfFactorySpec.scala | 2 +- .../src/test/scala/com/datastax/driver/core/RowMock.scala | 2 +- .../scala/com/datastax/spark/connector/CassandraRow.scala | 2 +- .../scala/com/datastax/spark/connector/GettableData.scala | 2 +- .../scala/com/datastax/spark/connector/UDTValue.scala | 2 +- .../connector/cql/LocalNodeFirstLoadBalancingPolicy.scala | 2 +- .../scala/com/datastax/spark/connector/cql/Schema.scala | 2 +- .../com/datastax/spark/connector/types/ColumnType.scala | 2 +- .../com/datastax/spark/connector/types/TupleType.scala | 2 +- .../datastax/spark/connector/types/TypeConverter.scala | 2 +- .../datastax/spark/connector/types/UserDefinedType.scala | 2 +- .../com/datastax/spark/connector/util/NameTools.scala | 2 +- .../cql/LocalNodeFirstLoadBalancingPolicySpec.scala | 2 +- project/Publishing.scala | 2 +- 51 files changed, 55 insertions(+), 55 deletions(-) diff --git a/build.sbt b/build.sbt index 9d79e02ae..2aa25827f 100644 --- a/build.sbt +++ b/build.sbt @@ -80,7 +80,7 @@ lazy val commonSettings = Seq( "org.scala-lang.modules" %% "scala-collection-compat" % "2.11.0", "org.scala-lang.modules" %% "scala-parallel-collections" % "1.0.4" ) - case _ => Seq() + case _ => Seq("org.scala-lang.modules" %% "scala-collection-compat" % "2.11.0") } } ) diff --git a/connector/src/it/scala/com/datastax/spark/connector/cluster/ClusterHolder.scala b/connector/src/it/scala/com/datastax/spark/connector/cluster/ClusterHolder.scala index 2e48a740c..f4b272bdd 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/cluster/ClusterHolder.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/cluster/ClusterHolder.scala @@ -4,7 +4,7 @@ import com.datastax.spark.connector.ccm.CcmBridge import com.datastax.spark.connector.util.{Logging, SerialShutdownHooks} import org.apache.commons.lang3.ClassUtils -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable /** Source of [[Cluster]]s for all tests executed within a single test group (single process/JVM). diff --git a/connector/src/it/scala/com/datastax/spark/connector/cql/CassandraAuthenticatedConnectorSpec.scala b/connector/src/it/scala/com/datastax/spark/connector/cql/CassandraAuthenticatedConnectorSpec.scala index 1c382b4da..d37a79af4 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/cql/CassandraAuthenticatedConnectorSpec.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/cql/CassandraAuthenticatedConnectorSpec.scala @@ -7,7 +7,7 @@ import com.datastax.oss.driver.api.core.auth.AuthenticationException import com.datastax.spark.connector.cluster.AuthCluster import com.datastax.spark.connector.{SparkCassandraITFlatSpecBase, _} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class CassandraAuthenticatedConnectorSpec extends SparkCassandraITFlatSpecBase with AuthCluster { diff --git a/connector/src/it/scala/com/datastax/spark/connector/datasource/CassandraCatalogNamespaceSpec.scala b/connector/src/it/scala/com/datastax/spark/connector/datasource/CassandraCatalogNamespaceSpec.scala index 8d29b5495..978d94bdf 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/datasource/CassandraCatalogNamespaceSpec.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/datasource/CassandraCatalogNamespaceSpec.scala @@ -2,7 +2,7 @@ package com.datastax.spark.connector.datasource import org.scalatest.concurrent.Eventually._ import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class CassandraCatalogNamespaceSpec extends CassandraCatalogSpecBase { diff --git a/connector/src/it/scala/com/datastax/spark/connector/datasource/CassandraCatalogTableSpec.scala b/connector/src/it/scala/com/datastax/spark/connector/datasource/CassandraCatalogTableSpec.scala index e1a6f32c8..8bcf5e23b 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/datasource/CassandraCatalogTableSpec.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/datasource/CassandraCatalogTableSpec.scala @@ -4,7 +4,7 @@ import com.datastax.oss.driver.api.core.CqlIdentifier.fromInternal import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, TableAlreadyExistsException} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable.{Map => MutableMap} class CassandraCatalogTableSpec extends CassandraCatalogSpecBase { diff --git a/connector/src/it/scala/com/datastax/spark/connector/rdd/DseGraphUnionRDDSpec.scala b/connector/src/it/scala/com/datastax/spark/connector/rdd/DseGraphUnionRDDSpec.scala index 92106f7a9..ea16139b0 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/rdd/DseGraphUnionRDDSpec.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/rdd/DseGraphUnionRDDSpec.scala @@ -8,7 +8,7 @@ package com.datastax.spark.connector.rdd import java.lang.{Integer => JInteger, String => JString} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.concurrent.Future import scala.language.existentials import scala.util.Random diff --git a/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/AbstractTypeTest.scala b/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/AbstractTypeTest.scala index 8ebd2ae0e..c8b5f5b6b 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/AbstractTypeTest.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/AbstractTypeTest.scala @@ -8,7 +8,7 @@ import com.datastax.spark.connector.rdd.reader.RowReaderFactory import com.datastax.spark.connector.types.TypeConverter import com.datastax.spark.connector.writer.RowWriterFactory -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.concurrent.Future import scala.reflect._ diff --git a/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/BlobTypeTest.scala b/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/BlobTypeTest.scala index 3c71ccefa..b8da4617d 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/BlobTypeTest.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/BlobTypeTest.scala @@ -6,7 +6,7 @@ import com.datastax.oss.driver.api.core.cql.Row import com.datastax.oss.protocol.internal.util.Bytes import com.datastax.spark.connector.cluster.DefaultCluster -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class BlobTypeTest extends AbstractTypeTest[Array[Byte], ByteBuffer] with DefaultCluster { @@ -80,7 +80,7 @@ class BlobTypeTest extends AbstractTypeTest[Array[Byte], ByteBuffer] with Defaul Bytes.toHexString(pkey), set1.map(Bytes.toHexString(_)), list1.map(Bytes.toHexString(_)), - map1.mapValues(Bytes.toHexString(_)), + map1.map{ case (k,v) => (k, Bytes.toHexString(v))}, map2.map{ case (k,v) => (Bytes.toHexString(k), v)}) } @@ -91,8 +91,8 @@ class BlobTypeTest extends AbstractTypeTest[Array[Byte], ByteBuffer] with Defaul x, expectedCollections._1.map(Bytes.toHexString(_)), expectedCollections._2.map(Bytes.toHexString(_)), - expectedCollections._3.mapValues(Bytes.toHexString(_)), - expectedCollections._4.map { case (k, v) => (Bytes.toHexString(k), v) })) + expectedCollections._3.map{ case (k, v) => (k, Bytes.toHexString(v)) }, + expectedCollections._4.map{ case (k, v) => (Bytes.toHexString(k), v) })) expected } } diff --git a/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/InetTypeTest.scala b/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/InetTypeTest.scala index bad4f0e2a..c151bdf6a 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/InetTypeTest.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/InetTypeTest.scala @@ -6,7 +6,7 @@ import com.datastax.oss.driver.api.core.cql.Row import com.datastax.spark.connector.cluster.DefaultCluster import org.apache.spark.sql.SaveMode -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ case class InetStringRow(pkey: String, ckey1: String, ckey2: String, data1: String) class InetTypeTest extends AbstractTypeTest[InetAddress, InetAddress] with DefaultCluster { diff --git a/connector/src/main/scala/com/datastax/bdp/spark/ContinuousPagingScanner.scala b/connector/src/main/scala/com/datastax/bdp/spark/ContinuousPagingScanner.scala index 4eea5cd35..86a4fed5d 100644 --- a/connector/src/main/scala/com/datastax/bdp/spark/ContinuousPagingScanner.scala +++ b/connector/src/main/scala/com/datastax/bdp/spark/ContinuousPagingScanner.scala @@ -19,7 +19,7 @@ import com.datastax.spark.connector.rdd.ReadConf import com.datastax.spark.connector.util.DriverUtil.toName import com.datastax.spark.connector.util._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ case class ContinuousPagingScanner( readConf: ReadConf, @@ -111,7 +111,7 @@ case class ContinuousPagingScanner( } private def getMetaData(result: ContinuousResultSet): CassandraRowMetadata = { - import scala.collection.JavaConverters._ + import scala.jdk.CollectionConverters._ val columnDefs = result.getColumnDefinitions.asScala val rsColumnNames = columnDefs.map(c => toName(c.getName)) val codecs = columnDefs diff --git a/connector/src/main/scala/com/datastax/spark/connector/cql/CassandraConnectionFactory.scala b/connector/src/main/scala/com/datastax/spark/connector/cql/CassandraConnectionFactory.scala index 2f684037a..b325f3e48 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/cql/CassandraConnectionFactory.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/cql/CassandraConnectionFactory.scala @@ -18,7 +18,7 @@ import com.datastax.spark.connector.util.{ConfigParameter, DeprecatedConfigParam import org.apache.spark.{SparkConf, SparkEnv, SparkFiles} import org.slf4j.LoggerFactory -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** Creates both native and Thrift connections to Cassandra. * The connector provides a DefaultConnectionFactory. diff --git a/connector/src/main/scala/com/datastax/spark/connector/cql/CassandraConnector.scala b/connector/src/main/scala/com/datastax/spark/connector/cql/CassandraConnector.scala index 33b583cfb..85e6b84cf 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/cql/CassandraConnector.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/cql/CassandraConnector.scala @@ -12,7 +12,7 @@ import org.apache.spark.{SparkConf, SparkContext} import java.io.IOException import java.net.{InetAddress, InetSocketAddress} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.language.reflectiveCalls /** Provides and manages connections to Cassandra. diff --git a/connector/src/main/scala/com/datastax/spark/connector/cql/QueryUtils.scala b/connector/src/main/scala/com/datastax/spark/connector/cql/QueryUtils.scala index 48dfec80c..49e1955aa 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/cql/QueryUtils.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/cql/QueryUtils.scala @@ -5,7 +5,7 @@ import java.nio.ByteBuffer import com.datastax.oss.driver.api.core.cql.BoundStatement import com.datastax.spark.connector.writer.NullKeyColumnException -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ object QueryUtils { /** diff --git a/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraCatalog.scala b/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraCatalog.scala index 8909fb389..467d73a46 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraCatalog.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraCatalog.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.mutable import scala.util.{Failure, Success, Try} diff --git a/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraScanBuilder.scala b/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraScanBuilder.scala index b1addefa1..f22bb75df 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraScanBuilder.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraScanBuilder.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.sql.{SparkSession, sources} import org.apache.spark.unsafe.types.UTF8String -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ case class CassandraScanBuilder( session: SparkSession, diff --git a/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraSourceUtil.scala b/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraSourceUtil.scala index 3ad335972..19764a0a6 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraSourceUtil.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraSourceUtil.scala @@ -12,7 +12,7 @@ import org.apache.spark.SparkConf import org.apache.spark.sql.types.{DataType => CatalystType} import org.apache.spark.sql.types.{BooleanType => SparkSqlBooleanType, DataType => SparkSqlDataType, DateType => SparkSqlDateType, DecimalType => SparkSqlDecimalType, DoubleType => SparkSqlDoubleType, FloatType => SparkSqlFloatType, MapType => SparkSqlMapType, TimestampType => SparkSqlTimestampType, UserDefinedType => SparkSqlUserDefinedType, _} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.language.postfixOps import scala.util.Try diff --git a/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraTable.scala b/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraTable.scala index 1cd77fd38..05a47a449 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraTable.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraTable.scala @@ -12,7 +12,7 @@ import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ case class CassandraTable(session: SparkSession, catalogConf: CaseInsensitiveStringMap, connector: CassandraConnector, catalogName: String, metadata: RelationMetadata, optionalSchema: Option[StructType] = None) //Used for adding metadata references diff --git a/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraWriteBuilder.scala b/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraWriteBuilder.scala index 46b2bb936..9e8a92686 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraWriteBuilder.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/datasource/CassandraWriteBuilder.scala @@ -13,7 +13,7 @@ import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactor import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.util.Try case class CassandraWriteBuilder( diff --git a/connector/src/main/scala/com/datastax/spark/connector/datasource/ScanHelper.scala b/connector/src/main/scala/com/datastax/spark/connector/datasource/ScanHelper.scala index 3bac59e64..36d0333b5 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/datasource/ScanHelper.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/datasource/ScanHelper.scala @@ -16,7 +16,7 @@ import com.datastax.spark.connector.util.{CqlWhereParser, Logging} import com.datastax.spark.connector.{ColumnName, ColumnRef, TTL, WriteTime} import org.apache.spark.sql.cassandra.DsePredicateRules.StorageAttachedIndex -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ object ScanHelper extends Logging { diff --git a/connector/src/main/scala/com/datastax/spark/connector/rdd/AbstractCassandraJoin.scala b/connector/src/main/scala/com/datastax/spark/connector/rdd/AbstractCassandraJoin.scala index 8a2a054ed..051a763b2 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/rdd/AbstractCassandraJoin.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/rdd/AbstractCassandraJoin.scala @@ -14,7 +14,7 @@ import org.apache.spark.metrics.InputMetricsUpdater import org.apache.spark.rdd.RDD import org.apache.spark.{Partition, TaskContext} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * This trait contains shared methods from [[com.datastax.spark.connector.rdd.CassandraJoinRDD]] and * [[com.datastax.spark.connector.rdd.CassandraLeftJoinRDD]] to avoid code duplication. diff --git a/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraCoGroupedRDD.scala b/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraCoGroupedRDD.scala index 6dd9ada5b..50eb6cd4b 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraCoGroupedRDD.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraCoGroupedRDD.scala @@ -31,7 +31,7 @@ import com.datastax.spark.connector.util.{CountingIterator, MultiMergeJoinIterat import scala.concurrent.Await import scala.concurrent.duration.Duration -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * A RDD which pulls from provided separate CassandraTableScanRDDs which share partition keys type and diff --git a/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraMergeJoinRDD.scala b/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraMergeJoinRDD.scala index fe0bdb10c..0acabb65f 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraMergeJoinRDD.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraMergeJoinRDD.scala @@ -29,7 +29,7 @@ import com.datastax.spark.connector.util.{CountingIterator, MergeJoinIterator, N import scala.concurrent.Await import scala.concurrent.duration.Duration -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * A RDD which pulls from two separate CassandraTableScanRDDs which share partition keys and diff --git a/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraTableScanRDD.scala b/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraTableScanRDD.scala index 57f97f133..8f0e936e0 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraTableScanRDD.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraTableScanRDD.scala @@ -17,7 +17,7 @@ import org.apache.spark.{Partition, Partitioner, SparkContext, TaskContext} import java.io.IOException import scala.language.existentials import scala.reflect.ClassTag -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** RDD representing a Table Scan of A Cassandra table. diff --git a/connector/src/main/scala/com/datastax/spark/connector/rdd/DseGraphUnionedRDD.scala b/connector/src/main/scala/com/datastax/spark/connector/rdd/DseGraphUnionedRDD.scala index bb1b3fb57..7da4bde76 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/rdd/DseGraphUnionedRDD.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/rdd/DseGraphUnionedRDD.scala @@ -19,7 +19,7 @@ import com.datastax.spark.connector.rdd.partitioner.dht.{Token, TokenFactory} import com.datastax.spark.connector.rdd.partitioner.{BucketingRangeIndex, CassandraPartition, TokenGenerator, TokenRangeWithPartitionIndex} import com.datastax.spark.connector.util.{Logging, schemaFromCassandra} import com.datastax.spark.connector.writer.RowWriter -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * A Java Friendly api for DseGraphUnionedRDD to make it easier to call diff --git a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/CassandraPartitionGenerator.scala b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/CassandraPartitionGenerator.scala index eaf9d853f..5321c7831 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/CassandraPartitionGenerator.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/CassandraPartitionGenerator.scala @@ -14,7 +14,7 @@ import com.datastax.spark.connector.cql.{CassandraConnector, TableDef} import com.datastax.spark.connector.rdd.partitioner.dht.{Token, TokenFactory} import com.datastax.spark.connector.writer.RowWriterFactory import org.apache.spark.sql.connector.read.InputPartition -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** Creates CassandraPartitions for given Cassandra table */ diff --git a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/DataSizeEstimates.scala b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/DataSizeEstimates.scala index a03df8d66..34a044bff 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/DataSizeEstimates.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/DataSizeEstimates.scala @@ -5,7 +5,7 @@ import com.datastax.oss.driver.api.core.cql.SimpleStatementBuilder import com.datastax.oss.driver.api.core.servererrors.InvalidQueryException import com.datastax.spark.connector.cql.CassandraConnector import com.datastax.spark.connector.rdd.partitioner.dht.{Token, TokenFactory} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** Estimates amount of data in the Cassandra table. diff --git a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/NodeAddresses.scala b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/NodeAddresses.scala index 2e9e838a2..53578e31f 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/NodeAddresses.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/NodeAddresses.scala @@ -4,7 +4,7 @@ import java.net.InetAddress import com.datastax.spark.connector.cql.CassandraConnector import com.datastax.spark.connector.util.DriverUtil.{toName, toOption} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** Looks up listen address of a cluster node given its Native Transport address. * Uses system.peers table as the source of information. diff --git a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/ReplicaPartitioner.scala b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/ReplicaPartitioner.scala index be6cfdf94..cb6ab1980 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/ReplicaPartitioner.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/ReplicaPartitioner.scala @@ -10,7 +10,7 @@ import com.datastax.spark.connector.writer.RowWriterFactory import org.apache.spark.{Partition, Partitioner} import scala.reflect.ClassTag -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ case class ReplicaPartition(index: Int, endpoints: Array[String]) extends EndpointPartition diff --git a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/TokenGenerator.scala b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/TokenGenerator.scala index c70d2d29b..4a7add1ed 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/TokenGenerator.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/TokenGenerator.scala @@ -10,7 +10,7 @@ import com.datastax.spark.connector.util.Logging import com.datastax.spark.connector.util.PatitionKeyTools._ import com.datastax.spark.connector.writer.{BoundStatementBuilder, NullKeyColumnException, RowWriter} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * A utility class for determining the token of a given key. Uses a bound statement to determine diff --git a/connector/src/main/scala/com/datastax/spark/connector/util/JavaApiHelper.scala b/connector/src/main/scala/com/datastax/spark/connector/util/JavaApiHelper.scala index 41a1c924b..434780240 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/util/JavaApiHelper.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/util/JavaApiHelper.scala @@ -14,7 +14,7 @@ import com.datastax.spark.connector.CassandraRow import com.datastax.spark.connector.mapper.{ColumnMapper, JavaBeanColumnMapper} import com.datastax.spark.connector.rdd.reader.RowReaderFactory import com.datastax.spark.connector.writer.RowWriterFactory -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** A helper class to make it possible to access components written in Scala from Java code. * INTERNAL API diff --git a/connector/src/main/scala/com/datastax/spark/connector/writer/AsyncExecutor.scala b/connector/src/main/scala/com/datastax/spark/connector/writer/AsyncExecutor.scala index 982a5b5a9..eef2cd128 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/writer/AsyncExecutor.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/writer/AsyncExecutor.scala @@ -5,7 +5,7 @@ import java.util.function.BiConsumer import com.datastax.spark.connector.util.Logging -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.collection.concurrent.TrieMap import scala.util.Try import AsyncExecutor.Handler diff --git a/connector/src/main/scala/com/datastax/spark/connector/writer/ObjectSizeEstimator.scala b/connector/src/main/scala/com/datastax/spark/connector/writer/ObjectSizeEstimator.scala index 8d1d91140..0140bb990 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/writer/ObjectSizeEstimator.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/writer/ObjectSizeEstimator.scala @@ -5,7 +5,7 @@ import java.nio.ByteBuffer import com.datastax.spark.connector.util.ByteBufferUtil -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** Estimates amount of memory required to serialize Java/Scala objects */ diff --git a/connector/src/main/scala/com/datastax/spark/connector/writer/ReplicaLocator.scala b/connector/src/main/scala/com/datastax/spark/connector/writer/ReplicaLocator.scala index 5b3411f5f..5e61894fe 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/writer/ReplicaLocator.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/writer/ReplicaLocator.scala @@ -11,7 +11,7 @@ import com.datastax.spark.connector.util.{DriverUtil, Logging, tableFromCassandr import com.datastax.spark.connector.util.PatitionKeyTools._ import scala.collection._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * A utility class for determining the Replica Set (Ip Addresses) of a particular Cassandra Row. Used diff --git a/connector/src/main/scala/org/apache/spark/metrics/CassandraSink.scala b/connector/src/main/scala/org/apache/spark/metrics/CassandraSink.scala index 1ad24bbbf..85a61c735 100644 --- a/connector/src/main/scala/org/apache/spark/metrics/CassandraSink.scala +++ b/connector/src/main/scala/org/apache/spark/metrics/CassandraSink.scala @@ -16,7 +16,7 @@ import org.apache.spark.{SecurityManager, SparkConf, SparkEnv} import com.datastax.oss.driver.api.core.cql.{AsyncResultSet, ResultSet} import com.datastax.spark.connector.cql.CassandraConnector import com.datastax.spark.connector.util.Logging -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class CassandraSink(val properties: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink with Runnable with Logging { diff --git a/connector/src/main/scala/org/apache/spark/sql/cassandra/CassandraSourceRelation.scala b/connector/src/main/scala/org/apache/spark/sql/cassandra/CassandraSourceRelation.scala index 2459a5b94..a895ce991 100644 --- a/connector/src/main/scala/org/apache/spark/sql/cassandra/CassandraSourceRelation.scala +++ b/connector/src/main/scala/org/apache/spark/sql/cassandra/CassandraSourceRelation.scala @@ -11,7 +11,7 @@ import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, Data import org.apache.spark.sql.sources._ import org.apache.spark.sql.util.CaseInsensitiveStringMap -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ sealed trait DirectJoinSetting case object AlwaysOn extends DirectJoinSetting diff --git a/connector/src/main/scala/org/apache/spark/sql/cassandra/DefaultSource.scala b/connector/src/main/scala/org/apache/spark/sql/cassandra/DefaultSource.scala index 67226b60f..b5b08b9dd 100644 --- a/connector/src/main/scala/org/apache/spark/sql/cassandra/DefaultSource.scala +++ b/connector/src/main/scala/org/apache/spark/sql/cassandra/DefaultSource.scala @@ -15,7 +15,7 @@ import org.apache.spark.sql.execution.streaming.Sink import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.util.CaseInsensitiveStringMap -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** * A Pointer to the DatasourceV2 Implementation of The Cassandra Source diff --git a/connector/src/main/scala/org/apache/spark/sql/cassandra/SolrPredicateRules.scala b/connector/src/main/scala/org/apache/spark/sql/cassandra/SolrPredicateRules.scala index c0bdac601..78440f8b2 100644 --- a/connector/src/main/scala/org/apache/spark/sql/cassandra/SolrPredicateRules.scala +++ b/connector/src/main/scala/org/apache/spark/sql/cassandra/SolrPredicateRules.scala @@ -9,7 +9,7 @@ import java.time.format.DateTimeFormatter import java.util.concurrent.TimeUnit import java.util.regex.Pattern -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.util.{Failure, Success, Try} import org.apache.commons.lang3.StringEscapeUtils diff --git a/connector/src/test/scala/com/datastax/bdp/spark/DseByosAuthConfFactorySpec.scala b/connector/src/test/scala/com/datastax/bdp/spark/DseByosAuthConfFactorySpec.scala index db89b265c..45cdb17cd 100644 --- a/connector/src/test/scala/com/datastax/bdp/spark/DseByosAuthConfFactorySpec.scala +++ b/connector/src/test/scala/com/datastax/bdp/spark/DseByosAuthConfFactorySpec.scala @@ -7,7 +7,7 @@ package com.datastax.bdp.spark /* TODO: -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.reflect.runtime.universe._ import org.apache.spark.SparkConf diff --git a/connector/src/test/scala/com/datastax/driver/core/RowMock.scala b/connector/src/test/scala/com/datastax/driver/core/RowMock.scala index bb115a0a7..1170cd1ca 100644 --- a/connector/src/test/scala/com/datastax/driver/core/RowMock.scala +++ b/connector/src/test/scala/com/datastax/driver/core/RowMock.scala @@ -12,7 +12,7 @@ import com.datastax.oss.driver.internal.core.cql.{DefaultColumnDefinition, Defau import com.datastax.oss.protocol.internal.ProtocolConstants import com.datastax.oss.protocol.internal.response.result.{ColumnSpec, RawType} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class RowMock(columnSizes: Option[Int]*) extends Row { diff --git a/driver/src/main/scala/com/datastax/spark/connector/CassandraRow.scala b/driver/src/main/scala/com/datastax/spark/connector/CassandraRow.scala index 1a7079af5..bb1d95abb 100644 --- a/driver/src/main/scala/com/datastax/spark/connector/CassandraRow.scala +++ b/driver/src/main/scala/com/datastax/spark/connector/CassandraRow.scala @@ -146,7 +146,7 @@ object CassandraRowMetadata { } private def fromColumnDefs(columnNames: IndexedSeq[String], columnDefs: ColumnDefinitions, registry: CodecRegistry): CassandraRowMetadata = { - import scala.collection.JavaConverters._ + import scala.jdk.CollectionConverters._ val scalaColumnDefs = columnDefs.asScala.toList val rsColumnNames = scalaColumnDefs.map(c => toName(c.getName)) val codecs = scalaColumnDefs.map(col => registry.codecFor(col.getType)) diff --git a/driver/src/main/scala/com/datastax/spark/connector/GettableData.scala b/driver/src/main/scala/com/datastax/spark/connector/GettableData.scala index d100b9838..03e9e7198 100644 --- a/driver/src/main/scala/com/datastax/spark/connector/GettableData.scala +++ b/driver/src/main/scala/com/datastax/spark/connector/GettableData.scala @@ -8,7 +8,7 @@ import com.datastax.oss.driver.api.core.`type`.codec.TypeCodec import com.datastax.oss.driver.api.core.data.{TupleValue => DriverTupleValue, UdtValue => DriverUDTValue} import com.datastax.spark.connector.types.TypeConverter.StringConverter import com.datastax.spark.connector.util.ByteBufferUtil -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ trait GettableData extends GettableByIndexData { diff --git a/driver/src/main/scala/com/datastax/spark/connector/UDTValue.scala b/driver/src/main/scala/com/datastax/spark/connector/UDTValue.scala index 5d67b9d73..b2a19f97e 100644 --- a/driver/src/main/scala/com/datastax/spark/connector/UDTValue.scala +++ b/driver/src/main/scala/com/datastax/spark/connector/UDTValue.scala @@ -5,7 +5,7 @@ import com.datastax.spark.connector.types.NullableTypeConverter import com.datastax.spark.connector.util.DriverUtil.toName import scala.reflect.runtime.universe._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ final case class UDTValue(metaData: CassandraRowMetadata, columnValues: IndexedSeq[AnyRef]) extends ScalaGettableData { diff --git a/driver/src/main/scala/com/datastax/spark/connector/cql/LocalNodeFirstLoadBalancingPolicy.scala b/driver/src/main/scala/com/datastax/spark/connector/cql/LocalNodeFirstLoadBalancingPolicy.scala index dc25da0e6..5d60286b1 100644 --- a/driver/src/main/scala/com/datastax/spark/connector/cql/LocalNodeFirstLoadBalancingPolicy.scala +++ b/driver/src/main/scala/com/datastax/spark/connector/cql/LocalNodeFirstLoadBalancingPolicy.scala @@ -19,7 +19,7 @@ import com.datastax.oss.driver.internal.core.util.collection.SimpleQueryPlan import com.datastax.spark.connector.cql.LocalNodeFirstLoadBalancingPolicy.{LoadBalancingShuffleNodes, _} import com.datastax.spark.connector.util.DriverUtil.{toAddress, toOption} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.util.Random /** Selects local node first and then nodes in local DC in random order. Never selects nodes from other DCs. diff --git a/driver/src/main/scala/com/datastax/spark/connector/cql/Schema.scala b/driver/src/main/scala/com/datastax/spark/connector/cql/Schema.scala index 3cb5e987f..48567ed23 100644 --- a/driver/src/main/scala/com/datastax/spark/connector/cql/Schema.scala +++ b/driver/src/main/scala/com/datastax/spark/connector/cql/Schema.scala @@ -13,7 +13,7 @@ import com.datastax.spark.connector.util.DriverUtil.{toName, toOption} import com.datastax.spark.connector.util.Quote._ import com.datastax.spark.connector.util.{Logging, NameTools} -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ import scala.language.existentials import scala.util.{Properties, Try} diff --git a/driver/src/main/scala/com/datastax/spark/connector/types/ColumnType.scala b/driver/src/main/scala/com/datastax/spark/connector/types/ColumnType.scala index 768b4e1dc..99a3f9fb3 100644 --- a/driver/src/main/scala/com/datastax/spark/connector/types/ColumnType.scala +++ b/driver/src/main/scala/com/datastax/spark/connector/types/ColumnType.scala @@ -12,7 +12,7 @@ import com.datastax.spark.connector.util._ import scala.reflect.runtime.universe._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** Serializable representation of column data type. */ trait ColumnType[T] extends Serializable { diff --git a/driver/src/main/scala/com/datastax/spark/connector/types/TupleType.scala b/driver/src/main/scala/com/datastax/spark/connector/types/TupleType.scala index d9e6445bf..2bf110ad7 100644 --- a/driver/src/main/scala/com/datastax/spark/connector/types/TupleType.scala +++ b/driver/src/main/scala/com/datastax/spark/connector/types/TupleType.scala @@ -13,7 +13,7 @@ import com.datastax.spark.connector.{ColumnName, TupleValue} import org.apache.commons.lang3.tuple.{Pair, Triple} import scala.reflect.runtime.universe._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ case class TupleFieldDef(index: Int, columnType: ColumnType[_]) extends FieldDef { override def columnName = index.toString diff --git a/driver/src/main/scala/com/datastax/spark/connector/types/TypeConverter.scala b/driver/src/main/scala/com/datastax/spark/connector/types/TypeConverter.scala index 75577cc26..58615965b 100644 --- a/driver/src/main/scala/com/datastax/spark/connector/types/TypeConverter.scala +++ b/driver/src/main/scala/com/datastax/spark/connector/types/TypeConverter.scala @@ -18,7 +18,7 @@ import org.apache.commons.lang3.tuple import scala.collection.immutable.{TreeMap, TreeSet} import scala.reflect.runtime.universe._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class TypeConversionException(val message: String, cause: Exception = null) extends Exception(message, cause) diff --git a/driver/src/main/scala/com/datastax/spark/connector/types/UserDefinedType.scala b/driver/src/main/scala/com/datastax/spark/connector/types/UserDefinedType.scala index 5efc9890e..2b8339187 100644 --- a/driver/src/main/scala/com/datastax/spark/connector/types/UserDefinedType.scala +++ b/driver/src/main/scala/com/datastax/spark/connector/types/UserDefinedType.scala @@ -11,7 +11,7 @@ import com.datastax.spark.connector.types.TypeAdapters.ValueByNameAdapter import com.datastax.spark.connector.{CassandraRowMetadata, ColumnName, UDTValue} import scala.reflect.runtime.universe._ -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ /** A Cassandra user defined type field metadata. It consists of a name and an associated column type. * The word `column` instead of `field` is used in member names because we want to treat UDT field diff --git a/driver/src/main/scala/com/datastax/spark/connector/util/NameTools.scala b/driver/src/main/scala/com/datastax/spark/connector/util/NameTools.scala index 0826d167d..580ae328a 100644 --- a/driver/src/main/scala/com/datastax/spark/connector/util/NameTools.scala +++ b/driver/src/main/scala/com/datastax/spark/connector/util/NameTools.scala @@ -6,7 +6,7 @@ import com.datastax.oss.driver.api.core.metadata.Metadata import com.datastax.spark.connector.util.DriverUtil.toName import org.apache.commons.lang3.StringUtils -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ object NameTools { diff --git a/driver/src/test/scala/com/datastax/spark/connector/cql/LocalNodeFirstLoadBalancingPolicySpec.scala b/driver/src/test/scala/com/datastax/spark/connector/cql/LocalNodeFirstLoadBalancingPolicySpec.scala index f181a8a69..5574971af 100644 --- a/driver/src/test/scala/com/datastax/spark/connector/cql/LocalNodeFirstLoadBalancingPolicySpec.scala +++ b/driver/src/test/scala/com/datastax/spark/connector/cql/LocalNodeFirstLoadBalancingPolicySpec.scala @@ -23,7 +23,7 @@ import org.mockito.{Matchers => m} import org.scalatest.{BeforeAndAfterEach, FlatSpec, Matchers} import org.scalatestplus.mockito.MockitoSugar -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ class NodeFilter(context: DriverContext, profileName: String) extends Predicate[Node] { override def test(t: Node): Boolean = DriverUtil.toAddress(t).get.getHostName.equals("192.168.123.2") diff --git a/project/Publishing.scala b/project/Publishing.scala index 0f27104c6..8fa02fe25 100644 --- a/project/Publishing.scala +++ b/project/Publishing.scala @@ -10,7 +10,7 @@ import scala.sys.process._ object Publishing extends sbt.librarymanagement.DependencyBuilders { val Version: String = { - sys.props.get("publish.version").orElse(sys.env.get("PUBLISH_VERSION")).getOrElse("git describe --tags").stripLineEnd.stripPrefix("v") + sys.props.get("publish.version").orElse(sys.env.get("PUBLISH_VERSION")).getOrElse("git describe --tags" !!).stripLineEnd.stripPrefix("v") } val altReleaseDeploymentRepository = sys.props.get("publish.repository.name") From 201492a35168fcb6ce4ee4fa177d84df1f716a59 Mon Sep 17 00:00:00 2001 From: SamTheisens <1911436+SamTheisens@users.noreply.github.com> Date: Tue, 11 Jul 2023 17:14:03 +0800 Subject: [PATCH 03/11] [SPARKC-686] Migrate from Stream to LazyList because Stream is deprecated and results in a stack overflow on scala 2.13 branch: feature/SPARKC-686-scala-213-support --- .../rdd/partitioner/TokenRangeClusterer.scala | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/TokenRangeClusterer.scala b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/TokenRangeClusterer.scala index 19fafaab7..1669bc70c 100644 --- a/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/TokenRangeClusterer.scala +++ b/connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/TokenRangeClusterer.scala @@ -9,6 +9,8 @@ import scala.annotation.tailrec import com.datastax.spark.connector.rdd.partitioner.TokenRangeClusterer.WholeRing +import scala.collection.compat.immutable.LazyList + /** Groups a set of token ranges into `groupCount` groups containing not more than `maxGroupSize` token * ranges. * Each group will form a single `CassandraRDDPartition`. @@ -30,16 +32,16 @@ class TokenRangeClusterer[V, T <: Token[V]](groupCount: Int, maxGroupSize: Int = } @tailrec - private def group(tokenRanges: Stream[TokenRange[V, T]], + private def group(tokenRanges: LazyList[TokenRange[V, T]], result: Vector[Seq[TokenRange[V, T]]], ringFractionPerGroup: Double): Iterable[Seq[TokenRange[V, T]]] = { tokenRanges match { - case Stream.Empty => result - case head #:: rest => - val firstEndpoint = head.replicas.min + case range if range.isEmpty => result + case range => + val firstEndpoint = range.head.replicas.min val ringFractions = tokenRanges.map(_.ringFraction) val cumulativeRingFractions = ringFractions.scanLeft(.0)(_ + _).tail // drop first item always == 0 - val ringFractionLimit = math.max(ringFractionPerGroup, head.ringFraction) // make sure first element will be always included + val ringFractionLimit = math.max(ringFractionPerGroup, range.head.ringFraction) // make sure first element will be always included val cluster = tokenRanges .take(math.max(1, maxGroupSize)) .zip(cumulativeRingFractions) @@ -58,7 +60,7 @@ class TokenRangeClusterer[V, T <: Token[V]](groupCount: Int, maxGroupSize: Int = // sort by endpoints lexicographically // this way ranges on the same host are grouped together val sortedRanges = tokenRanges.sortBy(_.replicas.toSeq.sorted) - group(sortedRanges.toStream, Vector.empty, ringFractionPerGroup) + group(sortedRanges.to(LazyList) , Vector.empty, ringFractionPerGroup) } } From dd3be50d71818b544fc5c616ad79426fdebb6d88 Mon Sep 17 00:00:00 2001 From: SamTheisens <1911436+SamTheisens@users.noreply.github.com> Date: Tue, 11 Jul 2023 19:34:07 +0800 Subject: [PATCH 04/11] [SPARKC-686] Avoid potential class cast exception `java.lang.ClassCastException: scala.collection.mutable.ArrayBuffer cannot be cast to scala.collection.immutable.Seq` branch: feature/SPARKC-686-scala-213-support --- .../scala/com/datastax/bdp/spark/ContinuousPagingScanner.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/connector/src/main/scala/com/datastax/bdp/spark/ContinuousPagingScanner.scala b/connector/src/main/scala/com/datastax/bdp/spark/ContinuousPagingScanner.scala index 86a4fed5d..80faa608a 100644 --- a/connector/src/main/scala/com/datastax/bdp/spark/ContinuousPagingScanner.scala +++ b/connector/src/main/scala/com/datastax/bdp/spark/ContinuousPagingScanner.scala @@ -112,7 +112,7 @@ case class ContinuousPagingScanner( private def getMetaData(result: ContinuousResultSet): CassandraRowMetadata = { import scala.jdk.CollectionConverters._ - val columnDefs = result.getColumnDefinitions.asScala + val columnDefs = result.getColumnDefinitions.asScala.toSeq val rsColumnNames = columnDefs.map(c => toName(c.getName)) val codecs = columnDefs .map(col => codecRegistry.codecFor(col.getType)) From 57efb2bfda195979b49271efa2e1d591761b9d71 Mon Sep 17 00:00:00 2001 From: SamTheisens <1911436+SamTheisens@users.noreply.github.com> Date: Tue, 11 Jul 2023 21:28:48 +0800 Subject: [PATCH 05/11] [SPARKC-686] Fix typos in documentation branch: feature/SPARKC-686-scala-213-support --- build.sbt | 3 --- .../rdd/partitioner/CassandraPartitionGeneratorSpec.scala | 2 +- doc/developers.md | 6 +++--- 3 files changed, 4 insertions(+), 7 deletions(-) diff --git a/build.sbt b/build.sbt index 2aa25827f..4a8f2581a 100644 --- a/build.sbt +++ b/build.sbt @@ -62,9 +62,6 @@ scalaReleaseVersion := { } } - - - lazy val commonSettings = Seq( // dependency updates check dependencyUpdatesFailBuild := true, diff --git a/connector/src/it/scala/com/datastax/spark/connector/rdd/partitioner/CassandraPartitionGeneratorSpec.scala b/connector/src/it/scala/com/datastax/spark/connector/rdd/partitioner/CassandraPartitionGeneratorSpec.scala index 050462655..123aa1904 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/rdd/partitioner/CassandraPartitionGeneratorSpec.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/rdd/partitioner/CassandraPartitionGeneratorSpec.scala @@ -69,7 +69,7 @@ class CassandraPartitionGeneratorSpec extends SparkCassandraITFlatSpecBase with testPartitionCount(10000, 9000, 11000) } - it should "round robing partition with different endpoints" in { + it should "round robin partition with different endpoints" in { val table = schemaFromCassandra(conn, Some(ks), Some("empty")).tables.head val partitioner = new MockCassandraPartitionGenerator(conn, table, 12) val partitions = partitioner.partitions diff --git a/doc/developers.md b/doc/developers.md index e9736af83..afaf1eb7f 100644 --- a/doc/developers.md +++ b/doc/developers.md @@ -5,7 +5,7 @@ ### Getting Started The Spark Cassandra Connector is built using sbt. There is a premade -launching script for sbt so it is unneccessary to download it. To invoke +launching script for sbt, so it is unnecessary to download it. To invoke this script you can run `./sbt/sbt` from a clone of this repository. For information on setting up your clone please follow the [Github @@ -48,7 +48,7 @@ datastax git@github.com:datastax/spark-cassandra-connector.git (push) ... ``` -Here is how the work should look like. +Here is what the work should look like. ```shell git fetch datastax @@ -108,7 +108,7 @@ In order to limit the number of test groups running simultaneously use the ### Set Cassandra Test Target Our CI Build runs through the Datastax Infrastructure and tests on all the builds -listed in build.yaml. In addition the _test-support_ module supports Cassandra +listed in build.yaml. In addition, the _test-support_ module supports Cassandra or other CCM Compatible installations. If using SBT you can set From 4a676f5c92e28695789dc834fcf928f8e5735fe5 Mon Sep 17 00:00:00 2001 From: SamTheisens <1911436+SamTheisens@users.noreply.github.com> Date: Wed, 12 Jul 2023 09:31:03 +0800 Subject: [PATCH 06/11] [SPARKC-686] Clean up conditional library import branch: feature/SPARKC-686-scala-213-support --- build.sbt | 25 +++++-------------------- project/Dependencies.scala | 13 +++++++++++++ project/Versions.scala | 20 ++------------------ 3 files changed, 20 insertions(+), 38 deletions(-) diff --git a/build.sbt b/build.sbt index 4a8f2581a..7321278b1 100644 --- a/build.sbt +++ b/build.sbt @@ -54,14 +54,6 @@ lazy val assemblySettings = Seq( }, ) -val scalaReleaseVersion = SettingKey[Int]("scalaReleaseVersion") -scalaReleaseVersion := { - val v = scalaVersion.value - CrossVersion.partialVersion(v).map(_._1.toInt).getOrElse { - throw new RuntimeException(s"could not get Scala release version from $v") - } -} - lazy val commonSettings = Seq( // dependency updates check dependencyUpdatesFailBuild := true, @@ -70,16 +62,6 @@ lazy val commonSettings = Seq( parallelExecution := true, testForkedParallel := false, testOptions += Tests.Argument(TestFrameworks.JUnit, "-v"), - libraryDependencies ++= { - CrossVersion.partialVersion(scalaVersion.value) match { - case Some((2, scalaMajor)) if scalaMajor == 13 => - Seq( - "org.scala-lang.modules" %% "scala-collection-compat" % "2.11.0", - "org.scala-lang.modules" %% "scala-parallel-collections" % "1.0.4" - ) - case _ => Seq("org.scala-lang.modules" %% "scala-collection-compat" % "2.11.0") - } - } ) @@ -127,6 +109,7 @@ lazy val connector = (project in file("connector")) Global / concurrentRestrictions := Seq(Tags.limitAll(Testing.parallelTasks)), libraryDependencies ++= Dependencies.Spark.dependencies + ++ Dependencies.Compatibility.dependencies(scalaVersion.value) ++ Dependencies.TestConnector.dependencies ++ Dependencies.Jetty.dependencies, @@ -143,7 +126,8 @@ lazy val testSupport = (project in file("test-support")) .settings( crossScalaVersions := supportedScalaVersions, name := "spark-cassandra-connector-test-support", - libraryDependencies ++= Dependencies.TestSupport.dependencies + libraryDependencies ++= Dependencies.Compatibility.dependencies(scalaVersion.value) + ++ Dependencies.TestSupport.dependencies ) lazy val driver = (project in file("driver")) @@ -153,7 +137,8 @@ lazy val driver = (project in file("driver")) crossScalaVersions := supportedScalaVersions, name := "spark-cassandra-connector-driver", assembly /test := {}, - libraryDependencies ++= Dependencies.Driver.dependencies + libraryDependencies ++= Dependencies.Compatibility.dependencies(scalaVersion.value) + ++ Dependencies.Driver.dependencies ++ Dependencies.TestDriver.dependencies :+ ("org.scala-lang" % "scala-reflect" % scalaVersion.value) ) diff --git a/project/Dependencies.scala b/project/Dependencies.scala index c8cb14ee7..d6d43598e 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -79,9 +79,22 @@ object Dependencies val commonsLang3 = "org.apache.commons" % "commons-lang3" % Versions.CommonsLang3 val paranamer = "com.thoughtworks.paranamer" % "paranamer" % Versions.Paranamer + val dependencies = Seq(driverCore, driverMapper, commonsLang3, paranamer) } + object Compatibility { + val scalaCompat = "org.scala-lang.modules" %% "scala-collection-compat" % Versions.ScalaCompat + val parallelCollections = "org.scala-lang.modules" %% "scala-parallel-collections" % Versions.ParallelCollections + + def dependencies(version: String): Seq[ModuleID] = { + CrossVersion.partialVersion(version) match { + case Some((2, scalaMajor)) if scalaMajor == 13 => Seq(scalaCompat, parallelCollections) + case _ => Seq(scalaCompat) + } + } + } + object TestDriver { val dependencies = Seq( TestCommon.scalaTest % "test", diff --git a/project/Versions.scala b/project/Versions.scala index 51922582d..a9d8b7e44 100644 --- a/project/Versions.scala +++ b/project/Versions.scala @@ -13,26 +13,10 @@ object Versions { val JUnitInterface = "0.11" val Mockito = "1.10.19" - // Spark version can be specified as: - // - regular version which is present in some public Maven repository - // - a release tag in https://github.com/apache/spark - // - one of main branches, like master or branch-x.y, followed by "-SNAPSHOT" suffix - // The last two cases trigger the build to clone the given revision of Spark from GitHub, build it - // and install in a local Maven repository. This is all done automatically, however it will work - // only on Unix/OSX operating system. Windows users have to build and install Spark manually if the - // desired version is not yet published into a public Maven repository. val ApacheSpark = "3.4.1" val SparkJetty = "9.4.50.v20221201" val SolrJ = "8.3.0" - /* - val status = (versionInReapply: String, binaryInReapply: String) => - println(s""" - | Scala: $versionInReapply - | Scala Binary: $binaryInReapply - | Java: target=$JDK user=${Properties.javaVersion} - | Cassandra version for testing: ${Testing.cassandraTestVersion} [can be overridden by specifying '-Dtest.cassandra.version='] - """.stripMargin) - - */ + val ScalaCompat = "2.11.0" + val ParallelCollections = "1.0.4" } From 364b374256924f2e96720441ab755c192741df6b Mon Sep 17 00:00:00 2001 From: SamTheisens <1911436+SamTheisens@users.noreply.github.com> Date: Wed, 12 Jul 2023 10:20:59 +0800 Subject: [PATCH 07/11] [SPARKC-686] Publish test results so we don't need to trawl through the (long) log output to find out which test failed. Annotate only, which doesn't require check permission. branch: feature/SPARKC-686-scala-213-support --- .github/workflows/main.yml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index dff6cb917..3de35cc2a 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -36,3 +36,10 @@ jobs: CCM_CASSANDRA_VERSION: ${{ matrix.db-version }} PUBLISH_VERSION: test run: sbt/sbt ++${{ matrix.scala }} test it:test + + - name: Publish Test Report + uses: mikepenz/action-junit-report@v3 + if: always() + with: + report_paths: '**/target/test-reports/*.xml' + annotate_only: true \ No newline at end of file From 9176052b452ed2d86dc1a6eeb67b9b17fe068300 Mon Sep 17 00:00:00 2001 From: SamTheisens <1911436+SamTheisens@users.noreply.github.com> Date: Sun, 16 Jul 2023 11:13:45 +0800 Subject: [PATCH 08/11] [SPARKC-686] Removing cassandra 5.1.24 as requested https://github.com/SamTheisens/spark-cassandra-connector/pull/1#discussion_r1263734916 branch: feature/SPARKC-686-scala-213-support --- .github/workflows/main.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 3de35cc2a..b585bc1f7 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -15,7 +15,7 @@ jobs: strategy: matrix: scala: [2.12.11, 2.13.11] - db-version: [6.8.13, 5.1.24, 3.11.10, 4.0-rc2] + db-version: [3.11.10, 4.0-rc2, 6.8.13] steps: - uses: actions/checkout@v2 From cb36121b0f6c6142153b94dc09f19659ac26ffbe Mon Sep 17 00:00:00 2001 From: SamTheisens <1911436+SamTheisens@users.noreply.github.com> Date: Fri, 28 Jul 2023 20:23:46 +0800 Subject: [PATCH 09/11] [SPARKC-686] Fix incorrect import branch: feature/SPARKC-686-scala-213-support --- .../src/it/scala/com/datastax/spark/connector/rdd/RDDSpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/connector/src/it/scala/com/datastax/spark/connector/rdd/RDDSpec.scala b/connector/src/it/scala/com/datastax/spark/connector/rdd/RDDSpec.scala index 3d03f5a6a..d882cbbd6 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/rdd/RDDSpec.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/rdd/RDDSpec.scala @@ -11,10 +11,10 @@ import com.datastax.spark.connector.cql.CassandraConnector import com.datastax.spark.connector.embedded.SparkTemplate._ import com.datastax.spark.connector.rdd.partitioner.EndpointPartition import com.datastax.spark.connector.writer.AsyncExecutor -import spire.ClassTag import scala.jdk.CollectionConverters._ import scala.concurrent.Future +import scala.reflect.ClassTag case class KVRow(key: Int) From c6d84c6b395f343903dc7d61aec3eb99da966885 Mon Sep 17 00:00:00 2001 From: SamTheisens <1911436+SamTheisens@users.noreply.github.com> Date: Fri, 28 Jul 2023 20:27:11 +0800 Subject: [PATCH 10/11] [SPARKC-686] Remove accidental dots in spec names branch: feature/SPARKC-686-scala-213-support --- .../spark/connector/writer/TableWriterSpec.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/connector/src/it/scala/com/datastax/spark/connector/writer/TableWriterSpec.scala b/connector/src/it/scala/com/datastax/spark/connector/writer/TableWriterSpec.scala index 69cb1afea..849ad1564 100644 --- a/connector/src/it/scala/com/datastax/spark/connector/writer/TableWriterSpec.scala +++ b/connector/src/it/scala/com/datastax/spark/connector/writer/TableWriterSpec.scala @@ -771,7 +771,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { verifyKeyValueTable("key_value") } - it should "be able to.append and.prepend elements to a C* list" in { + it should "be able to append and prepend elements to a C* list" in { val listElements = sc.parallelize(Seq( (1, Vector("One")), @@ -825,7 +825,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { testSet should contain allOf("One", "Two", "Three") } - it should "be able to.remove elements from a C* set" in { + it should "be able to remove elements from a C* set" in { val setElements = sc.parallelize(Seq( (4, Set("One")), (4, Set("Two")), @@ -866,7 +866,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { e.getMessage should include("group") } - it should "throw an exception if you try to.remove values from a map" in { + it should "throw an exception if you try to remove values from a map" in { val setElements = sc.parallelize(Seq( (5, Map("One" -> "One")), (5, Map("Two" -> "Two")), @@ -877,7 +877,7 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { e.getMessage should include("mcol") } - it should "throw an exception if you.prepend anything but a list" in { + it should "throw an exception if you prepend anything but a list" in { val setElements = sc.parallelize(Seq( (5, Map("One" -> "One"), Set("One")))) val e = intercept[IllegalArgumentException] { @@ -915,12 +915,12 @@ class TableWriterSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { results should contain theSameElementsAs Seq((1, "new"), (2, "new")) } - "Idempotent Queries" should "not be used with list.append" in { + "Idempotent Queries" should "not be used with list append" in { val listAppend = TableWriter(conn, ks, "collections_mod", SomeColumns("key", "lcol".append), WriteConf.fromSparkConf(sc.getConf)) listAppend.isIdempotent should be (false) } - it should "not be used with list.prepend" in { + it should "not be used with list prepend" in { val listPrepend = TableWriter(conn, ks, "collections_mod", SomeColumns("key", "lcol".prepend), WriteConf.fromSparkConf(sc.getConf)) listPrepend.isIdempotent should be (false) } From 1e5702e3acd4d5f5f01c5c3d8cd8e090a7c052a7 Mon Sep 17 00:00:00 2001 From: SamTheisens <1911436+SamTheisens@users.noreply.github.com> Date: Fri, 28 Jul 2023 20:54:59 +0800 Subject: [PATCH 11/11] [SPARKC-686] Replace closure with idiomatic mapValues the .toMap is necessary for scala 2.13 as the function returns a `scala.collection.MapView` instead of Map branch: feature/SPARKC-686-scala-213-support --- .../connector/mapper/GettableDataToMappedTypeConverter.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/driver/src/main/scala/com/datastax/spark/connector/mapper/GettableDataToMappedTypeConverter.scala b/driver/src/main/scala/com/datastax/spark/connector/mapper/GettableDataToMappedTypeConverter.scala index bc8aad1c5..b4afa5da6 100644 --- a/driver/src/main/scala/com/datastax/spark/connector/mapper/GettableDataToMappedTypeConverter.scala +++ b/driver/src/main/scala/com/datastax/spark/connector/mapper/GettableDataToMappedTypeConverter.scala @@ -175,7 +175,7 @@ class GettableDataToMappedTypeConverter[T : TypeTag : ColumnMapper]( for ((s, _) <- columnMap.setters) yield (s, ReflectionUtil.methodParamTypes(targetType, s).head) val setterColumnTypes: Map[String, ColumnType[_]] = - columnMap.setters.map{case (k, v) => (k, columnType(v))}.toMap + columnMap.setters.mapValues(columnType).toMap for (setterName <- setterParamTypes.keys) yield { val ct = setterColumnTypes(setterName) val pt = setterParamTypes(setterName)