Skip to content

Commit

Permalink
Move Functionality in Cassandra
Browse files Browse the repository at this point in the history
  • Loading branch information
James McClain authored and echeipesh committed Oct 4, 2017
1 parent b2d38a8 commit ec23c0d
Show file tree
Hide file tree
Showing 3 changed files with 121 additions and 77 deletions.
Expand Up @@ -30,6 +30,7 @@ class CassandraLayerReader(val attributeStore: AttributeStore, instance: Cassand
extends FilteringLayerReader[LayerId] {

val defaultNumPartitions = sc.defaultParallelism
def sparkContext: SparkContext = sc

def read[
K: AvroRecordCodec: Boundable: JsonFormat: ClassTag,
Expand Down
Expand Up @@ -35,96 +35,38 @@ class CassandraLayerUpdater(
layerReader: CassandraLayerReader
) extends LayerUpdater[LayerId] with LazyLogging {

protected def _overwrite[
implicit val sc = layerReader.sparkContext

def update[
K: AvroRecordCodec: Boundable: JsonFormat: ClassTag,
V: AvroRecordCodec: ClassTag,
M: JsonFormat: GetComponent[?, Bounds[K]]: Mergable
](
id: LayerId,
rdd: RDD[(K, V)] with Metadata[M],
keyBounds: KeyBounds[K]
): Unit = {
_update(id, rdd, keyBounds, None)
](id: LayerId, rdd: RDD[(K, V)] with Metadata[M], mergeFunc: (V, V) => V): Unit = {
val CassandraLayerHeader(_, _, keyspace, table) = attributeStore.readHeader[CassandraLayerHeader](id)
val layerWriter = new CassandraLayerWriter(attributeStore, instance, keyspace, table)
layerWriter.update(id, rdd, mergeFunc)
}

protected def _update[
def update[
K: AvroRecordCodec: Boundable: JsonFormat: ClassTag,
V: AvroRecordCodec: ClassTag,
M: JsonFormat: GetComponent[?, Bounds[K]]: Mergable
](
id: LayerId,
rdd: RDD[(K, V)] with Metadata[M],
keyBounds: KeyBounds[K],
mergeFunc: (V, V) => V
): Unit = {
_update(id, rdd, keyBounds, Some(mergeFunc))
](id: LayerId, rdd: RDD[(K, V)] with Metadata[M]): Unit = {
val CassandraLayerHeader(_, _, keyspace, table) = attributeStore.readHeader[CassandraLayerHeader](id)
val layerWriter = new CassandraLayerWriter(attributeStore, instance, keyspace, table)
layerWriter.update(id, rdd)
}

def _update[
def overwrite[
K: AvroRecordCodec: Boundable: JsonFormat: ClassTag,
V: AvroRecordCodec: ClassTag,
M: JsonFormat: GetComponent[?, Bounds[K]]: Mergable
](
id: LayerId,
rdd: RDD[(K, V)] with Metadata[M],
keyBounds: KeyBounds[K],
mergeFunc: Option[(V, V) => V]
) = {
if (!attributeStore.layerExists(id)) throw new LayerNotFoundError(id)

val LayerAttributes(header, metadata, keyIndex, writerSchema) = try {
attributeStore.readLayerAttributes[CassandraLayerHeader, M, K](id)
} catch {
case e: AttributeNotFoundError => throw new LayerUpdateError(id).initCause(e)
}

val (keyspace, table) = header.keyspace -> header.tileTable

if (!(keyIndex.keyBounds contains keyBounds))
throw new LayerOutOfKeyBoundsError(id, keyIndex.keyBounds)

val encodeKey = (key: K) => keyIndex.toIndex(key)

logger.info(s"Saving updated RDD for layer ${id} to table $table")
val existingTiles =
if(schemaHasChanged[K, V](writerSchema)) {
logger.warn(s"RDD schema has changed, this requires rewriting the entire layer.")
layerReader
.read[K, V, M](id)

} else {
val query =
new LayerQuery[K, M]
.where(Intersects(rdd.metadata.getComponent[Bounds[K]].get))

layerReader.read[K, V, M](id, query, layerReader.defaultNumPartitions, filterIndexOnly = true)
}

val updatedMetadata: M =
metadata.merge(rdd.metadata)

val updatedRdd: RDD[(K, V)] =
mergeFunc match {
case Some(mergeFunc) =>
existingTiles
.fullOuterJoin(rdd)
.flatMapValues {
case (Some(layerTile), Some(updateTile)) => Some(mergeFunc(layerTile, updateTile))
case (Some(layerTile), _) => Some(layerTile)
case (_, Some(updateTile)) => Some(updateTile)
case _ => None
}
case None => rdd
}

val codec = KeyValueRecordCodec[K, V]
val schema = codec.schema

// Write updated metadata, and the possibly updated schema
// Only really need to write the metadata and schema
attributeStore.writeLayerAttributes(id, header, updatedMetadata, keyIndex, schema)
CassandraRDDWriter.write(updatedRdd, instance, id, encodeKey, keyspace, table)
](id: LayerId, rdd: RDD[(K, V)] with Metadata[M]): Unit = {
val CassandraLayerHeader(_, _, keyspace, table) = attributeStore.readHeader[CassandraLayerHeader](id)
val layerWriter = new CassandraLayerWriter(attributeStore, instance, keyspace, table)
layerWriter.overwrite(id, rdd)
}

}

object CassandraLayerUpdater {
Expand Down
Expand Up @@ -21,9 +21,12 @@ import geotrellis.spark.io._
import geotrellis.spark.io.avro._
import geotrellis.spark.io.avro.codecs._
import geotrellis.spark.io.index._
import geotrellis.spark.merge._
import geotrellis.util._

import org.apache.spark.rdd.RDD
import org.apache.spark.SparkContext

import spray.json._

import scala.reflect._
Expand All @@ -33,8 +36,106 @@ class CassandraLayerWriter(
instance: CassandraInstance,
keyspace: String,
table: String
) extends LayerWriter[LayerId] {
) extends LayerWriter[LayerId] with LazyLogging {

// Layer updating
protected def _overwrite[
K: AvroRecordCodec: Boundable: JsonFormat: ClassTag,
V: AvroRecordCodec: ClassTag,
M: JsonFormat: GetComponent[?, Bounds[K]]: Mergable
](
sc: SparkContext,
id: LayerId,
rdd: RDD[(K, V)] with Metadata[M],
keyBounds: KeyBounds[K]
): Unit = {
_update(sc, id, rdd, keyBounds, None)
}

protected def _update[
K: AvroRecordCodec: Boundable: JsonFormat: ClassTag,
V: AvroRecordCodec: ClassTag,
M: JsonFormat: GetComponent[?, Bounds[K]]: Mergable
](
sc: SparkContext,
id: LayerId,
rdd: RDD[(K, V)] with Metadata[M],
keyBounds: KeyBounds[K],
mergeFunc: (V, V) => V
): Unit = {
_update(sc, id, rdd, keyBounds, Some(mergeFunc))
}

def _update[
K: AvroRecordCodec: Boundable: JsonFormat: ClassTag,
V: AvroRecordCodec: ClassTag,
M: JsonFormat: GetComponent[?, Bounds[K]]: Mergable
](
sc: SparkContext,
id: LayerId,
rdd: RDD[(K, V)] with Metadata[M],
keyBounds: KeyBounds[K],
mergeFunc: Option[(V, V) => V]
) = {
if (!attributeStore.layerExists(id)) throw new LayerNotFoundError(id)

val LayerAttributes(header, metadata, keyIndex, writerSchema) = try {
attributeStore.readLayerAttributes[CassandraLayerHeader, M, K](id)
} catch {
case e: AttributeNotFoundError => throw new LayerUpdateError(id).initCause(e)
}

val (keyspace, table) = header.keyspace -> header.tileTable

if (!(keyIndex.keyBounds contains keyBounds))
throw new LayerOutOfKeyBoundsError(id, keyIndex.keyBounds)

val encodeKey = (key: K) => keyIndex.toIndex(key)
implicit val sparkContext = sc
val layerReader = new CassandraLayerReader(attributeStore, instance)

logger.info(s"Saving updated RDD for layer ${id} to table $table")
val existingTiles =
if(schemaHasChanged[K, V](writerSchema)) {
logger.warn(s"RDD schema has changed, this requires rewriting the entire layer.")
layerReader
.read[K, V, M](id)

} else {
val query =
new LayerQuery[K, M]
.where(Intersects(rdd.metadata.getComponent[Bounds[K]].get))

layerReader.read[K, V, M](id, query, layerReader.defaultNumPartitions, filterIndexOnly = true)
}

val updatedMetadata: M =
metadata.merge(rdd.metadata)

val updatedRdd: RDD[(K, V)] =
mergeFunc match {
case Some(mergeFunc) =>
existingTiles
.fullOuterJoin(rdd)
.flatMapValues {
case (Some(layerTile), Some(updateTile)) => Some(mergeFunc(layerTile, updateTile))
case (Some(layerTile), _) => Some(layerTile)
case (_, Some(updateTile)) => Some(updateTile)
case _ => None
}
case None => rdd
}

val codec = KeyValueRecordCodec[K, V]
val schema = codec.schema

// Write updated metadata, and the possibly updated schema
// Only really need to write the metadata and schema
attributeStore.writeLayerAttributes(id, header, updatedMetadata, keyIndex, schema)
CassandraRDDWriter.write(updatedRdd, instance, id, encodeKey, keyspace, table)
}

// Layer writing
protected def _write[
K: AvroRecordCodec: JsonFormat: ClassTag,
V: AvroRecordCodec: ClassTag,
Expand Down

0 comments on commit ec23c0d

Please sign in to comment.