-
Notifications
You must be signed in to change notification settings - Fork 917
Commit
- Loading branch information
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,55 @@ | ||
package com.datastax.spark.connector.rdd.typeTests | ||
|
||
import com.datastax.oss.driver.api.core.CqlSession | ||
import com.datastax.spark.connector._ | ||
import com.datastax.spark.connector.cluster.DefaultCluster | ||
import com.datastax.spark.connector.cql.CassandraConnector | ||
import org.apache.spark.sql.cassandra.DataFrameWriterWrapper | ||
|
||
class VectorTypeTest extends SparkCassandraITFlatSpecBase with DefaultCluster | ||
Check failure on line 9 in connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/VectorTypeTest.scala GitHub Actions / build (2.12.19, 3.11.17)VectorTypeTest.(It is not a test it is a sbt.testing.SuiteSelector)
Check failure on line 9 in connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/VectorTypeTest.scala GitHub Actions / build (2.12.19, 4.0.12)VectorTypeTest.(It is not a test it is a sbt.testing.SuiteSelector)
Check failure on line 9 in connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/VectorTypeTest.scala GitHub Actions / build (2.12.19, 4.1.4)VectorTypeTest.(It is not a test it is a sbt.testing.SuiteSelector)
Check failure on line 9 in connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/VectorTypeTest.scala GitHub Actions / build (2.12.19, dse-6.8.44)VectorTypeTest.(It is not a test it is a sbt.testing.SuiteSelector)
Check failure on line 9 in connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/VectorTypeTest.scala GitHub Actions / build (2.13.13, 3.11.17)VectorTypeTest.(It is not a test it is a sbt.testing.SuiteSelector)
Check failure on line 9 in connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/VectorTypeTest.scala GitHub Actions / build (2.13.13, 4.0.12)VectorTypeTest.(It is not a test it is a sbt.testing.SuiteSelector)
Check failure on line 9 in connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/VectorTypeTest.scala GitHub Actions / build (2.13.13, 4.1.4)VectorTypeTest.(It is not a test it is a sbt.testing.SuiteSelector)
Check failure on line 9 in connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/VectorTypeTest.scala GitHub Actions / build (2.13.13, dse-6.8.44)VectorTypeTest.(It is not a test it is a sbt.testing.SuiteSelector)
|
||
{ | ||
override lazy val conn = CassandraConnector(sparkConf) | ||
|
||
val VectorTable = "vectors" | ||
|
||
def createVectorTable(session: CqlSession): Unit = { | ||
session.execute( | ||
s"""CREATE TABLE IF NOT EXISTS $ks.$VectorTable | ||
|(id int PRIMARY KEY, v vector<int, 5>);""".stripMargin) | ||
} | ||
|
||
override def beforeClass { | ||
conn.withSessionDo { session => | ||
session.execute( | ||
s"""CREATE KEYSPACE IF NOT EXISTS $ks | ||
|WITH REPLICATION = { 'class': 'SimpleStrategy', 'replication_factor': 1 }""" | ||
.stripMargin) | ||
createVectorTable(session) | ||
} | ||
} | ||
|
||
"SparkSql" should "write tuples with BLOB elements" in { | ||
spark.createDataFrame(Seq(VectorItem(1, Seq(1,2,3,4,5)), VectorItem(2, Seq(6,7,8,9,10)))) | ||
.write | ||
.cassandraFormat(VectorTable, ks) | ||
.mode("append") | ||
.save() | ||
|
||
val tupleRows = spark.sparkContext | ||
.cassandraTable[(Int, Seq[Int])](ks, VectorTable) | ||
.collect() | ||
.toList | ||
|
||
tupleRows should contain theSameElementsAs Seq((1, Seq(1,2,3,4,5)), (2, Seq(6,7,8,9,10))) | ||
|
||
val rows = spark.sparkContext | ||
.cassandraTable[VectorItem](ks, VectorTable) | ||
.collect() | ||
.toList | ||
|
||
rows should contain theSameElementsAs Seq(VectorItem(1, Seq(1,2,3,4,5)), VectorItem(2, Seq(6,7,8,9,10))) | ||
} | ||
|
||
} | ||
|
||
case class VectorItem(id: Int, v: Seq[Int]) |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,20 @@ | ||
package com.datastax.spark.connector.types | ||
|
||
import scala.language.existentials | ||
import scala.reflect.runtime.universe._ | ||
|
||
case class VectorType[T](elemType: ColumnType[T], dimension: Int) extends ColumnType[Seq[T]] { | ||
|
||
override def isCollection: Boolean = false | ||
|
||
@transient | ||
lazy val scalaTypeTag = { | ||
implicit val elemTypeTag = elemType.scalaTypeTag | ||
implicitly[TypeTag[Seq[T]]] | ||
} | ||
|
||
def cqlTypeName = s"vector<${elemType.cqlTypeName}, ${dimension}>" | ||
|
||
override def converterToCassandra: TypeConverter[_ <: AnyRef] = | ||
new TypeConverter.OptionToNullConverter(TypeConverter.seqConverter(elemType.converterToCassandra)) | ||
} |