From 2cc7abd51aec5592f456470469d6aa89af73fdc3 Mon Sep 17 00:00:00 2001 From: jamesemery Date: Tue, 22 May 2018 15:45:01 -0400 Subject: [PATCH] Updated MarkDuplicates to use Picard metrics code (#4779) --- .../markduplicates/MarkDuplicatesSpark.java | 25 +-- .../MarkDuplicatesSparkUtils.java | 50 ++---- .../EstimateLibraryComplexityGATK.java | 8 +- .../markduplicates/MarkDuplicatesGATK.java | 53 ++---- .../hellbender/utils/read/ReadUtils.java | 12 +- ...tractMarkDuplicatesCommandLineProgram.java | 10 +- .../markduplicates/DuplicationMetrics.java | 156 ------------------ .../GATKDuplicationMetrics.java | 65 ++++++++ .../markduplicates/LibraryIdGenerator.java | 27 ++- .../ReadEndsForMarkDuplicatesCodec.java | 2 +- .../utils/test/testers/SamFileTester.java | 99 ++++++++--- .../MarkDuplicatesSparkIntegrationTest.java | 16 +- ...tMarkDuplicatesCommandLineProgramTest.java | 116 ++++++++++--- .../testers/AbstractMarkDuplicatesTester.java | 41 ++--- ...r1.1-1K.unmarkedDups.markDuplicate.metrics | 33 ---- ...pected.optical_dupes.markDuplicate.metrics | 10 -- ...r1.1-1K.unmarkedDups.markDuplicate.metrics | 33 ++++ ...ted.inputSingleLibrarySolexa16404.metrics} | 8 +- ...sorted.optical_dupes.markDuplicate.metrics | 10 ++ 19 files changed, 401 insertions(+), 373 deletions(-) delete mode 100644 src/main/java/org/broadinstitute/hellbender/utils/read/markduplicates/DuplicationMetrics.java create mode 100644 src/main/java/org/broadinstitute/hellbender/utils/read/markduplicates/GATKDuplicationMetrics.java rename src/{main => test}/java/org/broadinstitute/hellbender/utils/test/testers/AbstractMarkDuplicatesTester.java (86%) delete mode 100644 src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/expected.chr1.1-1K.unmarkedDups.markDuplicate.metrics delete mode 100644 src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/expected.optical_dupes.markDuplicate.metrics create mode 100644 src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/expected.picard-2.15.0.sorted.chr1.1-1K.unmarkedDups.markDuplicate.metrics rename src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/{expected.inputSingleLibrarySolexa16404.metrics => expected.picard-2.15.0.sorted.inputSingleLibrarySolexa16404.metrics} (55%) create mode 100644 src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/expected.picard-2.15.0.sorted.optical_dupes.markDuplicate.metrics diff --git a/src/main/java/org/broadinstitute/hellbender/tools/spark/transforms/markduplicates/MarkDuplicatesSpark.java b/src/main/java/org/broadinstitute/hellbender/tools/spark/transforms/markduplicates/MarkDuplicatesSpark.java index 11939b250a5..01ce5caf5d5 100644 --- a/src/main/java/org/broadinstitute/hellbender/tools/spark/transforms/markduplicates/MarkDuplicatesSpark.java +++ b/src/main/java/org/broadinstitute/hellbender/tools/spark/transforms/markduplicates/MarkDuplicatesSpark.java @@ -17,19 +17,17 @@ import org.broadinstitute.hellbender.engine.filters.ReadFilter; import org.broadinstitute.hellbender.engine.filters.ReadFilterLibrary; import org.broadinstitute.hellbender.engine.spark.GATKSparkTool; -import org.broadinstitute.hellbender.engine.spark.datasources.ReadsSparkSource; import org.broadinstitute.hellbender.exceptions.GATKException; import org.broadinstitute.hellbender.utils.Utils; import org.broadinstitute.hellbender.utils.read.GATKRead; import org.broadinstitute.hellbender.utils.read.ReadUtils; import org.broadinstitute.hellbender.utils.read.SAMRecordToGATKReadAdapter; -import org.broadinstitute.hellbender.utils.read.markduplicates.DuplicationMetrics; +import org.broadinstitute.hellbender.utils.read.markduplicates.GATKDuplicationMetrics; import org.broadinstitute.hellbender.utils.read.markduplicates.MarkDuplicatesScoringStrategy; import org.broadinstitute.hellbender.utils.read.markduplicates.SerializableOpticalDuplicatesFinder; -import picard.sam.markduplicates.util.OpticalDuplicateFinder; import org.broadinstitute.hellbender.utils.spark.SparkUtils; -import picard.sam.markduplicates.util.OpticalDuplicateFinder; import picard.cmdline.programgroups.ReadDataManipulationProgramGroup; +import picard.sam.markduplicates.util.OpticalDuplicateFinder; import scala.Tuple2; import java.util.Collections; @@ -91,6 +89,7 @@ public static JavaRDD mark(final JavaRDD reads, final SAMFil final MarkDuplicatesScoringStrategy scoringStrategy, final OpticalDuplicateFinder opticalDuplicateFinder, final int numReducers, final boolean dontMarkUnmappedMates) { + final boolean markUnmappedMates = !dontMarkUnmappedMates; SAMFileHeader headerForTool = header.clone(); // If the input isn't queryname sorted, sort it before duplicate marking @@ -107,11 +106,13 @@ public static JavaRDD mark(final JavaRDD reads, final SAMFil // Here we combine the original bam with the repartitioned unmarked readnames to produce our marked reads return sortedReadsForMarking.zipPartitions(repartitionedReadNames, (readsIter, readNamesIter) -> { final Map namesOfNonDuplicateReadsAndOpticalCounts = Utils.stream(readNamesIter).collect(Collectors.toMap(Tuple2::_1,Tuple2::_2, (t1,t2) -> {throw new GATKException("Detected multiple mark duplicate records objects corresponding to read with name, this could be the result of readnames spanning more than one partition");})); - return Utils.stream(readsIter).peek(read -> { + return Utils.stream(readsIter) + .peek(read -> read.setIsDuplicate(false)) + .peek(read -> { // Handle reads that have been marked as non-duplicates (which also get tagged with optical duplicate summary statistics) - if( namesOfNonDuplicateReadsAndOpticalCounts.containsKey(read.getName())) { + if (namesOfNonDuplicateReadsAndOpticalCounts.containsKey(read.getName())) { read.setIsDuplicate(false); - if (!(dontMarkUnmappedMates && read.isUnmapped())) { + if (markUnmappedMates || !read.isUnmapped()) { int dupCount = namesOfNonDuplicateReadsAndOpticalCounts.replace(read.getName(), -1); if (dupCount > -1) { ((SAMRecordToGATKReadAdapter) read).setTransientAttribute(MarkDuplicatesSparkUtils.OPTICAL_DUPLICATE_TOTAL_ATTRIBUTE_NAME, dupCount); @@ -121,9 +122,11 @@ public static JavaRDD mark(final JavaRDD reads, final SAMFil } else if (ReadUtils.readAndMateAreUnmapped(read)) { read.setIsDuplicate(false); // Everything else is a duplicate - } else{ - if (!(dontMarkUnmappedMates && read.isUnmapped())) { + } else { + if (markUnmappedMates || !read.isUnmapped()) { read.setIsDuplicate(true); + } else { + read.setIsDuplicate(false); } } }).iterator(); @@ -182,9 +185,9 @@ protected void runTool(final JavaSparkContext ctx) { final JavaRDD finalReadsForMetrics = mark(reads, header, markDuplicatesSparkArgumentCollection.duplicatesScoringStrategy, finder, getRecommendedNumReducers(), markDuplicatesSparkArgumentCollection.dontMarkUnmappedMates); if (metricsFile != null) { - final JavaPairRDD metricsByLibrary = MarkDuplicatesSparkUtils.generateMetrics( + final JavaPairRDD metricsByLibrary = MarkDuplicatesSparkUtils.generateMetrics( header, finalReadsForMetrics); - final MetricsFile resultMetrics = getMetricsFile(); + final MetricsFile resultMetrics = getMetricsFile(); MarkDuplicatesSparkUtils.saveMetricsRDD(resultMetrics, header, metricsByLibrary, metricsFile); } header.setSortOrder(SAMFileHeader.SortOrder.coordinate); diff --git a/src/main/java/org/broadinstitute/hellbender/tools/spark/transforms/markduplicates/MarkDuplicatesSparkUtils.java b/src/main/java/org/broadinstitute/hellbender/tools/spark/transforms/markduplicates/MarkDuplicatesSparkUtils.java index a475cf9a632..77eb3a298f2 100644 --- a/src/main/java/org/broadinstitute/hellbender/tools/spark/transforms/markduplicates/MarkDuplicatesSparkUtils.java +++ b/src/main/java/org/broadinstitute/hellbender/tools/spark/transforms/markduplicates/MarkDuplicatesSparkUtils.java @@ -408,27 +408,12 @@ private static Tuple2, Integer> handleFragments(List generateMetrics(final SAMFileHeader header, final JavaRDD reads) { - return reads.filter(read -> !read.isSecondaryAlignment() && !read.isSupplementaryAlignment()) - .mapToPair(read -> { + static JavaPairRDD generateMetrics(final SAMFileHeader header, final JavaRDD reads) { + return reads.mapToPair(read -> { final String library = LibraryIdGenerator.getLibraryName(header, read.getReadGroup()); - DuplicationMetrics metrics = new DuplicationMetrics(); + GATKDuplicationMetrics metrics = new GATKDuplicationMetrics(); metrics.LIBRARY = library; - if (read.isUnmapped()) { - ++metrics.UNMAPPED_READS; - } else if (!read.isPaired() || read.mateIsUnmapped()) { - ++metrics.UNPAIRED_READS_EXAMINED; - } else { - ++metrics.READ_PAIRS_EXAMINED; - } - - if (read.isDuplicate()) { - if (!read.isPaired() || read.mateIsUnmapped()) { - ++metrics.UNPAIRED_READ_DUPLICATES; - } else { - ++metrics.READ_PAIR_DUPLICATES; - } - } + metrics.updateMetrics(read); // NOTE: we use the SAMRecord transientAttribute field here specifically to prevent the already // serialized read from being parsed again here for performance reasons. if (((SAMRecordToGATKReadAdapter) read).getTransientAttribute(OPTICAL_DUPLICATE_TOTAL_ATTRIBUTE_NAME)!=null) { @@ -438,31 +423,22 @@ static JavaPairRDD generateMetrics(final SAMFileHead } return new Tuple2<>(library, metrics); }) - .foldByKey(new DuplicationMetrics(), (metricsSum, m) -> { - if (metricsSum.LIBRARY == null) { - metricsSum.LIBRARY = m.LIBRARY; - } - // This should never happen, as we grouped by key using library as the key. + .foldByKey(new GATKDuplicationMetrics(), (metricsSum, m) -> { + metricsSum.merge(m); if (!metricsSum.LIBRARY.equals(m.LIBRARY)) { throw new GATKException("Two different libraries encountered while summing metrics: " + metricsSum.LIBRARY + " and " + m.LIBRARY); } - metricsSum.UNMAPPED_READS += m.UNMAPPED_READS; - metricsSum.UNPAIRED_READS_EXAMINED += m.UNPAIRED_READS_EXAMINED; - metricsSum.READ_PAIRS_EXAMINED += m.READ_PAIRS_EXAMINED; - metricsSum.UNPAIRED_READ_DUPLICATES += m.UNPAIRED_READ_DUPLICATES; - metricsSum.READ_PAIR_DUPLICATES += m.READ_PAIR_DUPLICATES; - metricsSum.READ_PAIR_OPTICAL_DUPLICATES += m.READ_PAIR_OPTICAL_DUPLICATES; return metricsSum; }) .mapValues(metrics -> { - DuplicationMetrics copy = metrics.copy(); + final GATKDuplicationMetrics copy = metrics.copy(); // Divide these by 2 because they are counted for each read // when they should be counted by pair. copy.READ_PAIRS_EXAMINED = metrics.READ_PAIRS_EXAMINED / 2; copy.READ_PAIR_DUPLICATES = metrics.READ_PAIR_DUPLICATES / 2; - copy.calculateDerivedMetrics(); + copy.calculateDerivedFields(); if (copy.ESTIMATED_LIBRARY_SIZE == null) { copy.ESTIMATED_LIBRARY_SIZE = 0L; } @@ -475,19 +451,19 @@ static JavaPairRDD generateMetrics(final SAMFileHead * Note: the SamFileHeader is needed in order to include libraries that didn't have any duplicates. * @param result metrics object, potentially pre-initialized with headers, */ - public static void saveMetricsRDD(final MetricsFile result, final SAMFileHeader header, final JavaPairRDD metricsRDD, final String metricsOutputPath) { + public static void saveMetricsRDD(final MetricsFile result, final SAMFileHeader header, final JavaPairRDD metricsRDD, final String metricsOutputPath) { final LibraryIdGenerator libraryIdGenerator = new LibraryIdGenerator(header); - final Map nonEmptyMetricsByLibrary = metricsRDD.collectAsMap(); //Unknown Library - final Map emptyMapByLibrary = libraryIdGenerator.getMetricsByLibraryMap();//with null + final Map nonEmptyMetricsByLibrary = metricsRDD.collectAsMap(); //Unknown Library + final Map emptyMapByLibrary = libraryIdGenerator.getMetricsByLibraryMap();//with null final List sortedListOfLibraryNames = new ArrayList<>(Sets.union(emptyMapByLibrary.keySet(), nonEmptyMetricsByLibrary.keySet())); sortedListOfLibraryNames.sort(Utils.COMPARE_STRINGS_NULLS_FIRST); for (final String library : sortedListOfLibraryNames) { //if a non-empty exists, take it, otherwise take from the the empties. This is done to include libraries with zero data in them. //But not all libraries are listed in the header (esp in testing data) so we union empty and non-empty - final DuplicationMetrics metricsToAdd = nonEmptyMetricsByLibrary.containsKey(library) ? nonEmptyMetricsByLibrary.get(library) : emptyMapByLibrary.get(library); - metricsToAdd.calculateDerivedMetrics(); + final GATKDuplicationMetrics metricsToAdd = nonEmptyMetricsByLibrary.containsKey(library) ? nonEmptyMetricsByLibrary.get(library) : emptyMapByLibrary.get(library); + metricsToAdd.calculateDerivedFields(); result.addMetric(metricsToAdd); } diff --git a/src/main/java/org/broadinstitute/hellbender/tools/walkers/markduplicates/EstimateLibraryComplexityGATK.java b/src/main/java/org/broadinstitute/hellbender/tools/walkers/markduplicates/EstimateLibraryComplexityGATK.java index 52d7a698e08..df6ce758373 100644 --- a/src/main/java/org/broadinstitute/hellbender/tools/walkers/markduplicates/EstimateLibraryComplexityGATK.java +++ b/src/main/java/org/broadinstitute/hellbender/tools/walkers/markduplicates/EstimateLibraryComplexityGATK.java @@ -20,7 +20,7 @@ import picard.cmdline.programgroups.DiagnosticsAndQCProgramGroup; import org.broadinstitute.hellbender.exceptions.GATKException; import org.broadinstitute.hellbender.utils.read.markduplicates.AbstractOpticalDuplicateFinderCommandLineProgram; -import org.broadinstitute.hellbender.utils.read.markduplicates.DuplicationMetrics; +import org.broadinstitute.hellbender.utils.read.markduplicates.GATKDuplicationMetrics; import picard.sam.markduplicates.util.OpticalDuplicateFinder; import org.broadinstitute.hellbender.utils.runtime.ProgressLogger; import picard.sam.util.PhysicalLocation; @@ -437,11 +437,11 @@ protected Object doWork() { } sorter.cleanup(); - final MetricsFile file = getMetricsFile(); + final MetricsFile file = getMetricsFile(); for (final String library : duplicationHistosByLibrary.keySet()) { final Histogram duplicationHisto = duplicationHistosByLibrary.get(library); final Histogram opticalHisto = opticalHistosByLibrary.get(library); - final DuplicationMetrics metrics = new DuplicationMetrics(); + final GATKDuplicationMetrics metrics = new GATKDuplicationMetrics(); metrics.LIBRARY = library; // Filter out any bins that have only a single entry in them and calcu @@ -456,7 +456,7 @@ protected Object doWork() { } } - metrics.calculateDerivedMetrics(); + metrics.calculateDerivedFields(); file.addMetric(metrics); file.addHistogram(duplicationHisto); diff --git a/src/main/java/org/broadinstitute/hellbender/tools/walkers/markduplicates/MarkDuplicatesGATK.java b/src/main/java/org/broadinstitute/hellbender/tools/walkers/markduplicates/MarkDuplicatesGATK.java index 4e616a57a36..5b98939b1f9 100644 --- a/src/main/java/org/broadinstitute/hellbender/tools/walkers/markduplicates/MarkDuplicatesGATK.java +++ b/src/main/java/org/broadinstitute/hellbender/tools/walkers/markduplicates/MarkDuplicatesGATK.java @@ -118,46 +118,29 @@ protected Object doWork() { try (final CloseableIterator iterator = headerAndIterator.iterator) { while (iterator.hasNext()) { final SAMRecord rec = iterator.next(); - if (!rec.isSecondaryOrSupplementary()) { - final String library = LibraryIdGenerator.getLibraryName(header, rec); - DuplicationMetrics metrics = libraryIdGenerator.getMetricsByLibrary(library); - if (metrics == null) { - metrics = new DuplicationMetrics(); - metrics.LIBRARY = library; - libraryIdGenerator.addMetricsByLibrary(library, metrics); - } + final String library = LibraryIdGenerator.getLibraryName(header, rec); + GATKDuplicationMetrics metrics = libraryIdGenerator.getMetricsByLibrary(library); + if (metrics == null) { + metrics = new GATKDuplicationMetrics(); + metrics.LIBRARY = library; + libraryIdGenerator.addMetricsByLibrary(library, metrics); + } - // First bring the simple metrics up to date - if (rec.getReadUnmappedFlag()) { - ++metrics.UNMAPPED_READS; - } else if (!rec.getReadPairedFlag() || rec.getMateUnmappedFlag()) { - ++metrics.UNPAIRED_READS_EXAMINED; + if (recordInFileIndex == nextDuplicateIndex) { + rec.setDuplicateReadFlag(true); + // Now try and figure out the next duplicate index + if (this.duplicateIndexes.hasNext()) { + nextDuplicateIndex = this.duplicateIndexes.next(); } else { - ++metrics.READ_PAIRS_EXAMINED; // will need to be divided by 2 at the end + // Only happens once we've marked all the duplicates + nextDuplicateIndex = -1; } + } else { + rec.setDuplicateReadFlag(false); + } + metrics.updateMetrics(rec); - if (recordInFileIndex == nextDuplicateIndex) { - rec.setDuplicateReadFlag(true); - - // Update the duplication metrics - if (!rec.getReadPairedFlag() || rec.getMateUnmappedFlag()) { - ++metrics.UNPAIRED_READ_DUPLICATES; - } else { - ++metrics.READ_PAIR_DUPLICATES;// will need to be divided by 2 at the end - } - - // Now try and figure out the next duplicate index - if (this.duplicateIndexes.hasNext()) { - nextDuplicateIndex = this.duplicateIndexes.next(); - } else { - // Only happens once we've marked all the duplicates - nextDuplicateIndex = -1; - } - } else { - rec.setDuplicateReadFlag(false); - } - } recordInFileIndex++; if (!this.REMOVE_DUPLICATES || !rec.getDuplicateReadFlag()) { diff --git a/src/main/java/org/broadinstitute/hellbender/utils/read/ReadUtils.java b/src/main/java/org/broadinstitute/hellbender/utils/read/ReadUtils.java index d29d37a195c..b0172d528d9 100644 --- a/src/main/java/org/broadinstitute/hellbender/utils/read/ReadUtils.java +++ b/src/main/java/org/broadinstitute/hellbender/utils/read/ReadUtils.java @@ -313,13 +313,21 @@ public static String prettyPrintSequenceRecords( final SAMSequenceDictionary seq } /** - * @param read read to check - * @return true if the read is paired and has a mapped mate, otherwise false + * @param read read to query + * @return true if the read has a mate and that mate is mapped, otherwise false */ public static boolean readHasMappedMate( final GATKRead read ) { return read.isPaired() && ! read.mateIsUnmapped(); } + /** + * @param read read to query + * @return true if the read has a mate and that mate is mapped, otherwise false + */ + public static boolean readHasMappedMate( final SAMRecord read ) { + return read.getReadPairedFlag() && ! read.getMateUnmappedFlag(); + } + /** * Check whether the given String represents a legal attribute name according to the SAM spec, * and throw an exception if it doesn't. diff --git a/src/main/java/org/broadinstitute/hellbender/utils/read/markduplicates/AbstractMarkDuplicatesCommandLineProgram.java b/src/main/java/org/broadinstitute/hellbender/utils/read/markduplicates/AbstractMarkDuplicatesCommandLineProgram.java index 4f386aa7e71..65192d3b6b1 100644 --- a/src/main/java/org/broadinstitute/hellbender/utils/read/markduplicates/AbstractMarkDuplicatesCommandLineProgram.java +++ b/src/main/java/org/broadinstitute/hellbender/utils/read/markduplicates/AbstractMarkDuplicatesCommandLineProgram.java @@ -117,17 +117,17 @@ protected Map getChainedPgIds(final SAMFileHeader outputHeader) */ protected void finalizeAndWriteMetrics(final LibraryIdGenerator libraryIdGenerator) { //We want to sort libraries by name - final SortedMap metricsByLibrary = new TreeMap<>(Utils.COMPARE_STRINGS_NULLS_FIRST); + final SortedMap metricsByLibrary = new TreeMap<>(Utils.COMPARE_STRINGS_NULLS_FIRST); metricsByLibrary.putAll(libraryIdGenerator.getMetricsByLibraryMap()); final Histogram opticalDuplicatesByLibraryId = libraryIdGenerator.getOpticalDuplicatesByLibraryIdMap(); final Map libraryIds = libraryIdGenerator.getLibraryIdsMap(); // Write out the metrics - final MetricsFile file = getMetricsFile(); - for (final Map.Entry entry : metricsByLibrary.entrySet()) { + final MetricsFile file = getMetricsFile(); + for (final Map.Entry entry : metricsByLibrary.entrySet()) { final String libraryName = entry.getKey(); - final DuplicationMetrics metrics = entry.getValue(); + final GATKDuplicationMetrics metrics = entry.getValue(); metrics.READ_PAIRS_EXAMINED = metrics.READ_PAIRS_EXAMINED / 2; metrics.READ_PAIR_DUPLICATES = metrics.READ_PAIR_DUPLICATES / 2; @@ -141,7 +141,7 @@ protected void finalizeAndWriteMetrics(final LibraryIdGenerator libraryIdGenerat metrics.READ_PAIR_OPTICAL_DUPLICATES = (long) bin.getValue(); } } - metrics.calculateDerivedMetrics(); + metrics.calculateDerivedFields(); file.addMetric(metrics); } diff --git a/src/main/java/org/broadinstitute/hellbender/utils/read/markduplicates/DuplicationMetrics.java b/src/main/java/org/broadinstitute/hellbender/utils/read/markduplicates/DuplicationMetrics.java deleted file mode 100644 index 88fe24ca3cc..00000000000 --- a/src/main/java/org/broadinstitute/hellbender/utils/read/markduplicates/DuplicationMetrics.java +++ /dev/null @@ -1,156 +0,0 @@ -package org.broadinstitute.hellbender.utils.read.markduplicates; - -import htsjdk.samtools.metrics.MetricBase; -import htsjdk.samtools.util.Histogram; -import org.broadinstitute.hellbender.utils.Utils; - -import java.io.Serializable; - -/** - * Metrics that are calculated during the process of marking duplicates - * within a stream of SAMRecords. - */ -@SuppressWarnings("serial") -public final class DuplicationMetrics extends MetricBase implements Serializable { - /** The library on which the duplicate marking was performed. */ - public String LIBRARY; - - /** - * The number of mapped reads examined which did not have a mapped mate pair, - * either because the read is unpaired, or the read is paired to an unmapped mate. - */ - public long UNPAIRED_READS_EXAMINED; - - /** The number of mapped read pairs examined. */ - public long READ_PAIRS_EXAMINED; - - /** The total number of unmapped reads examined. */ - public long UNMAPPED_READS; - - /** The number of fragments that were marked as duplicates. */ - public long UNPAIRED_READ_DUPLICATES; - - /** The number of read pairs that were marked as duplicates. */ - public long READ_PAIR_DUPLICATES; - - /** - * The number of read pairs duplicates that were caused by optical duplication. - * Value is always < READ_PAIR_DUPLICATES, which counts all duplicates regardless of source. - */ - public long READ_PAIR_OPTICAL_DUPLICATES; - - /** The percentage of mapped sequence that is marked as duplicate. */ - public Double PERCENT_DUPLICATION; - - /** The estimated number of unique molecules in the library based on PE duplication. */ - public Long ESTIMATED_LIBRARY_SIZE; - - /** - * Fills in the ESTIMATED_LIBRARY_SIZE based on the paired read data examined where - * possible and the PERCENT_DUPLICATION. - */ - public void calculateDerivedMetrics() { - this.ESTIMATED_LIBRARY_SIZE = estimateLibrarySize(this.READ_PAIRS_EXAMINED - this.READ_PAIR_OPTICAL_DUPLICATES, - this.READ_PAIRS_EXAMINED - this.READ_PAIR_DUPLICATES); - - PERCENT_DUPLICATION = (UNPAIRED_READ_DUPLICATES + READ_PAIR_DUPLICATES *2) /(double) (UNPAIRED_READS_EXAMINED + READ_PAIRS_EXAMINED *2); - } - - /** - * Estimates the size of a library based on the number of paired end molecules observed - * and the number of unique pairs ovserved. - * - * Based on the Lander-Waterman equation that states: - * C/X = 1 - exp( -N/X ) - * where - * X = number of distinct molecules in library - * N = number of read pairs - * C = number of distinct fragments observed in read pairs - */ - public static Long estimateLibrarySize(final long readPairs, final long uniqueReadPairs) { - final long readPairDuplicates = readPairs - uniqueReadPairs; - - if (readPairs > 0 && readPairDuplicates > 0) { - long n = readPairs; - long c = uniqueReadPairs; - - double m = 1.0, M = 100.0; - - Utils.validate(c < n && f(m * c, c, n) >= 0, () -> "Invalid values for pairs and unique pairs: " + n + ", " + c); - - while( f(M*c, c, n) >= 0 ) M *= 10.0; - - for (int i=0; i<40; i++ ) { - double r = (m+M)/2.0; - double u = f( r * c, c, n ); - if ( u == 0 ) break; - else if ( u > 0 ) m = r; - else if ( u < 0 ) M = r; - } - - return (long) (c * (m+M)/2.0); - } - else { - return null; - } - } - - /** Method that is used in the computation of estimated library size. */ - private static double f(double x, double c, double n) { - return c/x - 1 + Math.exp(-n/x); - } - - /** - * Estimates the ROI (return on investment) that one would see if a library was sequenced to - * x higher coverage than the observed coverage. - * - * @param estimatedLibrarySize the estimated number of molecules in the library - * @param x the multiple of sequencing to be simulated (i.e. how many X sequencing) - * @param pairs the number of pairs observed in the actual sequencing - * @param uniquePairs the number of unique pairs observed in the actual sequencing - * @return a number z <= x that estimates if you had pairs*x as your sequencing then you - * would observe uniquePairs*z unique pairs. - */ - public static double estimateRoi(long estimatedLibrarySize, double x, long pairs, long uniquePairs) { - return estimatedLibrarySize * ( 1 - Math.exp(-(x*pairs)/estimatedLibrarySize) ) / uniquePairs; - } - - /** - * Calculates a histogram using the estimateRoi method to estimate the effective yield - * doing x sequencing for x=1..10. - */ - public Histogram calculateRoiHistogram() { - if (ESTIMATED_LIBRARY_SIZE == null) { - try { - calculateDerivedMetrics(); - if (ESTIMATED_LIBRARY_SIZE == null) return null; - } - catch (IllegalStateException ise) { return null; } - } - - long uniquePairs = READ_PAIRS_EXAMINED - READ_PAIR_DUPLICATES; - Histogram histo = new Histogram<>(); - - for (double x=1; x<=100; x+=1) { - histo.increment(x, estimateRoi(ESTIMATED_LIBRARY_SIZE, x, READ_PAIRS_EXAMINED, uniquePairs)); - } - - return histo; - } - - public DuplicationMetrics copy() { - final DuplicationMetrics copy = new DuplicationMetrics(); - copy.LIBRARY = this.LIBRARY; - copy.UNPAIRED_READS_EXAMINED = this.UNPAIRED_READS_EXAMINED; - copy.READ_PAIRS_EXAMINED = this.READ_PAIRS_EXAMINED; - copy.UNMAPPED_READS = this.UNMAPPED_READS; - copy.UNPAIRED_READ_DUPLICATES = this.UNPAIRED_READ_DUPLICATES; - copy.READ_PAIR_DUPLICATES = this.READ_PAIR_DUPLICATES; - copy.READ_PAIR_OPTICAL_DUPLICATES = this.READ_PAIR_OPTICAL_DUPLICATES; - copy.PERCENT_DUPLICATION = this.PERCENT_DUPLICATION; - copy.ESTIMATED_LIBRARY_SIZE = this.ESTIMATED_LIBRARY_SIZE; - - return copy; - } - -} diff --git a/src/main/java/org/broadinstitute/hellbender/utils/read/markduplicates/GATKDuplicationMetrics.java b/src/main/java/org/broadinstitute/hellbender/utils/read/markduplicates/GATKDuplicationMetrics.java new file mode 100644 index 00000000000..696757c4461 --- /dev/null +++ b/src/main/java/org/broadinstitute/hellbender/utils/read/markduplicates/GATKDuplicationMetrics.java @@ -0,0 +1,65 @@ +package org.broadinstitute.hellbender.utils.read.markduplicates; + +import htsjdk.samtools.SAMRecord; +import org.broadinstitute.hellbender.utils.read.GATKRead; +import org.broadinstitute.hellbender.utils.read.ReadUtils; +import picard.sam.DuplicationMetrics; + +import java.io.Serializable; + +/** + * Metrics that are calculated during the process of marking duplicates + * within a stream of SAMRecords. + */ +public final class GATKDuplicationMetrics extends DuplicationMetrics implements Serializable { + @NoMergingKeepsValue //Currently picard requires all fields to be annotated. + private static final long serialVersionUID = 1L; + + public GATKDuplicationMetrics copy() { + final GATKDuplicationMetrics copy = new GATKDuplicationMetrics(); + copy.LIBRARY = this.LIBRARY; + copy.UNPAIRED_READS_EXAMINED = this.UNPAIRED_READS_EXAMINED; + copy.READ_PAIRS_EXAMINED = this.READ_PAIRS_EXAMINED; + copy.SECONDARY_OR_SUPPLEMENTARY_RDS = this.SECONDARY_OR_SUPPLEMENTARY_RDS; + copy.UNMAPPED_READS = this.UNMAPPED_READS; + copy.UNPAIRED_READ_DUPLICATES = this.UNPAIRED_READ_DUPLICATES; + copy.READ_PAIR_DUPLICATES = this.READ_PAIR_DUPLICATES; + copy.READ_PAIR_OPTICAL_DUPLICATES = this.READ_PAIR_OPTICAL_DUPLICATES; + copy.PERCENT_DUPLICATION = this.PERCENT_DUPLICATION; + copy.ESTIMATED_LIBRARY_SIZE = this.ESTIMATED_LIBRARY_SIZE; + + return copy; + } + + /** + * Update metrics given a record or GATKRead + */ + public void updateMetrics(final SAMRecord rec) { + update(rec.getReadUnmappedFlag(), rec.isSecondaryOrSupplementary(), ReadUtils.readHasMappedMate(rec), rec.getDuplicateReadFlag() ); + } + + public void updateMetrics(final GATKRead read) { + update(read.isUnmapped(), read.isSecondaryAlignment() || read.isSupplementaryAlignment(), ReadUtils.readHasMappedMate(read), read.isDuplicate() ); + } + + private void update(final boolean readUnmappedFlag, final boolean secondaryOrSupplementary, final boolean mappedMate, final boolean isDuplicate) { + if (readUnmappedFlag) { + ++this.UNMAPPED_READS; + } else if (secondaryOrSupplementary) { + ++this.SECONDARY_OR_SUPPLEMENTARY_RDS; + } else if (!mappedMate) { + ++this.UNPAIRED_READS_EXAMINED; + } else { + ++this.READ_PAIRS_EXAMINED; // will need to be divided by 2 at the end + } + // Update the duplication metrics + if (isDuplicate && !secondaryOrSupplementary) { + if (!mappedMate) { + ++this.UNPAIRED_READ_DUPLICATES; + } else { + ++this.READ_PAIR_DUPLICATES;// will need to be divided by 2 at the end + } + } + } + +} diff --git a/src/main/java/org/broadinstitute/hellbender/utils/read/markduplicates/LibraryIdGenerator.java b/src/main/java/org/broadinstitute/hellbender/utils/read/markduplicates/LibraryIdGenerator.java index c2ad17fe5f4..22997edbbb8 100644 --- a/src/main/java/org/broadinstitute/hellbender/utils/read/markduplicates/LibraryIdGenerator.java +++ b/src/main/java/org/broadinstitute/hellbender/utils/read/markduplicates/LibraryIdGenerator.java @@ -1,5 +1,6 @@ package org.broadinstitute.hellbender.utils.read.markduplicates; +import htsjdk.samtools.ReservedTagConstants; import htsjdk.samtools.SAMFileHeader; import htsjdk.samtools.SAMReadGroupRecord; import htsjdk.samtools.SAMRecord; @@ -7,6 +8,7 @@ import java.util.LinkedHashMap; import java.util.Map; +import java.util.Optional; /** * A class to generate library Ids and keep duplication metrics by library IDs. @@ -15,10 +17,12 @@ */ public final class LibraryIdGenerator { + public static final String UNKNOWN_LIBRARY = "Unknown Library"; + private final SAMFileHeader header; private final Map libraryIds = new LinkedHashMap<>(); // from library string to library id private short nextLibraryId = 1; - private final Map metricsByLibrary = new LinkedHashMap<>(); + private final Map metricsByLibrary = new LinkedHashMap<>(); private final Histogram opticalDuplicatesByLibraryId = new Histogram<>(); @@ -26,10 +30,10 @@ public LibraryIdGenerator(final SAMFileHeader header) { this.header = header; for (final SAMReadGroupRecord readGroup : header.getReadGroups()) { - final String library = readGroup.getLibrary(); - DuplicationMetrics metrics = metricsByLibrary.get(library); + final String library = getReadGroupLibraryName(readGroup); + GATKDuplicationMetrics metrics = metricsByLibrary.get(library); if (metrics == null) { - metrics = new DuplicationMetrics(); + metrics = new GATKDuplicationMetrics(); metrics.LIBRARY = library; metricsByLibrary.put(library, metrics); } @@ -38,17 +42,22 @@ public LibraryIdGenerator(final SAMFileHeader header) { public Map getLibraryIdsMap() { return this.libraryIds; } - public Map getMetricsByLibraryMap() { return this.metricsByLibrary; } + public Map getMetricsByLibraryMap() { return this.metricsByLibrary; } public Histogram getOpticalDuplicatesByLibraryIdMap() { return this.opticalDuplicatesByLibraryId; } + public static String getReadGroupLibraryName(final SAMReadGroupRecord readGroup) { + return Optional.ofNullable(readGroup.getLibrary()) + .orElse(UNKNOWN_LIBRARY); + } + /** * Gets the library name from the header for the record. If the RG tag is not present on * the record, or the library isn't denoted on the read group, a constant string is * returned. */ public static String getLibraryName(final SAMFileHeader header, final SAMRecord rec) { - final String readGroupId = (String) rec.getAttribute("RG"); + final String readGroupId = (String) rec.getAttribute(ReservedTagConstants.READ_GROUP_ID); return getLibraryName(header, readGroupId); } @@ -66,7 +75,7 @@ public static String getLibraryName(final SAMFileHeader header, String readGroup } } - return "Unknown Library"; + return UNKNOWN_LIBRARY; } /** Get the library ID for the given SAM record. */ @@ -82,11 +91,11 @@ public short getLibraryId(final SAMRecord rec) { return libraryId; } - public DuplicationMetrics getMetricsByLibrary(final String library) { + public GATKDuplicationMetrics getMetricsByLibrary(final String library) { return this.metricsByLibrary.get(library); } - public void addMetricsByLibrary(final String library, final DuplicationMetrics metrics) { + public void addMetricsByLibrary(final String library, final GATKDuplicationMetrics metrics) { this.metricsByLibrary.put(library, metrics); } diff --git a/src/main/java/org/broadinstitute/hellbender/utils/read/markduplicates/ReadEndsForMarkDuplicatesCodec.java b/src/main/java/org/broadinstitute/hellbender/utils/read/markduplicates/ReadEndsForMarkDuplicatesCodec.java index 5393bd8c352..0067a32881a 100644 --- a/src/main/java/org/broadinstitute/hellbender/utils/read/markduplicates/ReadEndsForMarkDuplicatesCodec.java +++ b/src/main/java/org/broadinstitute/hellbender/utils/read/markduplicates/ReadEndsForMarkDuplicatesCodec.java @@ -6,7 +6,7 @@ import java.io.*; -/** Coded for ReadEnds that just outputs the primitive fields and reads them back. */ +/** Codec for ReadEnds that just outputs the primitive fields and reads them back. */ public final class ReadEndsForMarkDuplicatesCodec implements SortingCollection.Codec { private DataInputStream in; private DataOutputStream out; diff --git a/src/main/java/org/broadinstitute/hellbender/utils/test/testers/SamFileTester.java b/src/main/java/org/broadinstitute/hellbender/utils/test/testers/SamFileTester.java index 53e5b1526e1..1627503bd1e 100644 --- a/src/main/java/org/broadinstitute/hellbender/utils/test/testers/SamFileTester.java +++ b/src/main/java/org/broadinstitute/hellbender/utils/test/testers/SamFileTester.java @@ -4,6 +4,7 @@ import htsjdk.samtools.util.CloseableIterator; import htsjdk.samtools.util.IOUtil; import org.broadinstitute.hellbender.cmdline.StandardArgumentDefinitions; +import org.testng.Assert; import org.broadinstitute.hellbender.utils.test.CommandLineProgramTester; import java.io.File; @@ -23,17 +24,29 @@ public abstract class SamFileTester implements CommandLineProgramTester { private boolean deleteOnExit = true; protected final List args = new ArrayList<>(); - public SamFileTester(final int readLength, final boolean deleteOnExit, final int defaultChromosomeLength, final DuplicateScoringStrategy.ScoringStrategy duplicateScoringStrategy) { + public SamFileTester(final int readLength, final boolean deleteOnExit, final int defaultChromosomeLength, final DuplicateScoringStrategy.ScoringStrategy duplicateScoringStrategy, final SAMFileHeader.SortOrder sortOrder, final boolean recordsNeedSorting) { this.deleteOnExit = deleteOnExit; - this.samRecordSetBuilder = new SAMRecordSetBuilder(true, SAMFileHeader.SortOrder.coordinate, true, defaultChromosomeLength, duplicateScoringStrategy); + this.samRecordSetBuilder = new SAMRecordSetBuilder(recordsNeedSorting, sortOrder, true, defaultChromosomeLength, duplicateScoringStrategy); samRecordSetBuilder.setReadLength(readLength); setOutputDir(); } + public SamFileTester(final int readLength, final boolean deleteOnExit, final int defaultChromosomeLength, final DuplicateScoringStrategy.ScoringStrategy duplicateScoringStrategy, final SAMFileHeader.SortOrder sortOrder) { + this(readLength, deleteOnExit, defaultChromosomeLength, duplicateScoringStrategy, SAMFileHeader.SortOrder.coordinate, true); + } + + public SamFileTester(final int readLength, final boolean deleteOnExit, final int defaultChromosomeLength, final DuplicateScoringStrategy.ScoringStrategy duplicateScoringStrategy) { + this(readLength, deleteOnExit, defaultChromosomeLength, duplicateScoringStrategy, SAMFileHeader.SortOrder.coordinate); + } + public SamFileTester(final int readLength, final boolean deleteOnExit, final int defaultChromosomeLength) { this(readLength, deleteOnExit, defaultChromosomeLength, SAMRecordSetBuilder.DEFAULT_DUPLICATE_SCORING_STRATEGY); } + public SamFileTester(final int readLength, final boolean deleteOnExit) { + this(readLength, deleteOnExit, SAMRecordSetBuilder.DEFAULT_CHROMOSOME_LENGTH); + } + public void setHeader(final SAMFileHeader header) { this.samRecordSetBuilder.setHeader(header); } @@ -43,7 +56,9 @@ public SAMFileHeader getHeader() { } public void addRecord(final SAMRecord record) { - this.duplicateFlags.put(samRecordToDuplicatesFlagsKey(record), record.getDuplicateReadFlag()); + final String key = samRecordToDuplicatesFlagsKey(record); + Assert.assertFalse(this.duplicateFlags.containsKey(key)); + this.duplicateFlags.put(key, record.getDuplicateReadFlag()); this.samRecordSetBuilder.addRecord(record); } @@ -95,18 +110,30 @@ public boolean getDeleteOnExit() { } protected String samRecordToDuplicatesFlagsKey(final SAMRecord record) { - String readName = record.getReadName() - + "-" - + record.getReadPairedFlag() - + "-"; + final StringBuilder nameBuilder = new StringBuilder(); + nameBuilder.append(record.getReadName()); + nameBuilder.append("-"); + + if (record.getReadUnmappedFlag()) { + nameBuilder.append("Unmapped"); + } else { + nameBuilder.append(record.getContig()) + .append("-") + .append(record.getAlignmentStart()); + } + nameBuilder.append("-") + .append(record.getReadPairedFlag()) + .append("-").append(record.isSecondaryAlignment()) + .append("-"); + if (record.getReadPairedFlag()) { - readName += record.getFirstOfPairFlag() - + "-" - + record.getSecondOfPairFlag(); + nameBuilder.append(record.getFirstOfPairFlag()) + .append("-") + .append(record.getSecondOfPairFlag()); } else { - readName += "false-false"; + nameBuilder.append("false-false"); } - return readName; + return nameBuilder.toString(); } // Below are a bunch of utility methods for adding records to the SAMRecordSetBuilder @@ -115,6 +142,11 @@ public void addUnmappedFragment(final int referenceSequenceIndex, addFragment(referenceSequenceIndex, -1, true, false, null, null, defaultQualityScore, false); } + public void addUnmappedFragment(final int referenceSequenceIndex, + final String qualityString) { + addFragment(referenceSequenceIndex, -1, true, false, null, qualityString, -1, false); + } + public void addUnmappedPair(final int referenceSequenceIndex, final int defaultQualityScore) { addMatePair(referenceSequenceIndex, -1, -1, true, true, false, false, null, null, false, false, false, false, false, defaultQualityScore); @@ -135,6 +167,18 @@ public void addMappedFragment(final int referenceSequenceIndex, final int alignm addFragment(referenceSequenceIndex, alignmentStart, false, isDuplicate, cigar, null, defaultQualityScore, false); } + public void addMappedFragment(final int referenceSequenceIndex, final int alignmentStart, final boolean isDuplicate, final String cigar, + final String qualityString, + final int defaultQualityScore) { + addFragment(referenceSequenceIndex, alignmentStart, false, isDuplicate, cigar, qualityString, defaultQualityScore, false); + } + + public void addMappedFragment(final String readName, final int referenceSequenceIndex, final int alignmentStart, final boolean isDuplicate, final String cigar, + final String qualityString, final boolean isSecondary, final boolean isSupplementary, + final int defaultQualityScore) { + addFragment(readName, referenceSequenceIndex, alignmentStart, false, isDuplicate, cigar, qualityString, defaultQualityScore, isSecondary, isSupplementary); + } + public void addMappedPair(final int referenceSequenceIndex, final int alignmentStart1, final int alignmentStart2, @@ -195,10 +239,19 @@ public void addMatePair(final int referenceSequenceIndex, private void addFragment(final int referenceSequenceIndex, final int alignmentStart, final boolean recordUnmapped, final boolean isDuplicate, final String cigar, final String qualityString, final int defaultQualityScore, final boolean isSecondary) { - final SAMRecord record = samRecordSetBuilder.addFrag("READ" + readNameCounter++, referenceSequenceIndex, alignmentStart, false, - recordUnmapped, cigar, qualityString, defaultQualityScore, isSecondary); + addFragment("READ" + readNameCounter++, referenceSequenceIndex, alignmentStart, recordUnmapped, isDuplicate, cigar, + qualityString, defaultQualityScore, isSecondary, false); + } + + private void addFragment(final String readName, final int referenceSequenceIndex, final int alignmentStart, final boolean recordUnmapped, final boolean isDuplicate, final String cigar, + final String qualityString, final int defaultQualityScore, final boolean isSecondary, final boolean isSupplementary) { + + final SAMRecord record = samRecordSetBuilder.addFrag(readName, referenceSequenceIndex, alignmentStart, false, + recordUnmapped, cigar, qualityString, defaultQualityScore, isSecondary, isSupplementary); - this.duplicateFlags.put(samRecordToDuplicatesFlagsKey(record), isDuplicate); + final String key = samRecordToDuplicatesFlagsKey(record); + Assert.assertFalse(this.duplicateFlags.containsKey(key)); + this.duplicateFlags.put(key, isDuplicate); } public void addMatePair(final String readName, @@ -237,8 +290,13 @@ public void addMatePair(final String readName, samRecordSetBuilder.getRecords().remove(record2); } - this.duplicateFlags.put(samRecordToDuplicatesFlagsKey(record1), isDuplicate1); - this.duplicateFlags.put(samRecordToDuplicatesFlagsKey(record2), isDuplicate2); + final String key1 = samRecordToDuplicatesFlagsKey(record1); + Assert.assertFalse(this.duplicateFlags.containsKey(key1)); + this.duplicateFlags.put(key1, isDuplicate1); + + final String key2 = samRecordToDuplicatesFlagsKey(record2); + Assert.assertFalse(this.duplicateFlags.containsKey(key2)); + this.duplicateFlags.put(key2, isDuplicate2); } public void addMatePair(final String readName, @@ -284,9 +342,7 @@ private File createInputFile() { // Create the input file final File input = new File(outputDir, "input.bam"); try (final SAMFileWriter writer = new SAMFileWriterFactory().setCreateIndex(true).makeBAMWriter(samRecordSetBuilder.getHeader(), true, input)) { - for (final SAMRecord record : samRecordSetBuilder.getRecords()) { - writer.addAlignment(record); - } + samRecordSetBuilder.getRecords().forEach(writer::addAlignment); } return input; } @@ -294,4 +350,7 @@ private File createInputFile() { public SamReader getInput() { return samRecordSetBuilder.getSamReader(); } + + public SAMRecordSetBuilder getSamRecordSetBuilder() { return samRecordSetBuilder; } + } diff --git a/src/test/java/org/broadinstitute/hellbender/tools/spark/pipelines/MarkDuplicatesSparkIntegrationTest.java b/src/test/java/org/broadinstitute/hellbender/tools/spark/pipelines/MarkDuplicatesSparkIntegrationTest.java index c8d25dbb15a..d2f0ad5dd18 100644 --- a/src/test/java/org/broadinstitute/hellbender/tools/spark/pipelines/MarkDuplicatesSparkIntegrationTest.java +++ b/src/test/java/org/broadinstitute/hellbender/tools/spark/pipelines/MarkDuplicatesSparkIntegrationTest.java @@ -6,13 +6,14 @@ import org.apache.spark.SparkException; import org.broadinstitute.hellbender.cmdline.CommandLineProgram; import org.broadinstitute.hellbender.cmdline.StandardArgumentDefinitions; +import org.broadinstitute.hellbender.cmdline.argumentcollections.MarkDuplicatesSparkArgumentCollection; import org.broadinstitute.hellbender.engine.ReadsDataSource; import org.broadinstitute.hellbender.engine.spark.GATKSparkTool; import org.broadinstitute.hellbender.exceptions.UserException; import org.broadinstitute.hellbender.tools.walkers.markduplicates.MarkDuplicatesGATKIntegrationTest; import org.broadinstitute.hellbender.tools.spark.transforms.markduplicates.MarkDuplicatesSpark; import org.broadinstitute.hellbender.utils.read.GATKRead; -import org.broadinstitute.hellbender.utils.read.markduplicates.DuplicationMetrics; +import org.broadinstitute.hellbender.utils.read.markduplicates.GATKDuplicationMetrics; import org.broadinstitute.hellbender.utils.read.markduplicates.MarkDuplicatesSparkTester; import org.broadinstitute.hellbender.utils.test.ArgumentsBuilder; import org.broadinstitute.hellbender.tools.walkers.markduplicates.AbstractMarkDuplicatesCommandLineProgramTest; @@ -35,7 +36,9 @@ public class MarkDuplicatesSparkIntegrationTest extends AbstractMarkDuplicatesCo @Override protected AbstractMarkDuplicatesTester getTester() { - return new MarkDuplicatesSparkTester(); + MarkDuplicatesSparkTester markDuplicatesSparkTester = new MarkDuplicatesSparkTester(); + markDuplicatesSparkTester.addArg("--"+ MarkDuplicatesSparkArgumentCollection.DO_NOT_MARK_UNMAPPED_MATES_LONG_NAME); + return markDuplicatesSparkTester; } @Override @@ -43,6 +46,9 @@ protected CommandLineProgram getCommandLineProgramInstance() { return new MarkDuplicatesSpark(); } + @Override + protected boolean markSecondaryAndSupplementaryRecordsLikeTheCanonical() { return true; } + @Test(dataProvider = "testMDdata", groups = "spark") @Override public void testMDOrder(final File input, final File expectedOutput) throws Exception { @@ -122,13 +128,13 @@ public void testMarkDuplicatesSparkIntegrationTestLocal( Assert.assertEquals(totalReads, totalExpected, "Wrong number of reads in output BAM"); Assert.assertEquals(duplicateReads, dupsExpected, "Wrong number of duplicate reads in output BAM"); - final MetricsFile> metricsOutput = new MetricsFile<>(); + final MetricsFile> metricsOutput = new MetricsFile<>(); try { metricsOutput.read(new FileReader(metricsFile)); } catch (final FileNotFoundException ex) { System.err.println("Metrics file not found: " + ex); } - final List nonEmptyMetrics = metricsOutput.getMetrics().stream().filter( + final List nonEmptyMetrics = metricsOutput.getMetrics().stream().filter( metric -> metric.UNPAIRED_READS_EXAMINED != 0L || metric.READ_PAIRS_EXAMINED != 0L || @@ -143,7 +149,7 @@ public void testMarkDuplicatesSparkIntegrationTestLocal( Assert.assertEquals(nonEmptyMetrics.size(), metricsExpected.size(), "Wrong number of metrics with non-zero fields."); for (int i = 0; i < nonEmptyMetrics.size(); i++ ){ - final DuplicationMetrics observedMetrics = nonEmptyMetrics.get(i); + final GATKDuplicationMetrics observedMetrics = nonEmptyMetrics.get(i); List expectedList = metricsExpected.get(observedMetrics.LIBRARY); Assert.assertNotNull(expectedList, "Unexpected library found: " + observedMetrics.LIBRARY); Assert.assertEquals(observedMetrics.UNPAIRED_READS_EXAMINED, expectedList.get(0)); diff --git a/src/test/java/org/broadinstitute/hellbender/tools/walkers/markduplicates/AbstractMarkDuplicatesCommandLineProgramTest.java b/src/test/java/org/broadinstitute/hellbender/tools/walkers/markduplicates/AbstractMarkDuplicatesCommandLineProgramTest.java index abc197d2d76..a014389b3f9 100644 --- a/src/test/java/org/broadinstitute/hellbender/tools/walkers/markduplicates/AbstractMarkDuplicatesCommandLineProgramTest.java +++ b/src/test/java/org/broadinstitute/hellbender/tools/walkers/markduplicates/AbstractMarkDuplicatesCommandLineProgramTest.java @@ -36,6 +36,8 @@ public abstract class AbstractMarkDuplicatesCommandLineProgramTest extends Comma protected abstract CommandLineProgram getCommandLineProgramInstance(); + protected boolean markSecondaryAndSupplementaryRecordsLikeTheCanonical() { return false; } + // ELIGIBLE_BASE_QUALITY is the minimum quality considered by the dataflow version // for marking a specific pair as best. We use it to ensure that the expected // pair wins the tie so that the tie is not broken randomly. @@ -43,21 +45,17 @@ public abstract class AbstractMarkDuplicatesCommandLineProgramTest extends Comma protected static final int ELIGIBLE_BASE_QUALITY = 15; @Test - public void testTwoMappedPairsWithSoftClippingFirstOfPairOnly() { + public void testSingleUnmappedFragment() { final AbstractMarkDuplicatesTester tester = getTester(); - // NB: no duplicates - // 5'1: 2, 5'2:46+73M=118 - // 5'1: 2, 5'2:51+68M=118 - tester.addMappedPair(0, 12, 46, false, false, "6S42M28S", "3S73M", true, 50); // only add the first one - // NB: this next record should not be a duplicate in MarkDuplicatesGATK - tester.addMappedPair(0, 12, 51, false, false, "6S42M28S", "8S68M", true, 50); // only add the first one + tester.addUnmappedFragment(-1, DEFAULT_BASE_QUALITY); tester.runTest(); } @Test - public void testSingleUnmappedFragment() { + public void testTwoUnmappedFragments() { final AbstractMarkDuplicatesTester tester = getTester(); tester.addUnmappedFragment(-1, DEFAULT_BASE_QUALITY); + tester.addUnmappedFragment(-1, DEFAULT_BASE_QUALITY); tester.runTest(); } @@ -162,6 +160,7 @@ public void testOpticalDuplicateFinding() { @Test public void testOpticalDuplicateClusterSamePositionNoOpticalDuplicates() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(101); tester.setExpectedOpticalDuplicate(0); tester.addMatePair("RUNID:7:1203:2886:82292", 1, 485253, 485253, false, false, true, true, "42M59S", "59S42M", false, true, false, false, false, DEFAULT_BASE_QUALITY); // duplicate tester.addMatePair("RUNID:7:1203:2884:16834", 1, 485253, 485253, false, false, false, false, "59S42M", "42M59S", true, false, false, false, false, ELIGIBLE_BASE_QUALITY); @@ -214,6 +213,7 @@ public void testOpticalDuplicatesAndPCRDuplicatesOrientationDifference() { @Test public void testOpticalDuplicateClusterSamePositionNoOpticalDuplicatesWithinPixelDistance() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(101); tester.setExpectedOpticalDuplicate(0); tester.addMatePair("RUNID:7:1203:2886:16834", 1, 485253, 485253, false, false, true, true, "42M59S", "59S42M", false, true, false, false, false, DEFAULT_BASE_QUALITY); // duplicate tester.addMatePair("RUNID:7:1203:2884:16835", 1, 485253, 485253, false, false, false, false, "59S42M", "42M59S", true, false, false, false, false, ELIGIBLE_BASE_QUALITY); @@ -223,6 +223,7 @@ public void testOpticalDuplicateClusterSamePositionNoOpticalDuplicatesWithinPixe @Test public void testOpticalDuplicateClusterSamePositionOneOpticalDuplicatesWithinPixelDistance() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(45); tester.setExpectedOpticalDuplicate(1); tester.addMatePair("RUNID:7:1203:2886:16834", 1, 485253, 485253, false, false, true, true, "45M", "45M", false, true, false, false, false, DEFAULT_BASE_QUALITY); // duplicate tester.addMatePair("RUNID:7:1203:2884:16835", 1, 485253, 485253, false, false, false, false, "45M", "45M", false, true, false, false, false, ELIGIBLE_BASE_QUALITY); @@ -232,6 +233,7 @@ public void testOpticalDuplicateClusterSamePositionOneOpticalDuplicatesWithinPix @Test public void testOpticalDuplicateClusterOneEndSamePositionOneCluster() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(101); tester.setExpectedOpticalDuplicate(1); tester.addMatePair("RUNID:7:2205:17939:39728", 1, 485328, 485312, false, false, false, false, "55M46S", "30S71M", false, true, false, false, false, ELIGIBLE_BASE_QUALITY); tester.addMatePair("RUNID:7:2205:17949:39745", 1, 485328, 485328, false, false, true, true, "55M46S", "46S55M", false, true, false, false, false, DEFAULT_BASE_QUALITY); // duplicate @@ -250,6 +252,7 @@ public void testTwoMappedPairsAndMappedSecondaryFragment() { @Test public void testMappedFragmentAndMappedPairFirstOfPairNonPrimary() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(76); tester.addMappedFragment(1, 1, false, ELIGIBLE_BASE_QUALITY); tester.addMatePair(1, 200, 0, false, true, false, false, "54M22S", null, false, false, true, true, false, DEFAULT_BASE_QUALITY); tester.runTest(); @@ -258,14 +261,19 @@ public void testMappedFragmentAndMappedPairFirstOfPairNonPrimary() { @Test public void testTwoMappedPairsMatesSoftClipped() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(76); tester.addMappedPair(1, 10022, 10051, false, false, "76M", "8S68M", false, true, false, DEFAULT_BASE_QUALITY); tester.addMappedPair(1, 10022, 10063, false, false, "76M", "5S71M", false, true, false, DEFAULT_BASE_QUALITY); tester.runTest(); } + @Test public void testTwoMappedPairsWithSoftClipping() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(76); + //todo Picard says no duplicates, we say duplicate, what's going on? + // NB: no duplicates // 5'1: 2, 5'2:46+73M=118 // 5'1: 2, 5'2:51+68M=118 tester.addMappedPair(1, 2, 46, false, false, "6S42M28S", "3S73M", false, ELIGIBLE_BASE_QUALITY); @@ -276,6 +284,7 @@ public void testTwoMappedPairsWithSoftClipping() { @Test public void testTwoMappedPairsWithSoftClippingFirstOfPairOnlyNoMateCigar() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(76); tester.setNoMateCigars(true); // NB: no duplicates // 5'1: 2, 5'2:46+73M=118 @@ -288,7 +297,10 @@ public void testTwoMappedPairsWithSoftClippingFirstOfPairOnlyNoMateCigar() { @Test public void testTwoMappedPairsWithSoftClippingBoth() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(76); + // mapped reference length: 73 + 42 = 115 tester.addMappedPair(1, 10046, 10002, true, true, "3S73M", "6S42M28S", true, false, false, DEFAULT_BASE_QUALITY); // duplicate + // mapped reference length: 68 + 48 = 116 tester.addMappedPair(1, 10051, 10002, false, false, "8S68M", "6S48M22S", true, false, false, ELIGIBLE_BASE_QUALITY); tester.runTest(); } @@ -296,6 +308,7 @@ public void testTwoMappedPairsWithSoftClippingBoth() { @Test public void testMatePairSecondUnmapped() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(76); tester.addMatePair(1, 10049, 10049, false, true, false, false, "11M2I63M", null, false, false, false, false, false, DEFAULT_BASE_QUALITY); // neither are duplicates tester.runTest(); } @@ -303,6 +316,7 @@ public void testMatePairSecondUnmapped() { @Test public void testMatePairFirstUnmapped() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(76); tester.addMatePair(1, 10056, 10056, true, false, false, false, null, "54M22S", false, false, false, false, false, DEFAULT_BASE_QUALITY); // neither are duplicates tester.runTest(); } @@ -310,7 +324,11 @@ public void testMatePairFirstUnmapped() { @Test public void testMappedFragmentAndMatePairSecondUnmapped() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(76); tester.addMatePair(1, 10049, 10049, false, true, false, false, "11M2I63M", null, false, false, false, false, false, ELIGIBLE_BASE_QUALITY); + // We set the length here separately because some of the MD variants use TOTAL_MAPPED_REFERENCE_LENGTH as their DUPLICATE_SCORING_STRATEGY. If we kept the read length + // as 76 like with the mate pair, it would mark the mapped read in the pair as a duplicate because it has less reference length due to the two insertions in the cigar + tester.getSamRecordSetBuilder().setReadLength(36); tester.addMappedFragment(1, 10049, true, DEFAULT_BASE_QUALITY); // duplicate tester.runTest(); } @@ -318,7 +336,11 @@ public void testMappedFragmentAndMatePairSecondUnmapped() { @Test public void testMappedFragmentAndMatePairFirstUnmapped() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(76); tester.addMatePair(1, 10049, 10049, true, false, false, false, null, "11M2I63M", false, false, false, false, false, ELIGIBLE_BASE_QUALITY); + // We set the length here separately because some of the MD variants use TOTAL_MAPPED_REFERENCE_LENGTH as their DUPLICATE_SCORING_STRATEGY. If we kept the read length + // as 76 like with the mate pair, it would mark the mapped read in the pair as a duplicate because it has less reference length due to the two insertions in the cigar + tester.getSamRecordSetBuilder().setReadLength(36); tester.addMappedFragment(1, 10049, true, DEFAULT_BASE_QUALITY); // duplicate tester.runTest(); } @@ -326,6 +348,7 @@ public void testMappedFragmentAndMatePairFirstUnmapped() { @Test public void testMappedPairAndMatePairSecondUnmapped() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(76); tester.addMatePair(1, 10040, 10040, false, true, true, false, "76M", null, false, false, false, false, false, DEFAULT_BASE_QUALITY); // second a duplicate, // second end unmapped tester.addMappedPair(1, 10189, 10040, false, false, "41S35M", "65M11S", true, false, false, ELIGIBLE_BASE_QUALITY); // mapped OK @@ -335,6 +358,7 @@ public void testMappedPairAndMatePairSecondUnmapped() { @Test public void testMappedPairAndMatePairFirstUnmapped() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(76); tester.addMatePair(1, 10040, 10040, true, false, false, true, null, "76M", false, false, false, false, false, DEFAULT_BASE_QUALITY); // first a duplicate, // first end unmapped tester.addMappedPair(1, 10189, 10040, false, false, "41S35M", "65M11S", true, false, false, ELIGIBLE_BASE_QUALITY); // mapped OK @@ -345,6 +369,7 @@ public void testMappedPairAndMatePairFirstUnmapped() { @Test public void testMappedPairAndMatePairFirstOppositeStrandSecondUnmapped() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(101); // first end mapped OK -, second end unmapped tester.addMatePair(1, 484071, 484071, false, true, false, false, "66S35M", null, true, false, false, false, false, DEFAULT_BASE_QUALITY); // mapped OK +/- @@ -355,32 +380,29 @@ public void testMappedPairAndMatePairFirstOppositeStrandSecondUnmapped() { @Test public void testMappedPairAndMappedFragmentAndMatePairSecondUnmapped() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(76); tester.addMatePair(1, 10040, 10040, false, true, true, false, "76M", null, false, false, false, false, false, DEFAULT_BASE_QUALITY); // first a duplicate, // second end unmapped tester.addMappedPair(1, 10189, 10040, false, false, "41S35M", "65M11S", true, false, false, ELIGIBLE_BASE_QUALITY); // mapped OK tester.addMappedFragment(1, 10040, true, DEFAULT_BASE_QUALITY); // duplicate - if (this instanceof MarkDuplicatesSparkIntegrationTest) { - tester.addArg("--"+ MarkDuplicatesSparkArgumentCollection.DO_NOT_MARK_UNMAPPED_MATES_LONG_NAME); - } tester.runTest(); } @Test public void testMappedPairAndMappedFragmentAndMatePairFirstUnmapped() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(76); tester.addMatePair(1, 10040, 10040, true, false, false, true, null, "76M", false, false, false, false, false, DEFAULT_BASE_QUALITY); // first a duplicate, // first end unmapped tester.addMappedPair(1, 10189, 10040, false, false, "41S35M", "65M11S", true, false, false, ELIGIBLE_BASE_QUALITY); // mapped OK tester.addMappedFragment(1, 10040, true, DEFAULT_BASE_QUALITY); // duplicate - if (this instanceof MarkDuplicatesSparkIntegrationTest) { - tester.addArg("--"+ MarkDuplicatesSparkArgumentCollection.DO_NOT_MARK_UNMAPPED_MATES_LONG_NAME); - } tester.runTest(); } @Test public void testTwoMappedPairsWithOppositeOrientations() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(76); tester.addMappedPair(1, 10182, 10038, true, true, "32S44M", "66M10S", true, false, false, DEFAULT_BASE_QUALITY); // -/+ tester.addMappedPair(1, 10038, 10182, false, false, "70M6S", "32S44M", false, true, false, ELIGIBLE_BASE_QUALITY); // +/-, both are duplicates tester.runTest(); @@ -389,6 +411,7 @@ public void testTwoMappedPairsWithOppositeOrientations() { @Test public void testTwoMappedPairsWithOppositeOrientationsNumberTwo() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(76); tester.addMappedPair(1, 10038, 10182, false, false, "70M6S", "32S44M", false, true, false, ELIGIBLE_BASE_QUALITY); // +/-, both are duplicates tester.addMappedPair(1, 10182, 10038, true, true, "32S44M", "66M10S", true, false, false, DEFAULT_BASE_QUALITY); // -/+ tester.runTest(); @@ -397,6 +420,7 @@ public void testTwoMappedPairsWithOppositeOrientationsNumberTwo() { @Test public void testThreeMappedPairsWithMatchingSecondMate() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(76); // Read0 and Read2 are duplicates // 10181+41=10220, 10058 tester.addMappedPair(1, 10181, 10058, false, false, "41S35M", "47M29S", true, false, false, ELIGIBLE_BASE_QUALITY); // -/+ @@ -410,6 +434,7 @@ public void testThreeMappedPairsWithMatchingSecondMate() { @Test public void testMappedPairWithSamePosition() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(76); tester.addMappedPair(1, 4914, 4914, false, false, "37M39S", "73M3S", false, false, false, DEFAULT_BASE_QUALITY); // +/+ tester.runTest(); } @@ -417,6 +442,7 @@ public void testMappedPairWithSamePosition() { @Test public void testMappedPairWithSamePositionSameCigar() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(76); tester.addMappedPair(1, 4914, 4914, false, false, "37M39S", "37M39S", false, false, false, DEFAULT_BASE_QUALITY); // +/+ tester.runTest(); } @@ -424,6 +450,7 @@ public void testMappedPairWithSamePositionSameCigar() { @Test public void testTwoMappedPairWithSamePosition() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(76); tester.addMappedPair(0, 5604914, 5604914, false, false, "37M39S", "73M3S", false, false, false, ELIGIBLE_BASE_QUALITY); // +/+ tester.addMappedPair(0, 5604914, 5604914, true, true, "37M39S", "73M3S", false, false, false, DEFAULT_BASE_QUALITY); // +/+ tester.runTest(); @@ -448,6 +475,7 @@ public void testTwoMappedPairWithSamePositionDifferentStrands2() { @Test public void testMappedPairWithFirstEndSamePositionAndOther() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(76); tester.addMappedPair(0, 5604914, 5605914, false, false, "37M39S", "73M3S", false, false, false, DEFAULT_BASE_QUALITY); // +/+ tester.addMappedPair(0, 5604914, 5604914, false, false, "37M39S", "73M3S", false, false, false, DEFAULT_BASE_QUALITY); // +/+ tester.runTest(); @@ -504,6 +532,7 @@ public void testThreeGroupsOnDifferentChromosomesOfThreeMappedPairs() { @Test public void testBulkFragmentsNoDuplicates() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(100); for(int position = 1; position <= 10000; position += 1) { tester.addMappedFragment(0, position, false, "100M", DEFAULT_BASE_QUALITY); } @@ -513,6 +542,7 @@ public void testBulkFragmentsNoDuplicates() { @Test public void testBulkFragmentsWithDuplicates() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(100); for(int position = 1; position <= 10000; position += 1) { tester.addMappedFragment(0, position, false, "100M", ELIGIBLE_BASE_QUALITY); tester.addMappedFragment(0, position, true, "100M", DEFAULT_BASE_QUALITY); @@ -526,8 +556,10 @@ public void testBulkFragmentsWithDuplicates() { @Test public void testStackOverFlowPairSetSwap() { final AbstractMarkDuplicatesTester tester = getTester(); - File input = new File(TEST_DATA_DIR, "markDuplicatesWithMateCigar.pairSet.swap.sam"); - SamReader reader = SamReaderFactory.makeDefault().open(input); + tester.getSamRecordSetBuilder().setReadLength(68); + + final File input = new File(TEST_DATA_DIR, "markDuplicatesWithMateCigar.pairSet.swap.sam"); + final SamReader reader = SamReaderFactory.makeDefault().open(input); tester.setHeader(reader.getFileHeader()); for (final SAMRecord record : reader) { tester.addRecord(record); @@ -540,6 +572,7 @@ public void testStackOverFlowPairSetSwap() { @Test public void testSecondEndIsBeforeFirstInCoordinate() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(68); tester.addMappedPair(0, 108855339, 108855323, false, false, "33S35M", "17S51M", true, true, false, DEFAULT_BASE_QUALITY); // +/- tester.runTest(); } @@ -547,6 +580,7 @@ public void testSecondEndIsBeforeFirstInCoordinate() { @Test public void testPathologicalOrderingAtTheSamePosition() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(68); tester.setExpectedOpticalDuplicate(1); @@ -554,12 +588,12 @@ public void testPathologicalOrderingAtTheSamePosition() { tester.addMatePair("RUNID:3:1:15029:113060", 0, 129384554, 129384554, false, false, true, true, "68M", "68M", false, false, false, false, false, DEFAULT_BASE_QUALITY); // Create the pathology - try (CloseableIterator iterator = tester.getRecordIterator()) { - int[] qualityOffset = {20, 30, 10, 40}; // creates an interesting pathological ordering + try (final CloseableIterator iterator = tester.getRecordIterator()) { + final int[] qualityOffset = {20, 30, 10, 40}; // creates an interesting pathological ordering int index = 0; while (iterator.hasNext()) { final SAMRecord record = iterator.next(); - byte[] quals = new byte[record.getReadLength()]; + final byte[] quals = new byte[record.getReadLength()]; for (int i = 0; i < record.getReadLength(); i++) { quals[i] = (byte) (qualityOffset[index] + 10); } @@ -575,6 +609,7 @@ public void testPathologicalOrderingAtTheSamePosition() { @Test public void testDifferentChromosomesInOppositeOrder() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(101); tester.setExpectedOpticalDuplicate(1); tester.addMatePair("RUNID:6:101:17642:6835", 0, 1, 123989, 18281, false, false, true, true, "37S64M", "52M49S", false, false, false, false, false, DEFAULT_BASE_QUALITY, "1"); tester.addMatePair("RUNID:6:101:17616:6888", 1, 0, 18281, 123989, false, false, false, false, "52M49S", "37S64M", false, false, false, false, false, ELIGIBLE_BASE_QUALITY, "1"); @@ -584,12 +619,36 @@ public void testDifferentChromosomesInOppositeOrder() { @Test public void testOpticalDuplicateClustersAddingSecondEndFirstSameCoordinate() { final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(68); tester.setExpectedOpticalDuplicate(1); tester.addMatePair("RUNID:1:1:15993:13361", 2, 41212324, 41212310, false, false, false, false, "33S35M", "19S49M", true, true, false, false, false, ELIGIBLE_BASE_QUALITY); tester.addMatePair("RUNID:1:1:16020:13352", 2, 41212324, 41212319, false, false, true, true, "33S35M", "28S40M", true, true, false, false, false, DEFAULT_BASE_QUALITY); tester.runTest(); } + @DataProvider(name = "secondarySupplementaryData") + public Object[][] secondarySupplementaryData() { + return new Object[][] { + { true, true , true}, + { true, false, true}, + { false, true , true}, + { true, true , false}, + { true, false, false}, + { false, true , false} + }; + } + + @Test(dataProvider = "secondarySupplementaryData") + public void testTwoMappedPairsWithSupplementaryReads(final Boolean additionalFragSecondary, final Boolean additionalFragSupplementary, final Boolean fragLikeFirst) { + final AbstractMarkDuplicatesTester tester = getTester(); + tester.getSamRecordSetBuilder().setReadLength(68); + tester.setExpectedOpticalDuplicate(1); + tester.addMatePair("RUNID:1:1:15993:13361", 2, 41212324, 41212310, false, false, false, false, "33S35M", "19S49M", true, true, false, false, false, DEFAULT_BASE_QUALITY); + tester.addMatePair("RUNID:1:1:16020:13352", 2, 41212324, 41212319, false, false, true, true, "33S35M", "28S40M", true, true, false, false, false, DEFAULT_BASE_QUALITY); // duplicate pair + tester.addMappedFragment(fragLikeFirst ? "RUNID:1:1:15993:13361" : "RUNID:1:1:16020:13352", 1, 400, markSecondaryAndSupplementaryRecordsLikeTheCanonical() && !fragLikeFirst, null, null, additionalFragSecondary, additionalFragSupplementary, DEFAULT_BASE_QUALITY); + tester.runTest(); + } + @Test(dataProvider = "testDuplicateDetectionDataProviderWithMetrics") public void testDuplicateDetectionDataProviderWithMetrics(final File sam, final File expectedMetrics) throws IOException { final File outputDir = BaseTest.createTempDir("MarkDups"); @@ -624,9 +683,9 @@ public void testDuplicateDetectionDataProviderWithMetrics(final File sam, final public Object[][] testDuplicateDetectionDataProviderWithMetrics() { //Note: the expected metrics files were created using picard 1.130 return new Object[][] { - {new File(TEST_DATA_DIR, "example.chr1.1-1K.unmarkedDups.bam"), new File(TEST_DATA_DIR,"expected.chr1.1-1K.unmarkedDups.markDuplicate.metrics")}, - {new File(TEST_DATA_DIR, "optical_dupes.bam"), new File(TEST_DATA_DIR,"expected.optical_dupes.markDuplicate.metrics")}, - {new File(TEST_DATA_DIR, "inputSingleLibrarySolexa16404.bam"), new File(TEST_DATA_DIR,"expected.inputSingleLibrarySolexa16404.metrics")}, + {new File(TEST_DATA_DIR, "example.chr1.1-1K.unmarkedDups.bam"), new File(TEST_DATA_DIR,"expected.picard-2.15.0.sorted.chr1.1-1K.unmarkedDups.markDuplicate.metrics")}, + {new File(TEST_DATA_DIR, "optical_dupes.bam"), new File(TEST_DATA_DIR,"expected.picard-2.15.0.sorted.optical_dupes.markDuplicate.metrics")}, + {new File(TEST_DATA_DIR, "inputSingleLibrarySolexa16404.bam"), new File(TEST_DATA_DIR,"expected.picard-2.15.0.sorted.inputSingleLibrarySolexa16404.metrics")}, }; } @@ -647,6 +706,19 @@ public void testMDOrder(final File input, final File expectedOutput) throws Exce testMDOrderImpl(input, expectedOutput, ""); } + @Test + public void testTwoMappedPairsWithSoftClippingFirstOfPairOnly() { + final AbstractMarkDuplicatesTester tester = getTester(); + // NB: no duplicates + // 5'1: 2, 5'2:46+73M=118 + // 5'1: 2, 5'2:51+68M=118 + tester.getSamRecordSetBuilder().setReadLength(76); + tester.addMappedPair(0, 12, 46, false, false, "6S42M28S", "3S73M", true, 50); // only add the first one + // NB: this next record should not be a duplicate in MarkDuplicatesGATK + tester.addMappedPair(0, 12, 51, false, false, "6S42M28S", "8S68M", true, 50); // only add the first one + tester.runTest(); + } + protected void testMDOrderImpl(final File input, final File expectedOutput, final String extraArgs) throws Exception { final File metricsFile = createTempFile("markdups_metrics", ".txt"); final File outputFile = createTempFile("markdups", ".bam"); diff --git a/src/main/java/org/broadinstitute/hellbender/utils/test/testers/AbstractMarkDuplicatesTester.java b/src/test/java/org/broadinstitute/hellbender/utils/test/testers/AbstractMarkDuplicatesTester.java similarity index 86% rename from src/main/java/org/broadinstitute/hellbender/utils/test/testers/AbstractMarkDuplicatesTester.java rename to src/test/java/org/broadinstitute/hellbender/utils/test/testers/AbstractMarkDuplicatesTester.java index 361476ba8f6..81a8b67b4f1 100644 --- a/src/main/java/org/broadinstitute/hellbender/utils/test/testers/AbstractMarkDuplicatesTester.java +++ b/src/test/java/org/broadinstitute/hellbender/utils/test/testers/AbstractMarkDuplicatesTester.java @@ -9,8 +9,8 @@ import htsjdk.samtools.util.TestUtil; import org.broadinstitute.hellbender.cmdline.CommandLineProgram; import org.broadinstitute.hellbender.cmdline.StandardArgumentDefinitions; -import org.broadinstitute.hellbender.utils.read.markduplicates.DuplicationMetrics; import org.testng.Assert; +import picard.sam.DuplicationMetrics; import java.io.File; import java.io.FileNotFoundException; @@ -20,14 +20,17 @@ * This class is an extension of SamFileTester used to test AbstractMarkDuplicatesCommandLineProgram's with SAM files generated on the fly. * This performs the underlying tests defined by classes such as AbstractMarkDuplicatesCommandLineProgramTest. */ -// TODO: it should live in the test sources, as the implementations - not required to be packed in the testing framework public abstract class AbstractMarkDuplicatesTester extends SamFileTester { final private File metricsFile; final DuplicationMetrics expectedMetrics; - public AbstractMarkDuplicatesTester(final ScoringStrategy duplicateScoringStrategy) { - super(50, true, SAMRecordSetBuilder.DEFAULT_CHROMOSOME_LENGTH, duplicateScoringStrategy); + public AbstractMarkDuplicatesTester(final ScoringStrategy duplicateScoringStrategy, final SAMFileHeader.SortOrder sortOrder) { + this(duplicateScoringStrategy, sortOrder, true); + } + + public AbstractMarkDuplicatesTester(final ScoringStrategy duplicateScoringStrategy, final SAMFileHeader.SortOrder sortOrder, final boolean recordNeedSorting) { + super(50, true, SAMRecordSetBuilder.DEFAULT_CHROMOSOME_LENGTH, duplicateScoringStrategy, sortOrder, recordNeedSorting); expectedMetrics = new DuplicationMetrics(); expectedMetrics.READ_PAIR_OPTICAL_DUPLICATES = 0; @@ -37,14 +40,12 @@ public AbstractMarkDuplicatesTester(final ScoringStrategy duplicateScoringStrate addArg("--"+StandardArgumentDefinitions.DUPLICATE_SCORING_STRATEGY_LONG_NAME, duplicateScoringStrategy.name()); } - public AbstractMarkDuplicatesTester() { - super(50, true, SAMRecordSetBuilder.DEFAULT_CHROMOSOME_LENGTH); - - expectedMetrics = new DuplicationMetrics(); - expectedMetrics.READ_PAIR_OPTICAL_DUPLICATES = 0; + public AbstractMarkDuplicatesTester(final ScoringStrategy duplicateScoringStrategy) { + this(duplicateScoringStrategy, SAMFileHeader.SortOrder.coordinate); + } - metricsFile = new File(getOutputDir(), "metrics.txt"); - addArg("--"+StandardArgumentDefinitions.METRICS_FILE_LONG_NAME, metricsFile.getAbsolutePath()); + public AbstractMarkDuplicatesTester() { + this(SAMRecordSetBuilder.DEFAULT_DUPLICATE_SCORING_STRATEGY); } public File getMetricsFile() { @@ -64,7 +65,9 @@ private void updateExpectedDuplicationMetrics() { final CloseableIterator inputRecordIterator = this.getRecordIterator(); while (inputRecordIterator.hasNext()) { final SAMRecord record = inputRecordIterator.next(); - if (!record.isSecondaryOrSupplementary()) { + if (record.isSecondaryOrSupplementary()) { + ++expectedMetrics.SECONDARY_OR_SUPPLEMENTARY_RDS; + } else { final String key = samRecordToDuplicatesFlagsKey(record); if (!this.duplicateFlags.containsKey(key)) { System.err.println("DOES NOT CONTAIN KEY: " + key); @@ -74,12 +77,10 @@ private void updateExpectedDuplicationMetrics() { // First bring the simple metricsFile up to date if (record.getReadUnmappedFlag()) { ++expectedMetrics.UNMAPPED_READS; - } - else if (!record.getReadPairedFlag() || record.getMateUnmappedFlag()) { + } else if (!record.getReadPairedFlag() || record.getMateUnmappedFlag()) { ++expectedMetrics.UNPAIRED_READS_EXAMINED; if (isDuplicate) ++expectedMetrics.UNPAIRED_READ_DUPLICATES; - } - else { + } else { ++expectedMetrics.READ_PAIRS_EXAMINED; // will need to be divided by 2 at the end if (isDuplicate) ++expectedMetrics.READ_PAIR_DUPLICATES; // will need to be divided by 2 at the end } @@ -87,7 +88,7 @@ else if (!record.getReadPairedFlag() || record.getMateUnmappedFlag()) { } expectedMetrics.READ_PAIR_DUPLICATES = expectedMetrics.READ_PAIR_DUPLICATES / 2; expectedMetrics.READ_PAIRS_EXAMINED = expectedMetrics.READ_PAIRS_EXAMINED / 2; - expectedMetrics.calculateDerivedMetrics(); + expectedMetrics.calculateDerivedFields(); // Have to run this Double value through the same format/parsing operations as during a file write/read expectedMetrics.PERCENT_DUPLICATION = formatter.parseDouble(formatter.format(expectedMetrics.PERCENT_DUPLICATION)); @@ -132,8 +133,9 @@ public void test() { catch (final FileNotFoundException ex) { System.err.println("Metrics file not found: " + ex); } - // NB: Test writes an initial metrics line with a null entry for LIBRARY and 0 values for all metrics. So we find the first non-null one - final DuplicationMetrics observedMetrics = metricsOutput.getMetrics().stream().filter(metric -> metric.LIBRARY != null).findFirst().get(); + + Assert.assertEquals(metricsOutput.getMetrics().size(), 1); + final DuplicationMetrics observedMetrics = metricsOutput.getMetrics().get(0); Assert.assertEquals(observedMetrics.UNPAIRED_READS_EXAMINED, expectedMetrics.UNPAIRED_READS_EXAMINED, "UNPAIRED_READS_EXAMINED does not match expected"); Assert.assertEquals(observedMetrics.READ_PAIRS_EXAMINED, expectedMetrics.READ_PAIRS_EXAMINED, "READ_PAIRS_EXAMINED does not match expected"); Assert.assertEquals(observedMetrics.UNMAPPED_READS, expectedMetrics.UNMAPPED_READS, "UNMAPPED_READS does not match expected"); @@ -150,6 +152,7 @@ public void test() { expectedMetrics.ESTIMATED_LIBRARY_SIZE = 0L; } Assert.assertEquals(observedMetrics.ESTIMATED_LIBRARY_SIZE, expectedMetrics.ESTIMATED_LIBRARY_SIZE, "ESTIMATED_LIBRARY_SIZE does not match expected"); + Assert.assertEquals(observedMetrics.SECONDARY_OR_SUPPLEMENTARY_RDS, expectedMetrics.SECONDARY_OR_SUPPLEMENTARY_RDS, "SECONDARY_OR_SUPPLEMENTARY_RDS does not match expected"); } finally { TestUtil.recursiveDelete(getOutputDir()); } diff --git a/src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/expected.chr1.1-1K.unmarkedDups.markDuplicate.metrics b/src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/expected.chr1.1-1K.unmarkedDups.markDuplicate.metrics deleted file mode 100644 index e00b792142d..00000000000 --- a/src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/expected.chr1.1-1K.unmarkedDups.markDuplicate.metrics +++ /dev/null @@ -1,33 +0,0 @@ -## htsjdk.samtools.metrics.StringHeader -# picard.sam.markduplicates.MarkDuplicates INPUT=[src/test/resources/org/broadinstitute/hellbender/tools/picard/sam/MarkDuplicates/example.chr1.1-1K.unmarkedDups.bam] OUTPUT=out.bam.picard METRICS_FILE=expected_duplicate_metrics.txt.picard MAX_SEQUENCES_FOR_DISK_READ_ENDS_MAP=50000 MAX_FILE_HANDLES_FOR_READ_ENDS_MAP=8000 SORTING_COLLECTION_SIZE_RATIO=0.25 PROGRAM_RECORD_ID=MarkDuplicates PROGRAM_GROUP_NAME=MarkDuplicates REMOVE_DUPLICATES=false ASSUME_SORTED=false DUPLICATE_SCORING_STRATEGY=SUM_OF_BASE_QUALITIES READ_NAME_REGEX=[a-zA-Z0-9]+:[0-9]:([0-9]+):([0-9]+):([0-9]+).* OPTICAL_DUPLICATE_PIXEL_DISTANCE=100 VERBOSITY=INFO QUIET=false VALIDATION_STRINGENCY=STRICT COMPRESSION_LEVEL=5 MAX_RECORDS_IN_RAM=500000 CREATE_INDEX=false CREATE_MD5_FILE=false -## htsjdk.samtools.metrics.StringHeader -# Started on: Thu Nov 19 15:30:43 EST 2015 - -## METRICS CLASS picard.sam.DuplicationMetrics -LIBRARY UNPAIRED_READS_EXAMINED READ_PAIRS_EXAMINED UNMAPPED_READS UNPAIRED_READ_DUPLICATES READ_PAIR_DUPLICATES READ_PAIR_OPTICAL_DUPLICATES PERCENT_DUPLICATION ESTIMATED_LIBRARY_SIZE -Solexa-16398 0 0 0 0 0 0 ? -Solexa-16399 0 0 0 0 0 0 ? -Solexa-16400 0 0 0 0 0 0 ? -Solexa-16402 0 0 0 0 0 0 ? -Solexa-16403 0 0 0 0 0 0 ? -Solexa-16404 3 9 3 0 2 0 0.190476 17 -Solexa-16406 1 10 1 0 0 0 0 -Solexa-16407 0 0 0 0 0 0 ? -Solexa-16408 0 0 0 0 0 0 ? -Solexa-16410 0 0 0 0 0 0 ? -Solexa-16411 0 0 0 0 0 0 ? -Solexa-16412 3 6 3 0 1 0 0.133333 15 -Solexa-16414 0 0 0 0 0 0 ? -Solexa-16415 0 0 0 0 0 0 ? -Solexa-16416 2 2 2 0 0 0 0 -Solexa-16418 0 0 0 0 0 0 ? -Solexa-16419 4 4 4 0 0 0 0 -Solexa-16420 0 0 0 0 0 0 ? -Solexa-16421 0 0 0 0 0 0 ? -Solexa-16422 0 0 0 0 0 0 ? -Solexa-16423 0 0 0 0 0 0 ? -Solexa-16424 0 0 0 0 0 0 ? -Solexa-16425 0 0 0 0 0 0 ? -Solexa-16426 0 0 0 0 0 0 ? - - diff --git a/src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/expected.optical_dupes.markDuplicate.metrics b/src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/expected.optical_dupes.markDuplicate.metrics deleted file mode 100644 index 1efbfd92bd0..00000000000 --- a/src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/expected.optical_dupes.markDuplicate.metrics +++ /dev/null @@ -1,10 +0,0 @@ -## htsjdk.samtools.metrics.StringHeader -# picard.sam.markduplicates.MarkDuplicates INPUT=[src/test/resources/org/broadinstitute/hellbender/tools/picard/sam/MarkDuplicates/optical_dupes.bam] OUTPUT=o.bam METRICS_FILE=optical_dupes.metrics MAX_SEQUENCES_FOR_DISK_READ_ENDS_MAP=50000 MAX_FILE_HANDLES_FOR_READ_ENDS_MAP=8000 SORTING_COLLECTION_SIZE_RATIO=0.25 PROGRAM_RECORD_ID=MarkDuplicates PROGRAM_GROUP_NAME=MarkDuplicates REMOVE_DUPLICATES=false ASSUME_SORTED=false DUPLICATE_SCORING_STRATEGY=SUM_OF_BASE_QUALITIES READ_NAME_REGEX=[a-zA-Z0-9]+:[0-9]:([0-9]+):([0-9]+):([0-9]+).* OPTICAL_DUPLICATE_PIXEL_DISTANCE=100 VERBOSITY=INFO QUIET=false VALIDATION_STRINGENCY=STRICT COMPRESSION_LEVEL=5 MAX_RECORDS_IN_RAM=500000 CREATE_INDEX=false CREATE_MD5_FILE=false -## htsjdk.samtools.metrics.StringHeader -# Started on: Fri Nov 20 12:00:51 EST 2015 - -## METRICS CLASS picard.sam.DuplicationMetrics -LIBRARY UNPAIRED_READS_EXAMINED READ_PAIRS_EXAMINED UNMAPPED_READS UNPAIRED_READ_DUPLICATES READ_PAIR_DUPLICATES READ_PAIR_OPTICAL_DUPLICATES PERCENT_DUPLICATION ESTIMATED_LIBRARY_SIZE -mylib 0 2 0 0 1 1 0.5 - - diff --git a/src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/expected.picard-2.15.0.sorted.chr1.1-1K.unmarkedDups.markDuplicate.metrics b/src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/expected.picard-2.15.0.sorted.chr1.1-1K.unmarkedDups.markDuplicate.metrics new file mode 100644 index 00000000000..6368abd7c4f --- /dev/null +++ b/src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/expected.picard-2.15.0.sorted.chr1.1-1K.unmarkedDups.markDuplicate.metrics @@ -0,0 +1,33 @@ +## htsjdk.samtools.metrics.StringHeader +# MarkDuplicates INPUT=[example.chr1.1-1K.unmarkedDups.bam] OUTPUT=out.bam METRICS_FILE=expected.picard-2.15.0.sorted.chr1.1-1K.unmarkedDups.markDuplicate.metrics MAX_SEQUENCES_FOR_DISK_READ_ENDS_MAP=50000 MAX_FILE_HANDLES_FOR_READ_ENDS_MAP=8000 SORTING_COLLECTION_SIZE_RATIO=0.25 TAG_DUPLICATE_SET_MEMBERS=false REMOVE_SEQUENCING_DUPLICATES=false TAGGING_POLICY=DontTag CLEAR_DT=true ADD_PG_TAG_TO_READS=true REMOVE_DUPLICATES=false ASSUME_SORTED=false DUPLICATE_SCORING_STRATEGY=SUM_OF_BASE_QUALITIES PROGRAM_RECORD_ID=MarkDuplicates PROGRAM_GROUP_NAME=MarkDuplicates READ_NAME_REGEX= OPTICAL_DUPLICATE_PIXEL_DISTANCE=100 MAX_OPTICAL_DUPLICATE_SET_SIZE=300000 VERBOSITY=INFO QUIET=false VALIDATION_STRINGENCY=STRICT COMPRESSION_LEVEL=5 MAX_RECORDS_IN_RAM=500000 CREATE_INDEX=false CREATE_MD5_FILE=false GA4GH_CLIENT_SECRETS=client_secrets.json USE_JDK_DEFLATER=false USE_JDK_INFLATER=false +## htsjdk.samtools.metrics.StringHeader +# Started on: Tue Dec 05 14:51:11 EST 2017 + +## METRICS CLASS picard.sam.DuplicationMetrics +LIBRARY UNPAIRED_READS_EXAMINED READ_PAIRS_EXAMINED SECONDARY_OR_SUPPLEMENTARY_RDS UNMAPPED_READS UNPAIRED_READ_DUPLICATES READ_PAIR_DUPLICATES READ_PAIR_OPTICAL_DUPLICATES PERCENT_DUPLICATION ESTIMATED_LIBRARY_SIZE +Solexa-16398 0 0 0 0 0 0 0 ? +Solexa-16399 0 0 0 0 0 0 0 ? +Solexa-16400 0 0 0 0 0 0 0 ? +Solexa-16402 0 0 0 0 0 0 0 ? +Solexa-16403 0 0 0 0 0 0 0 ? +Solexa-16404 3 9 0 3 0 2 0 0.190476 17 +Solexa-16406 1 10 0 1 0 0 0 0 +Solexa-16407 0 0 0 0 0 0 0 ? +Solexa-16408 0 0 0 0 0 0 0 ? +Solexa-16410 0 0 0 0 0 0 0 ? +Solexa-16411 0 0 0 0 0 0 0 ? +Solexa-16412 3 6 0 3 0 1 0 0.133333 15 +Solexa-16414 0 0 0 0 0 0 0 ? +Solexa-16415 0 0 0 0 0 0 0 ? +Solexa-16416 2 2 0 2 0 0 0 0 +Solexa-16418 0 0 0 0 0 0 0 ? +Solexa-16419 4 4 0 4 0 0 0 0 +Solexa-16420 0 0 0 0 0 0 0 ? +Solexa-16421 0 0 0 0 0 0 0 ? +Solexa-16422 0 0 0 0 0 0 0 ? +Solexa-16423 0 0 0 0 0 0 0 ? +Solexa-16424 0 0 0 0 0 0 0 ? +Solexa-16425 0 0 0 0 0 0 0 ? +Solexa-16426 0 0 0 0 0 0 0 ? + + diff --git a/src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/expected.inputSingleLibrarySolexa16404.metrics b/src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/expected.picard-2.15.0.sorted.inputSingleLibrarySolexa16404.metrics similarity index 55% rename from src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/expected.inputSingleLibrarySolexa16404.metrics rename to src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/expected.picard-2.15.0.sorted.inputSingleLibrarySolexa16404.metrics index 211cc0e7d07..9efcca95d3e 100644 --- a/src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/expected.inputSingleLibrarySolexa16404.metrics +++ b/src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/expected.picard-2.15.0.sorted.inputSingleLibrarySolexa16404.metrics @@ -1,11 +1,11 @@ ## htsjdk.samtools.metrics.StringHeader -# picard.sam.markduplicates.MarkDuplicates INPUT=[inputSingleLibrarySolexa16404.bam] OUTPUT=inputSingleLibrarySolexa16404.md.bam METRICS_FILE=inputSingleLibrarySolexa16404.metrics MAX_SEQUENCES_FOR_DISK_READ_ENDS_MAP=50000 MAX_FILE_HANDLES_FOR_READ_ENDS_MAP=8000 SORTING_COLLECTION_SIZE_RATIO=0.25 PROGRAM_RECORD_ID=MarkDuplicates PROGRAM_GROUP_NAME=MarkDuplicates REMOVE_DUPLICATES=false ASSUME_SORTED=false DUPLICATE_SCORING_STRATEGY=SUM_OF_BASE_QUALITIES READ_NAME_REGEX=[a-zA-Z0-9]+:[0-9]:([0-9]+):([0-9]+):([0-9]+).* OPTICAL_DUPLICATE_PIXEL_DISTANCE=100 VERBOSITY=INFO QUIET=false VALIDATION_STRINGENCY=STRICT COMPRESSION_LEVEL=5 MAX_RECORDS_IN_RAM=500000 CREATE_INDEX=false CREATE_MD5_FILE=false GA4GH_CLIENT_SECRETS=client_secrets.json +# MarkDuplicates INPUT=[inputSingleLibrarySolexa16404.bam] OUTPUT=out.bam METRICS_FILE=expected.picard-2.15.0.sorted.inputSingleLibrarySolexa16404.metrics MAX_SEQUENCES_FOR_DISK_READ_ENDS_MAP=50000 MAX_FILE_HANDLES_FOR_READ_ENDS_MAP=8000 SORTING_COLLECTION_SIZE_RATIO=0.25 TAG_DUPLICATE_SET_MEMBERS=false REMOVE_SEQUENCING_DUPLICATES=false TAGGING_POLICY=DontTag CLEAR_DT=true ADD_PG_TAG_TO_READS=true REMOVE_DUPLICATES=false ASSUME_SORTED=false DUPLICATE_SCORING_STRATEGY=SUM_OF_BASE_QUALITIES PROGRAM_RECORD_ID=MarkDuplicates PROGRAM_GROUP_NAME=MarkDuplicates READ_NAME_REGEX= OPTICAL_DUPLICATE_PIXEL_DISTANCE=100 MAX_OPTICAL_DUPLICATE_SET_SIZE=300000 VERBOSITY=INFO QUIET=false VALIDATION_STRINGENCY=STRICT COMPRESSION_LEVEL=5 MAX_RECORDS_IN_RAM=500000 CREATE_INDEX=false CREATE_MD5_FILE=false GA4GH_CLIENT_SECRETS=client_secrets.json USE_JDK_DEFLATER=false USE_JDK_INFLATER=false ## htsjdk.samtools.metrics.StringHeader -# Started on: Fri Nov 20 13:26:46 EST 2015 +# Started on: Tue Dec 05 14:30:09 EST 2017 ## METRICS CLASS picard.sam.DuplicationMetrics -LIBRARY UNPAIRED_READS_EXAMINED READ_PAIRS_EXAMINED UNMAPPED_READS UNPAIRED_READ_DUPLICATES READ_PAIR_DUPLICATES READ_PAIR_OPTICAL_DUPLICATES PERCENT_DUPLICATION ESTIMATED_LIBRARY_SIZE -Solexa-16404 3 9 3 0 2 0 0.190476 17 +LIBRARY UNPAIRED_READS_EXAMINED READ_PAIRS_EXAMINED SECONDARY_OR_SUPPLEMENTARY_RDS UNMAPPED_READS UNPAIRED_READ_DUPLICATES READ_PAIR_DUPLICATES READ_PAIR_OPTICAL_DUPLICATES PERCENT_DUPLICATION ESTIMATED_LIBRARY_SIZE +Solexa-16404 3 9 0 3 0 2 0 0.190476 17 ## HISTOGRAM java.lang.Double BIN VALUE diff --git a/src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/expected.picard-2.15.0.sorted.optical_dupes.markDuplicate.metrics b/src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/expected.picard-2.15.0.sorted.optical_dupes.markDuplicate.metrics new file mode 100644 index 00000000000..8ef26dc8429 --- /dev/null +++ b/src/test/resources/org/broadinstitute/hellbender/tools/walkers/MarkDuplicatesGATK/expected.picard-2.15.0.sorted.optical_dupes.markDuplicate.metrics @@ -0,0 +1,10 @@ +## htsjdk.samtools.metrics.StringHeader +# MarkDuplicates INPUT=[optical_dupes.bam] OUTPUT=out.bam METRICS_FILE=expected.picard-2.15.0.sorted.optical_dupes.markDuplicate.metrics MAX_SEQUENCES_FOR_DISK_READ_ENDS_MAP=50000 MAX_FILE_HANDLES_FOR_READ_ENDS_MAP=8000 SORTING_COLLECTION_SIZE_RATIO=0.25 TAG_DUPLICATE_SET_MEMBERS=false REMOVE_SEQUENCING_DUPLICATES=false TAGGING_POLICY=DontTag CLEAR_DT=true ADD_PG_TAG_TO_READS=true REMOVE_DUPLICATES=false ASSUME_SORTED=false DUPLICATE_SCORING_STRATEGY=SUM_OF_BASE_QUALITIES PROGRAM_RECORD_ID=MarkDuplicates PROGRAM_GROUP_NAME=MarkDuplicates READ_NAME_REGEX= OPTICAL_DUPLICATE_PIXEL_DISTANCE=100 MAX_OPTICAL_DUPLICATE_SET_SIZE=300000 VERBOSITY=INFO QUIET=false VALIDATION_STRINGENCY=STRICT COMPRESSION_LEVEL=5 MAX_RECORDS_IN_RAM=500000 CREATE_INDEX=false CREATE_MD5_FILE=false GA4GH_CLIENT_SECRETS=client_secrets.json USE_JDK_DEFLATER=false USE_JDK_INFLATER=false +## htsjdk.samtools.metrics.StringHeader +# Started on: Tue Dec 05 14:51:32 EST 2017 + +## METRICS CLASS picard.sam.DuplicationMetrics +LIBRARY UNPAIRED_READS_EXAMINED READ_PAIRS_EXAMINED SECONDARY_OR_SUPPLEMENTARY_RDS UNMAPPED_READS UNPAIRED_READ_DUPLICATES READ_PAIR_DUPLICATES READ_PAIR_OPTICAL_DUPLICATES PERCENT_DUPLICATION ESTIMATED_LIBRARY_SIZE +mylib 0 2 0 0 0 1 1 0.5 + +