Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

SPARKC-577: Removal of Driver Duplicate Classes #1245

Closed
wants to merge 12 commits into from
Closed
Show file tree
Hide file tree
Changes from 7 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
package com.datastax.spark.connector

import com.datastax.oss.driver.api.core.ProtocolVersion
import com.datastax.oss.driver.api.core.metadata.schema.ClusteringOrder
import com.datastax.spark.connector.cql._
import com.datastax.spark.connector.mapper.DataFrameColumnMapper
import org.apache.spark.SparkContext
Expand All @@ -22,80 +22,45 @@ class DatasetFunctions[K: Encoder](dataset: Dataset[K]) extends Serializable {
* the first column will be used as the partition key and there will be no clustering
* keys.
*/
def createCassandraTable(
keyspaceName: String,
def createCassandraTable(keyspaceName: String,
tableName: String,
partitionKeyColumns: Option[Seq[String]] = None,
clusteringKeyColumns: Option[Seq[String]] = None)(
implicit
connector: CassandraConnector = CassandraConnector(sparkContext)): Unit = {
clusteringKeyColumns: Option[Seq[String]] = None)
(implicit connector: CassandraConnector = CassandraConnector(sparkContext)): Unit = {

val protocolVersion = connector.withSessionDo(_.getContext.getProtocolVersion)
val rawTable = new DataFrameColumnMapper(dataset.schema).newTable(keyspaceName, tableName, protocolVersion)
val partitionKeyNames = partitionKeyColumns.getOrElse(rawTable.partitionKey.map(_.columnName))
val clusteringKeyNames = clusteringKeyColumns.getOrElse(Seq.empty)
val partitionKeys = partitionKeyColumns.map(_.toSet).getOrElse(Set[String]())
val clusteringKeys = clusteringKeyColumns.map(_.toSet).getOrElse(Set[String]())

createCassandraTableEx(keyspaceName, tableName, partitionKeyNames,
clusteringKeyNames.map((_, ClusteringColumn.Ascending)))(connector)
}

/**
* Creates a C* table based on the Dataset Struct provided.
* Takes in a list of partition columns, clustering columns names, and optionally, the table options.
*/
def createCassandraTableEx(
keyspaceName: String,
tableName: String,
partitionKeyColumns: Seq[String],
clusteringKeyColumns: Seq[(String, ClusteringColumn.SortingOrder)],
ifNotExists: Boolean = false,
tableOptions: Map[String, String] = Map())(
implicit
connector: CassandraConnector = CassandraConnector(sparkContext)): Unit = {

val protocolVersion = connector.withSessionDo(_.getContext.getProtocolVersion)
val originalTable = new DataFrameColumnMapper(dataset.schema).newTable(keyspaceName, tableName, protocolVersion)
val newColumns = originalTable.cols.map(col => {
col.copy(
isParitionKey = partitionKeys(col.name),
clusteringKey = if (clusteringKeys(col.name)) Option(ClusteringOrder.ASC) else Option.empty
)
})
val newPartitionKeys = newColumns.filter(_.isParitionKey).map(_.name).toSet
val newClusteringKeys = newColumns.filter(_.clusteringKey.isDefined).map(_.name).toSet

val rawTable = new DataFrameColumnMapper(dataset.schema).newTable(keyspaceName, tableName, protocolVersion)
val columnMapping = rawTable.columnByName

val columnNames = columnMapping.keys.toSet
val partitionKeyNames = partitionKeyColumns
val clusteringKeyNames = clusteringKeyColumns.map(_._1)
val regularColumnNames = (columnNames -- (partitionKeyNames ++ clusteringKeyNames)).toSeq

def missingColumnException(columnName: String, columnType: String) = {
new IllegalArgumentException(
s""""$columnName" not Found. Unable to make specified column $columnName a $columnType.
|Available Columns: $columnNames""".stripMargin)
val missedPartitionKeys = partitionKeys -- newPartitionKeys
if (!missedPartitionKeys.isEmpty) {
throw new IllegalArgumentException(
s""""Columns $missedPartitionKeys" not Found. Unable to make specified column(s) partition key(s).
|Available Columns: ${originalTable.cols.map(_.name)}""".stripMargin)
}
val missedClusteringKeys = clusteringKeys -- newClusteringKeys
if (!missedClusteringKeys.isEmpty) {
throw new IllegalArgumentException(
s""""Columns $missedClusteringKeys" not Found. Unable to make specified column(s) clustering key(s).
|Available Columns: ${originalTable.cols.map(_.name)}""".stripMargin)
}

val table = rawTable.copy (
partitionKey = partitionKeyNames
.map(partitionKeyName =>
columnMapping.getOrElse(partitionKeyName,
throw missingColumnException(partitionKeyName, "Partition Key Column")))
.map(_.copy(columnRole = PartitionKeyColumn))
,
clusteringColumns = clusteringKeyColumns
.map(clusteringKey =>
(columnMapping.getOrElse(clusteringKey._1,
throw missingColumnException(clusteringKey._1, "Clustering Column")),
clusteringKey._2))
.zipWithIndex
.map { case (col, index) => col._1.copy(columnRole = ClusteringColumn(index, col._2))}
,
regularColumns = regularColumnNames
.map(regularColumnName =>
columnMapping.getOrElse(regularColumnName,
throw missingColumnException(regularColumnName, "Regular Column")))
.map(_.copy(columnRole = RegularColumn))
,
ifNotExists = ifNotExists
,
tableOptions = tableOptions
val newTable = originalTable.copy(
cols = newColumns,
ifNotExists = false,
options = Map()
)

connector.withSessionDo(session => session.execute(table.cql))
connector.withSessionDo(session => session.execute(newTable.cql))
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,7 @@ package com.datastax.spark.connector
import java.net.InetAddress

import com.datastax.spark.connector.cql._
import com.datastax.spark.connector.mapper.ColumnMapper
import com.datastax.spark.connector.mapper.{ColumnMapper, TableDescriptor}
import com.datastax.spark.connector.rdd.partitioner.{CassandraPartitionedRDD, ReplicaPartitioner}
import com.datastax.spark.connector.rdd.reader._
import com.datastax.spark.connector.rdd._
Expand Down Expand Up @@ -53,15 +53,15 @@ class RDDFunctions[T](rdd: RDD[T]) extends WritableToCassandra[T] with Serializa
* from items of the [[org.apache.spark.rdd.RDD RDD]]
*/
def saveAsCassandraTableEx(
table: TableDef,
table: TableDescriptor,
columns: ColumnSelector = AllColumns,
writeConf: WriteConf = WriteConf.fromSparkConf(sparkContext.getConf))(
implicit
connector: CassandraConnector = CassandraConnector(sparkContext),
rwf: RowWriterFactory[T]): Unit = {

connector.withSessionDo(session => session.execute(table.cql))
saveToCassandra(table.keyspaceName, table.tableName, columns, writeConf)
saveToCassandra(table.keyspace, table.name, columns, writeConf)
}

/**
Expand Down Expand Up @@ -92,7 +92,7 @@ class RDDFunctions[T](rdd: RDD[T]) extends WritableToCassandra[T] with Serializa

val protocolVersion = connector.withSessionDo(_.getContext.getProtocolVersion)

val table = TableDef.fromType[T](keyspaceName, tableName, protocolVersion)
val table = TableDescriptor.fromType[T](keyspaceName, tableName, protocolVersion)
saveAsCassandraTableEx(table, columns, writeConf)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,14 +26,15 @@ class DataFrameColumnMapper[T](structType: StructType) extends ColumnMapper[T] {
override def newTable(
keyspaceName: String,
tableName: String,
protocolVersion: ProtocolVersion = ProtocolVersion.DEFAULT): TableDef = {
protocolVersion: ProtocolVersion = ProtocolVersion.DEFAULT): TableDescriptor = {

val columns = structType.zipWithIndex.map { case (field, i) => {
val columnRole = if (i == 0) PartitionKeyColumn else RegularColumn
ColumnDef(field.name, columnRole, DataFrameColumnMapper.fromSparkSqlType(field.dataType, protocolVersion))
}}
val columns = structType.zipWithIndex.map { case (field:StructField, i:Int) =>
ColumnDescriptor(field.name,
DataFrameColumnMapper.fromSparkSqlType(field.dataType, protocolVersion),
i == 0)
}

TableDef(keyspaceName, tableName, Seq(columns.head), Seq.empty, columns.tail)
TableDescriptor(keyspaceName, tableName, columns)
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,6 @@ package com.datastax.spark.connector.rdd

import java.util.concurrent.Future

import com.datastax.driver.core._
import com.datastax.oss.driver.api.core.CqlSession
import com.datastax.oss.driver.api.core.cql.{PreparedStatement, Row, SimpleStatement}
import com.datastax.spark.connector._
Expand Down Expand Up @@ -71,7 +70,7 @@ private[rdd] trait AbstractCassandraJoin[L, R] {
case PrimaryKeyColumns =>
tableDef.primaryKey.map(col => col.columnName: ColumnRef)
case PartitionKeyColumns =>
tableDef.partitionKey.map(col => col.columnName: ColumnRef)
tableDef.partitionKey.map(col => col.columnName: ColumnRef).toSeq
case SomeColumns(cs @ _*) =>
checkColumnsExistence(cs)
cs.map {
Expand Down Expand Up @@ -108,8 +107,8 @@ private[rdd] trait AbstractCassandraJoin[L, R] {
.filter(cc => colNames.contains(cc.columnName))
if (!tableDef.clusteringColumns.startsWith(chosenClusteringColumns)) {
val maxCol = chosenClusteringColumns.last
val maxIndex = maxCol.componentIndex.get
val requiredColumns = tableDef.clusteringColumns.takeWhile(_.componentIndex.get <= maxIndex)
val maxIndex = tableDef.clusteringColumns.indexOf(maxCol)
val requiredColumns = tableDef.clusteringColumns.take(maxIndex + 1)
Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Wanted to highlight this for review. I'm pretty sure the logic I have in there now mirrors what was being done but wanted to make sure this was looked at more closely.

val missingColumns = requiredColumns.toSet -- chosenClusteringColumns.toSet
throw new IllegalArgumentException(
s"Can't pushdown join on column $maxCol without also specifying [ $missingColumns ]"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -280,9 +280,9 @@ object TableWriter {
private def onlyPartitionKeyAndStatic(table: TableDef, columnNames: Seq[String]): Boolean = {
val nonPartitionKeyColumnNames = columnNames.toSet -- table.partitionKey.map(_.columnName)
val nonPartitionKeyColumnRefs = table
.allColumns
.cols
.filter(columnDef => nonPartitionKeyColumnNames.contains(columnDef.columnName))
nonPartitionKeyColumnRefs.forall( columnDef => columnDef.columnRole == StaticColumn)
nonPartitionKeyColumnRefs.forall( columnDef => columnDef.isStatic)
}

/**
Expand Down Expand Up @@ -376,15 +376,13 @@ object TableWriter {
checkPartitionKey: Boolean = false): TableWriter[T] = {

val tableDef = tableFromCassandra(connector, keyspaceName, tableName)
val optionColumns = writeConf.optionsAsColumns(keyspaceName, tableName)
val tablDefWithMeta = tableDef.copy(regularColumns = tableDef.regularColumns ++ optionColumns)

val selectedColumns = columnNames
.selectFrom(tablDefWithMeta)
.selectFrom(tableDef)
.filter(col => !InternalColumns.contains(col.columnName))
val rowWriter = implicitly[RowWriterFactory[T]].rowWriter(tablDefWithMeta, selectedColumns)
val rowWriter = implicitly[RowWriterFactory[T]].rowWriter(tableDef, selectedColumns)

checkColumns(tablDefWithMeta, selectedColumns, checkPartitionKey)
new TableWriter[T](connector, tablDefWithMeta, selectedColumns, rowWriter, writeConf)
checkColumns(tableDef, selectedColumns, checkPartitionKey)
new TableWriter[T](connector, tableDef, selectedColumns, rowWriter, writeConf)
}
}
Original file line number Diff line number Diff line change
@@ -1,9 +1,6 @@
package com.datastax.spark.connector.writer

import com.datastax.oss.driver.api.core.{ConsistencyLevel, DefaultConsistencyLevel}
import com.datastax.oss.driver.api.core.`type`.{DataType, DataTypes}
import com.datastax.spark.connector.cql.{ColumnDef, RegularColumn}
import com.datastax.spark.connector.types.ColumnType
import com.datastax.spark.connector.util.ConfigCheck.ConnectorConfigurationException
import com.datastax.spark.connector.util.{ConfigCheck, ConfigParameter, DeprecatedConfigParameter}
import com.datastax.spark.connector.{BatchSize, BytesInBatch, RowsInBatch}
Expand Down Expand Up @@ -42,16 +39,6 @@ case class WriteConf(
case WriteOption(PerRowWriteOptionValue(placeholder)) => placeholder
}

private[writer] val optionsAsColumns: (String, String) => Seq[ColumnDef] = { (keyspace, table) =>
def toRegularColDef(opt: WriteOption[_], dataType: DataType) = opt match {
case WriteOption(PerRowWriteOptionValue(placeholder)) =>
Some(ColumnDef(placeholder, RegularColumn, ColumnType.fromDriverType(dataType)))
case _ => None
}

Seq(toRegularColDef(ttl, DataTypes.INT), toRegularColDef(timestamp, DataTypes.BIGINT)).flatten
}

val throttlingEnabled = throughputMiBPS.isDefined
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@ import org.apache.spark.sql.execution.datasources.LogicalRelation
import org.apache.spark.sql.sources._
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
import com.datastax.spark.connector.cql.{CassandraConnector, CassandraConnectorConf, ColumnDef, Schema, TableDef}
import com.datastax.spark.connector.cql.{CassandraConnector, CassandraConnectorConf, ColumnDef, TableDef}
import com.datastax.spark.connector.rdd.partitioner.DataSizeEstimates
import com.datastax.spark.connector.rdd.partitioner.dht.TokenFactory.forSystemLocalPartitioner
import com.datastax.spark.connector.rdd.{CassandraJoinRDD, CassandraRDD, CassandraTableScanRDD, ReadConf}
Expand Down Expand Up @@ -402,7 +402,7 @@ case class CassandraSourceRelation(
val tableIsSolrIndexed =
rdd.tableDef
.indexes
.exists(index => index.className.contains(SolrConstants.DseSolrIndexClassName))
.exists(index => index.index.getClassName.filter(s => s.contains(SolrConstants.DseSolrIndexClassName)).isPresent)
val countRDD =
if (searchOptimization.enabled && tableIsSolrIndexed && rdd.where.predicates.isEmpty){
//This will shortcut actually reading the rows out of Cassandra and just hit the
Expand Down
Loading