Skip to content

Commit

Permalink
Register read, sequence, and slice schema records
Browse files Browse the repository at this point in the history
  • Loading branch information
heuermh authored and fnothaft committed Nov 15, 2016
1 parent c750830 commit c06143b
Show file tree
Hide file tree
Showing 7 changed files with 284 additions and 0 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
/**
* Licensed to Big Data Genomics (BDG) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The BDG licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.bdgenomics.adam.projections

import org.bdgenomics.formats.avro.Read

/**
* Enumeration of Read field names for predicates and projections.
*/
object ReadField extends FieldEnumeration(Read.SCHEMA$) {

val name, description, alphabet, sequence, length, qualityScores, qualityScoreVariant = SchemaValue
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
/**
* Licensed to Big Data Genomics (BDG) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The BDG licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.bdgenomics.adam.projections

import org.bdgenomics.formats.avro.Sequence

/**
* Enumeration of Sequence field names for predicates and projections.
*/
object SequenceField extends FieldEnumeration(Sequence.SCHEMA$) {

val name, description, alphabet, sequence, length = SchemaValue
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
/**
* Licensed to Big Data Genomics (BDG) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The BDG licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.bdgenomics.adam.projections

import org.bdgenomics.formats.avro.Slice

/**
* Enumeration of Slice field names for predicates and projections.
*/
object SliceField extends FieldEnumeration(Slice.SCHEMA$) {

val name, description, alphabet, sequence, start, end, strand, length = SchemaValue
}
Original file line number Diff line number Diff line change
Expand Up @@ -197,10 +197,16 @@ class ADAMKryoRegistrator extends KryoRegistrator {
new AvroSerializer[org.bdgenomics.formats.avro.NucleotideContigFragment])
kryo.register(classOf[org.bdgenomics.formats.avro.OntologyTerm],
new AvroSerializer[org.bdgenomics.formats.avro.OntologyTerm])
kryo.register(classOf[org.bdgenomics.formats.avro.Read],
new AvroSerializer[org.bdgenomics.formats.avro.Read])
kryo.register(classOf[org.bdgenomics.formats.avro.RecordGroupMetadata],
new AvroSerializer[org.bdgenomics.formats.avro.RecordGroupMetadata])
kryo.register(classOf[org.bdgenomics.formats.avro.Sample],
new AvroSerializer[org.bdgenomics.formats.avro.Sample])
kryo.register(classOf[org.bdgenomics.formats.avro.Sequence],
new AvroSerializer[org.bdgenomics.formats.avro.Sequence])
kryo.register(classOf[org.bdgenomics.formats.avro.Slice],
new AvroSerializer[org.bdgenomics.formats.avro.Slice])
kryo.register(classOf[org.bdgenomics.formats.avro.Strand])
kryo.register(classOf[org.bdgenomics.formats.avro.TranscriptEffect],
new AvroSerializer[org.bdgenomics.formats.avro.TranscriptEffect])
Expand Down Expand Up @@ -229,8 +235,11 @@ class ADAMKryoRegistrator extends KryoRegistrator {
kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.GenotypeAllele]])
kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.OntologyTerm]])
kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.NucleotideContigFragment]])
kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.Read]])
kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.RecordGroupMetadata]])
kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.Sample]])
kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.Sequence]])
kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.Slice]])
kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.TranscriptEffect]])
kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.Variant]])
kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.VariantAnnotation]])
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,66 @@
/**
* Licensed to Big Data Genomics (BDG) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The BDG licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.bdgenomics.adam.projections

import org.apache.spark.rdd.RDD
import org.bdgenomics.adam.projections.ReadField._
import org.bdgenomics.adam.rdd.ADAMContext._
import org.bdgenomics.adam.rdd.TestSaveArgs
import org.bdgenomics.adam.util.ADAMFunSuite
import org.bdgenomics.formats.avro.{
Alphabet,
QualityScoreVariant,
Read
}

class ReadFieldSuite extends ADAMFunSuite {

sparkTest("Use projection when reading parquet reads") {
val path = tmpFile("reads.parquet")
val rdd = sc.parallelize(Seq(Read.newBuilder()
.setName("read 1")
.setDescription("read 1")
.setAlphabet(Alphabet.DNA)
.setSequence("ACTG")
.setLength(4)
.setQualityScores("0123")
.setQualityScoreVariant(QualityScoreVariant.FASTQ_SANGER)
.build()))
rdd.saveAsParquet(TestSaveArgs(path))

val projection = Projection(
name,
description,
alphabet,
sequence,
length,
qualityScores,
qualityScoreVariant
)

val reads: RDD[Read] = sc.loadParquet(path, projection = Some(projection))
assert(reads.count() === 1)
assert(reads.first.getName === "read 1")
assert(reads.first.getDescription === "read 1")
assert(reads.first.getAlphabet === Alphabet.DNA)
assert(reads.first.getSequence === "ACTG")
assert(reads.first.getLength === 4)
assert(reads.first.getQualityScores === "0123")
assert(reads.first.getQualityScoreVariant === QualityScoreVariant.FASTQ_SANGER)
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
/**
* Licensed to Big Data Genomics (BDG) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The BDG licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.bdgenomics.adam.projections

import org.apache.spark.rdd.RDD
import org.bdgenomics.adam.projections.SequenceField._
import org.bdgenomics.adam.rdd.ADAMContext._
import org.bdgenomics.adam.rdd.TestSaveArgs
import org.bdgenomics.adam.util.ADAMFunSuite
import org.bdgenomics.formats.avro.{ Alphabet, Sequence }

class SequenceFieldSuite extends ADAMFunSuite {

sparkTest("Use projection when reading parquet sequences") {
val path = tmpFile("sequences.parquet")
val rdd = sc.parallelize(Seq(Sequence.newBuilder()
.setName("6")
.setDescription("Chromosome 6")
.setAlphabet(Alphabet.DNA)
.setSequence("ACTG")
.setLength(4)
.build()))
rdd.saveAsParquet(TestSaveArgs(path))

val projection = Projection(
name,
description,
alphabet,
sequence,
length
)

val sequences: RDD[Sequence] = sc.loadParquet(path, projection = Some(projection));
assert(sequences.count() === 1)
assert(sequences.first.getName === "6")
assert(sequences.first.getDescription === "Chromosome 6")
assert(sequences.first.getAlphabet === Alphabet.DNA)
assert(sequences.first.getSequence === "ACTG")
assert(sequences.first.getLength === 4)
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
/**
* Licensed to Big Data Genomics (BDG) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The BDG licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.bdgenomics.adam.projections

import org.apache.spark.rdd.RDD
import org.bdgenomics.adam.projections.SliceField._
import org.bdgenomics.adam.rdd.ADAMContext._
import org.bdgenomics.adam.rdd.TestSaveArgs
import org.bdgenomics.adam.util.ADAMFunSuite
import org.bdgenomics.formats.avro.{
Alphabet,
Slice,
Strand
}

class SliceFieldSuite extends ADAMFunSuite {

sparkTest("Use projection when reading parquet slices") {
val path = tmpFile("slices.parquet")
val rdd = sc.parallelize(Seq(Slice.newBuilder()
.setName("6")
.setDescription("Chromosome 6")
.setAlphabet(Alphabet.DNA)
.setSequence("ACTG")
.setStart(0)
.setEnd(4)
.setStrand(Strand.FORWARD)
.setLength(4)
.build()))
rdd.saveAsParquet(TestSaveArgs(path))

val projection = Projection(
name,
description,
alphabet,
sequence,
start,
end,
strand,
length
)

val slices: RDD[Slice] = sc.loadParquet(path, projection = Some(projection));
assert(slices.count() === 1)
assert(slices.first.getName === "6")
assert(slices.first.getDescription === "Chromosome 6")
assert(slices.first.getAlphabet === Alphabet.DNA)
assert(slices.first.getSequence === "ACTG")
assert(slices.first.getStart === 0)
assert(slices.first.getEnd === 4)
assert(slices.first.getStrand === Strand.FORWARD)
assert(slices.first.getLength === 4)
}
}

0 comments on commit c06143b

Please sign in to comment.