-
Notifications
You must be signed in to change notification settings - Fork 364
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Merge pull request #1606 from pomadchin/feature/collections-api
Collections API
- Loading branch information
Showing
68 changed files
with
1,141 additions
and
271 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -1,4 +1,7 @@ | ||
geotrellis.accumulo { | ||
catalog = "metadata" | ||
threads.rdd.write = 32 | ||
} | ||
threads { | ||
collection.read = default | ||
rdd.write = default | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
68 changes: 68 additions & 0 deletions
68
accumulo/src/main/scala/geotrellis/spark/io/accumulo/AccumuloCollectionReader.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,68 @@ | ||
package geotrellis.spark.io.accumulo | ||
|
||
import geotrellis.spark.io._ | ||
import geotrellis.spark.io.avro.codecs.KeyValueRecordCodec | ||
import geotrellis.spark.io.avro.{AvroEncoder, AvroRecordCodec} | ||
import geotrellis.spark.{Boundable, KeyBounds} | ||
|
||
import scalaz.std.vector._ | ||
import scalaz.concurrent.{Strategy, Task} | ||
import scalaz.stream.{Process, channel, nondeterminism, tee} | ||
import org.apache.accumulo.core.data.{Range => AccumuloRange} | ||
import org.apache.accumulo.core.security.Authorizations | ||
import org.apache.avro.Schema | ||
import org.apache.hadoop.io.Text | ||
import com.typesafe.config.ConfigFactory | ||
|
||
import scala.collection.JavaConversions._ | ||
import scala.reflect.ClassTag | ||
import java.util.concurrent.Executors | ||
|
||
object AccumuloCollectionReader { | ||
def read[K: Boundable: AvroRecordCodec: ClassTag, V: AvroRecordCodec: ClassTag]( | ||
table: String, | ||
columnFamily: Text, | ||
queryKeyBounds: Seq[KeyBounds[K]], | ||
decomposeBounds: KeyBounds[K] => Seq[AccumuloRange], | ||
filterIndexOnly: Boolean, | ||
writerSchema: Option[Schema] = None, | ||
threads: Int = ConfigFactory.load().getThreads("geotrellis.accumulo.threads.collection.read") | ||
)(implicit instance: AccumuloInstance): Seq[(K, V)] = { | ||
if(queryKeyBounds.isEmpty) return Seq.empty[(K, V)] | ||
|
||
val codec = KeyValueRecordCodec[K, V] | ||
val includeKey = (key: K) => queryKeyBounds.includeKey(key) | ||
|
||
val ranges = queryKeyBounds.flatMap(decomposeBounds).toIterator | ||
|
||
val pool = Executors.newFixedThreadPool(threads) | ||
|
||
val range: Process[Task, AccumuloRange] = Process.unfold(ranges) { iter => | ||
if (iter.hasNext) Some(iter.next(), iter) | ||
else None | ||
} | ||
|
||
val readChannel = channel.lift { (range: AccumuloRange) => Task { | ||
val scanner = instance.connector.createScanner(table, new Authorizations()) | ||
scanner.setRange(range) | ||
scanner.fetchColumnFamily(columnFamily) | ||
val result = scanner.iterator.map { case entry => | ||
AvroEncoder.fromBinary(writerSchema.getOrElse(codec.schema), entry.getValue.get)(codec) | ||
}.flatMap { pairs: Vector[(K, V)] => | ||
if(filterIndexOnly) pairs | ||
else pairs.filter { pair => includeKey(pair._1) } | ||
}.toVector | ||
scanner.close() | ||
result | ||
}(pool) } | ||
|
||
val read = range.tee(readChannel)(tee.zipApply).map(Process.eval) | ||
|
||
|
||
try { | ||
nondeterminism | ||
.njoin(maxOpen = threads, maxQueued = threads) { read }(Strategy.Executor(pool)) | ||
.runFoldMap(identity).unsafePerformSync: Seq[(K, V)] | ||
} finally pool.shutdown() | ||
} | ||
} |
47 changes: 47 additions & 0 deletions
47
accumulo/src/main/scala/geotrellis/spark/io/accumulo/AccumuloLayerCollectionReader.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,47 @@ | ||
package geotrellis.spark.io.accumulo | ||
|
||
import geotrellis.spark._ | ||
import geotrellis.spark.io._ | ||
import geotrellis.spark.io.avro._ | ||
import geotrellis.util._ | ||
|
||
import org.apache.accumulo.core.data.{Range => AccumuloRange} | ||
import org.apache.hadoop.io.Text | ||
import spray.json._ | ||
|
||
import scala.reflect._ | ||
|
||
class AccumuloLayerCollectionReader(val attributeStore: AttributeStore)(implicit instance: AccumuloInstance) extends CollectionLayerReader[LayerId] { | ||
|
||
def read[ | ||
K: AvroRecordCodec: Boundable: JsonFormat: ClassTag, | ||
V: AvroRecordCodec: ClassTag, | ||
M: JsonFormat: GetComponent[?, Bounds[K]] | ||
](id: LayerId, rasterQuery: LayerQuery[K, M], filterIndexOnly: Boolean) = { | ||
if (!attributeStore.layerExists(id)) throw new LayerNotFoundError(id) | ||
|
||
val LayerAttributes(header, metadata, keyIndex, writerSchema) = try { | ||
attributeStore.readLayerAttributes[AccumuloLayerHeader, M, K](id) | ||
} catch { | ||
case e: AttributeNotFoundError => throw new LayerReadError(id).initCause(e) | ||
} | ||
|
||
val queryKeyBounds = rasterQuery(metadata) | ||
|
||
val decompose = (bounds: KeyBounds[K]) => | ||
keyIndex.indexRanges(bounds).map { case (min, max) => | ||
new AccumuloRange(new Text(AccumuloKeyEncoder.long2Bytes(min)), new Text(AccumuloKeyEncoder.long2Bytes(max))) | ||
} | ||
|
||
val seq = AccumuloCollectionReader.read[K, V](header.tileTable, columnFamily(id), queryKeyBounds, decompose, filterIndexOnly, Some(writerSchema)) | ||
new ContextCollection(seq, metadata) | ||
} | ||
} | ||
|
||
object AccumuloLayerCollectionReader { | ||
def apply(attributeStore: AccumuloAttributeStore)(implicit instance: AccumuloInstance): AccumuloLayerCollectionReader = | ||
new AccumuloLayerCollectionReader(attributeStore) | ||
|
||
def apply(implicit instance: AccumuloInstance): AccumuloLayerCollectionReader = | ||
new AccumuloLayerCollectionReader(AccumuloAttributeStore(instance.connector)) | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
62 changes: 62 additions & 0 deletions
62
cassandra/src/main/scala/geotrellis/spark/io/cassandra/CassandraCollectionReader.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,62 @@ | ||
package geotrellis.spark.io.cassandra | ||
|
||
import geotrellis.spark.{Boundable, KeyBounds, LayerId} | ||
import geotrellis.spark.io._ | ||
import geotrellis.spark.io.avro.codecs.KeyValueRecordCodec | ||
import geotrellis.spark.io.avro.{AvroEncoder, AvroRecordCodec} | ||
import geotrellis.spark.io.index.MergeQueue | ||
import geotrellis.spark.util.KryoWrapper | ||
|
||
import org.apache.avro.Schema | ||
import com.datastax.driver.core.querybuilder.QueryBuilder | ||
import com.datastax.driver.core.querybuilder.QueryBuilder.{eq => eqs} | ||
import com.typesafe.config.ConfigFactory | ||
|
||
import scala.collection.JavaConversions._ | ||
import scala.reflect.ClassTag | ||
|
||
object CassandraCollectionReader { | ||
def read[K: Boundable : AvroRecordCodec : ClassTag, V: AvroRecordCodec : ClassTag]( | ||
instance: CassandraInstance, | ||
keyspace: String, | ||
table: String, | ||
layerId: LayerId, | ||
queryKeyBounds: Seq[KeyBounds[K]], | ||
decomposeBounds: KeyBounds[K] => Seq[(Long, Long)], | ||
filterIndexOnly: Boolean, | ||
writerSchema: Option[Schema] = None, | ||
threads: Int = ConfigFactory.load().getThreads("geotrellis.cassandra.threads.collection.read") | ||
): Seq[(K, V)] = { | ||
if (queryKeyBounds.isEmpty) return Seq.empty[(K, V)] | ||
|
||
val includeKey = (key: K) => queryKeyBounds.includeKey(key) | ||
val _recordCodec = KeyValueRecordCodec[K, V] | ||
val kwWriterSchema = KryoWrapper(writerSchema) //Avro Schema is not Serializable | ||
|
||
val ranges = if (queryKeyBounds.length > 1) | ||
MergeQueue(queryKeyBounds.flatMap(decomposeBounds)) | ||
else | ||
queryKeyBounds.flatMap(decomposeBounds) | ||
|
||
val query = QueryBuilder.select("value") | ||
.from(keyspace, table) | ||
.where(eqs("key", QueryBuilder.bindMarker())) | ||
.and(eqs("name", layerId.name)) | ||
.and(eqs("zoom", layerId.zoom)) | ||
.toString | ||
|
||
instance.withSessionDo { session => | ||
val statement = session.prepare(query) | ||
|
||
LayerReader.njoin[K, V](ranges.toIterator, threads){ index: Long => | ||
val row = session.execute(statement.bind(index.asInstanceOf[java.lang.Long])) | ||
if (row.nonEmpty) { | ||
val bytes = row.one().getBytes("value").array() | ||
val recs = AvroEncoder.fromBinary(kwWriterSchema.value.getOrElse(_recordCodec.schema), bytes)(_recordCodec) | ||
if (filterIndexOnly) recs | ||
else recs.filter { row => includeKey(row._1) } | ||
} else Vector.empty | ||
} | ||
}: Seq[(K, V)] | ||
} | ||
} |
42 changes: 42 additions & 0 deletions
42
cassandra/src/main/scala/geotrellis/spark/io/cassandra/CassandraLayerCollectionReader.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,42 @@ | ||
package geotrellis.spark.io.cassandra | ||
|
||
import geotrellis.spark._ | ||
import geotrellis.spark.io._ | ||
import geotrellis.spark.io.avro._ | ||
import geotrellis.util._ | ||
|
||
import spray.json._ | ||
|
||
import scala.reflect._ | ||
|
||
class CassandraLayerCollectionReader(val attributeStore: AttributeStore, instance: CassandraInstance) extends CollectionLayerReader[LayerId] { | ||
|
||
def read[ | ||
K: AvroRecordCodec: Boundable: JsonFormat: ClassTag, | ||
V: AvroRecordCodec: ClassTag, | ||
M: JsonFormat: GetComponent[?, Bounds[K]] | ||
](id: LayerId, rasterQuery: LayerQuery[K, M], filterIndexOnly: Boolean) = { | ||
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 LayerReadError(id).initCause(e) | ||
} | ||
|
||
val queryKeyBounds = rasterQuery(metadata) | ||
|
||
val decompose = (bounds: KeyBounds[K]) => keyIndex.indexRanges(bounds) | ||
|
||
val seq = CassandraCollectionReader.read[K, V](instance, header.keyspace, header.tileTable, id, queryKeyBounds, decompose, filterIndexOnly, Some(writerSchema)) | ||
new ContextCollection(seq, metadata) | ||
} | ||
} | ||
|
||
object CassandraLayerCollectionReader { | ||
def apply(instance: CassandraInstance): CassandraLayerCollectionReader = | ||
new CassandraLayerCollectionReader(CassandraAttributeStore(instance), instance) | ||
|
||
def apply(attributeStore: CassandraAttributeStore): CassandraLayerCollectionReader = | ||
new CassandraLayerCollectionReader(attributeStore, attributeStore.instance) | ||
} |
Oops, something went wrong.