From e72d40517dce26f9a3632005a55ebe1f24d74188 Mon Sep 17 00:00:00 2001 From: shae-selix Date: Mon, 25 Feb 2019 16:49:18 -0800 Subject: [PATCH 01/14] adding Lift enum to EvaluationMetrics --- .../scala/com/salesforce/op/evaluators/EvaluationMetrics.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/com/salesforce/op/evaluators/EvaluationMetrics.scala b/core/src/main/scala/com/salesforce/op/evaluators/EvaluationMetrics.scala index f227cd79b0..b25a8af29f 100644 --- a/core/src/main/scala/com/salesforce/op/evaluators/EvaluationMetrics.scala +++ b/core/src/main/scala/com/salesforce/op/evaluators/EvaluationMetrics.scala @@ -140,6 +140,7 @@ object BinaryClassEvalMetrics extends Enum[ClassificationEvalMetric] { case object FP extends ClassificationEvalMetric("FP", "false positive") case object FN extends ClassificationEvalMetric("FN", "false negative") case object BrierScore extends ClassificationEvalMetric("brierScore", "brier score") + case object LiftMetrics extends ClassificationEvalMetric("liftMetrics", "lift plot") } /** From 155a11372c4d2cc5f1d76b7e619a51f5bd2cb0bb Mon Sep 17 00:00:00 2001 From: shae-selix Date: Tue, 26 Feb 2019 16:58:26 -0800 Subject: [PATCH 02/14] rearchitecting LiftEvaluator for tmog --- .../op/evaluators/LiftEvaluator.scala | 213 ++++++++++++++++++ .../OpBinaryClassificationEvaluator.scala | 11 +- .../selector/ModelSelectorSummaryTest.scala | 3 +- 3 files changed, 223 insertions(+), 4 deletions(-) create mode 100644 core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala diff --git a/core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala b/core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala new file mode 100644 index 0000000000..c49217b6ae --- /dev/null +++ b/core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala @@ -0,0 +1,213 @@ +package com.salesforce.op.evaluators + +import org.apache.spark.rdd.RDD + +/** + * Object to calculate Lift metrics for BinaryClassification problems + * Intended for write-back to core for Scorecard or to Looker + * + * Algorithm for calculating a chart as seen here: + * https://www.kdnuggets.com/2016/03/lift-analysis-data-scientist-secret-weapon.html + */ +object LiftEvaluator { + + /** + * Scoreband name that represents the lift values across + * all scores, 0 to 100 + */ + private[op] val overallScoreband = "overall" + + /** + * Stores basic lift values for a specific band of scores + * + * @param group name / key for score band + * @param lowerBound minimum score represented in lift + * @param upperBound maximum score represented in lift + * @param rate calculated lift value, i.e. # yes / total count + * @param average lift rate across all score bands + * @param totalCount total number of records in score band + * @param yesCount number of yes records in score band + * @param noCount number of no records in score band + */ + case class LiftMetricBand + ( + group: String, + lowerBound: Double, + upperBound: Double, + rate: Double, + average: Double, + totalCount: Long, + yesCount: Long, + noCount: Long + ) + + /** + * Builds Lift Map for serialization, wrapper for liftMap function + * for the DataFrame api + * + * @param holdoutDF DataFrame of scored hold-out data + * @param labelCol column name for labels + * @param scoreCol column name for scores + * @return AutoMLMetrics: Metrics object for serialization + */ + def apply + ( + scoreAndLabels: RDD[(Double, Double)] + ): Seq[LiftMetricBand] = { + liftMetricBands( + scoreAndLabels, + getDefaultScoreBands + ) + } + + /** + * Builds Lift Map for serialization using RDD api + * + * @param labelsAndScores RDD[(Double, Double)] of BinaryClassification (label, score) tuples + * @return Seq of LiftMetricBand containers of Lift calculations + */ + private[op] def liftMetricBands + ( + scoreAndLabels: RDD[(Double, Double)], + getScoreBands: RDD[Double] => Seq[(Double, Double, String)] + ): Seq[LiftMetricBand] = { + val bands = getScoreBands(scoreAndLabels.map{case (score, _) => score}) + val bandedLabels = scoreAndLabels.map { case (score, label) => + (categorizeScoreIntoBand((score, bands)), label) + }.collect { case (Some(band), label) => (band, label) } + val perBandCounts = aggregateBandedLabels(bandedLabels) + val overallRate = overallLiftRate(perBandCounts) + bands.map({ case (lower, upper, band) => + formatLiftMetricBand(lower, upper, band, perBandCounts, overallRate) + }) + } + + /** + * function to return score bands for calculating lift + * Default: 10 equidistant bands for all 0.1 increments + * from 0.0 to 1.0 + * + * @return sequence of (lowerBound, upperBound, bandString) tuples + */ + private[op] def getDefaultScoreBands(scores: RDD[Double]): + Seq[(Double, Double, String)] = + Seq( + (0.0, 0.1, "0-10"), + (0.1, 0.2, "10-20"), + (0.2, 0.3, "20-30"), + (0.3, 0.4, "30-40"), + (0.4, 0.5, "40-50"), + (0.5, 0.6, "50-60"), + (0.6, 0.7, "60-70"), + (0.7, 0.8, "70-80"), + (0.8, 0.9, "80-90"), + (0.9, 1.0, "90-100") + ) + + /** + * PartialFunction. Defined when scores are [0.0, 1.0] + * Places a score Double into a score band based on + * lower and upper bounds + * + * @param score BinaryClassification score Double, [0.0, 1.0] + * @param bands sequence of upper/lower score bands + * @return optional key to describe categorized band, if found + */ + private[op] def categorizeScoreIntoBand: + PartialFunction[(Double, Seq[(Double, Double, String)]), Option[String]] = { + case (score: Double, bands: Seq[(Double, Double, String)]) + if (score >= 0.0) & (score <= 1.0) => + bands.find { case (l, u, _) => + (score >= l) & (score <= u) + } match { + case Some((_, _, bandString)) => Some(bandString) + case None => None + } + } + + /** + * aggregates labels into counts by lift band + * + * @param bandedLabels PairRDD of (bandString, label) + * @return Map of bandString -> (total count, count of positive labels) + */ + private[op] def aggregateBandedLabels + ( + bandedLabels: RDD[(String, Double)] + ): Map[String, (Long, Long)] = { + val countsPerBand = bandedLabels.countByKey() + val truesPerBand = bandedLabels.aggregateByKey(zeroValue = 0.0)( + { case (sum, label) => sum + label }, + { case (sumX, sumY) => sumX + sumY }) + .collectAsMap() + countsPerBand.map { case (band, count) => + band -> (count, truesPerBand.getOrElse(band, 0.0).toLong) + }.toMap + } + + /** + * + * + * @param perBandCounts + * @return + */ + private[op] def overallLiftRate(perBandCounts: Map[String, (Long, Long)]): Double = { + val overallTotalCount = perBandCounts.values.map({case (totalCount, _) => totalCount}).sum + val overallYesCount = perBandCounts.values.map({case (_, yesCount) => yesCount}).sum + overallTotalCount match { + case 0L => Double.NaN + case _ => overallYesCount.toDouble / overallTotalCount + } + } + + /** + * Formats lift data in one band into LiftMetricBand data, + * including lower bound of score band, upper bound, total record + * count per band, and lift (# trues / total) + * + * @param lower lower bound of band + * @param upper upper bound of band + * @param bandString String key of band e.g. "10-20" + * @param perBandCounts calculated total counts and counts of true labels + * @return LiftMetricBand container of metrics + */ + private[op] def formatLiftMetricBand + ( + lower: Double, + upper: Double, + bandString: String, + perBandCounts: Map[String, (Long, Long)], + overallRate: Double + ): LiftMetricBand = { + perBandCounts.get(bandString) match { + case Some((numTotal, numYes)) => { + val lift = numTotal match { + case 0.0 => Double.NaN + case _ => numYes.toDouble / numTotal + } + LiftMetricBand( + group = bandString, + lowerBound = lower, + upperBound = upper, + rate = lift, + average = overallRate, + totalCount = numTotal, + yesCount = numYes, + noCount = numTotal - numYes + ) + } + case None => LiftMetricBand( + group = bandString, + lowerBound = lower, + upperBound = upper, + rate = Double.NaN, + average = overallRate, + totalCount = 0L, + yesCount = 0L, + noCount = 0L + ) + } + } + + +} diff --git a/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala b/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala index f2f27d627d..609e84ad1a 100644 --- a/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala +++ b/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala @@ -34,6 +34,7 @@ import com.fasterxml.jackson.databind.annotation.JsonDeserialize import com.salesforce.op.UID import com.salesforce.op.utils.spark.RichEvaluator._ import com.salesforce.op.evaluators.BinaryClassEvalMetrics._ +import com.salesforce.op.evaluators.LiftEvaluator.LiftMetricBand import org.apache.spark.ml.evaluation.{BinaryClassificationEvaluator, MulticlassClassificationEvaluator} import org.apache.spark.ml.linalg.Vector import org.apache.spark.mllib.evaluation.{MulticlassMetrics, BinaryClassificationMetrics => SparkMLBinaryClassificationMetrics} @@ -82,7 +83,8 @@ private[op] class OpBinaryClassificationEvaluator if (rdd.isEmpty()) { log.warn("The dataset is empty. Returning empty metrics.") - BinaryClassificationMetrics(0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, Seq(), Seq(), Seq(), Seq()) + BinaryClassificationMetrics(0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, + Seq(), Seq(), Seq(), Seq(), Seq()) } else { val multiclassMetrics = new MulticlassMetrics(rdd) val labels = multiclassMetrics.labels @@ -113,10 +115,12 @@ private[op] class OpBinaryClassificationEvaluator val falsePositiveRateByThreshold = sparkMLMetrics.roc().collect().map(_._1).slice(1, thresholds.length + 1) val aUROC = sparkMLMetrics.areaUnderROC() val aUPR = sparkMLMetrics.areaUnderPR() + val liftMetrics = LiftEvaluator(scoreAndLabels) val metrics = BinaryClassificationMetrics( Precision = precision, Recall = recall, F1 = f1, AuROC = aUROC, AuPR = aUPR, Error = error, TP = tp, TN = tn, FP = fp, FN = fn, - thresholds, precisionByThreshold, recallByThreshold, falsePositiveRateByThreshold + thresholds, precisionByThreshold, recallByThreshold, falsePositiveRateByThreshold, + LiftMetrics = liftMetrics ) log.info("Evaluated metrics: {}", metrics.toString) metrics @@ -195,7 +199,8 @@ case class BinaryClassificationMetrics @JsonDeserialize(contentAs = classOf[java.lang.Double]) recallByThreshold: Seq[Double], @JsonDeserialize(contentAs = classOf[java.lang.Double]) - falsePositiveRateByThreshold: Seq[Double] + falsePositiveRateByThreshold: Seq[Double], + LiftMetrics: Seq[LiftMetricBand] ) extends EvaluationMetrics { def rocCurve: Seq[(Double, Double)] = recallByThreshold.zip(falsePositiveRateByThreshold) def prCurve: Seq[(Double, Double)] = precisionByThreshold.zip(recallByThreshold) diff --git a/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala b/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala index 1bd9595542..56ef443a98 100644 --- a/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala +++ b/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala @@ -54,7 +54,8 @@ class ModelSelectorSummaryTest extends FlatSpec with TestSparkContext { validationResults = Seq(ModelEvaluation("test4", "test5", "test6", SingleMetric("test7", 0.1), Map.empty)), trainEvaluation = BinaryClassificationMetrics(Precision = 0.1, Recall = 0.2, F1 = 0.3, AuROC = 0.4, AuPR = 0.5, Error = 0.6, TP = 0.7, TN = 0.8, FP = 0.9, FN = 1.0, thresholds = Seq(1.1), - precisionByThreshold = Seq(1.2), recallByThreshold = Seq(1.3), falsePositiveRateByThreshold = Seq(1.4)), + precisionByThreshold = Seq(1.2), recallByThreshold = Seq(1.3), falsePositiveRateByThreshold = Seq(1.4), + LiftMetrics = Seq()), holdoutEvaluation = Option(RegressionMetrics(RootMeanSquaredError = 1.3, MeanSquaredError = 1.4, R2 = 1.5, MeanAbsoluteError = 1.6)) ) From 0bd5e060fe119f0a8af87243e235a29aafb7ca89 Mon Sep 17 00:00:00 2001 From: shae-selix Date: Wed, 27 Feb 2019 16:59:43 -0800 Subject: [PATCH 03/14] Adding tests --- .../op/evaluators/LiftEvaluator.scala | 27 ++-- .../op/evaluators/LiftEvaluatorTest.scala | 130 ++++++++++++++++++ .../OpBinaryClassificationEvaluatorTest.scala | 20 +++ .../selector/ModelSelectorSummaryTest.scala | 2 +- 4 files changed, 161 insertions(+), 18 deletions(-) create mode 100644 core/src/test/scala/com/salesforce/op/evaluators/LiftEvaluatorTest.scala diff --git a/core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala b/core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala index c49217b6ae..12c2c991d1 100644 --- a/core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala +++ b/core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala @@ -11,12 +11,6 @@ import org.apache.spark.rdd.RDD */ object LiftEvaluator { - /** - * Scoreband name that represents the lift values across - * all scores, 0 to 100 - */ - private[op] val overallScoreband = "overall" - /** * Stores basic lift values for a specific band of scores * @@ -42,13 +36,10 @@ object LiftEvaluator { ) /** - * Builds Lift Map for serialization, wrapper for liftMap function - * for the DataFrame api + * Builds Lift Map for serialization, wrapper for liftMetricBands function * - * @param holdoutDF DataFrame of scored hold-out data - * @param labelCol column name for labels - * @param scoreCol column name for scores - * @return AutoMLMetrics: Metrics object for serialization + * @param scoreAndLabels RDD[(Double, Double)] of BinaryClassification (score, label) tuples + * @return Seq of LiftMetricBand containers of Lift calculations */ def apply ( @@ -63,7 +54,8 @@ object LiftEvaluator { /** * Builds Lift Map for serialization using RDD api * - * @param labelsAndScores RDD[(Double, Double)] of BinaryClassification (label, score) tuples + * @param scoreAndLabels RDD[(Double, Double)] of BinaryClassification (score, label) tuples + * @param getScoreBands function to calculate score bands, potentially using score distribution * @return Seq of LiftMetricBand containers of Lift calculations */ private[op] def liftMetricBands @@ -79,7 +71,7 @@ object LiftEvaluator { val overallRate = overallLiftRate(perBandCounts) bands.map({ case (lower, upper, band) => formatLiftMetricBand(lower, upper, band, perBandCounts, overallRate) - }) + }).sortBy(band => band.lowerBound) } /** @@ -146,10 +138,10 @@ object LiftEvaluator { } /** - * + * calculates a baseline "yes" rate across score bands * * @param perBandCounts - * @return + * @return overall # yes / total records across all bands */ private[op] def overallLiftRate(perBandCounts: Map[String, (Long, Long)]): Double = { val overallTotalCount = perBandCounts.values.map({case (totalCount, _) => totalCount}).sum @@ -169,6 +161,7 @@ object LiftEvaluator { * @param upper upper bound of band * @param bandString String key of band e.g. "10-20" * @param perBandCounts calculated total counts and counts of true labels + * @param overallRate overall Lift rate across all bands * @return LiftMetricBand container of metrics */ private[op] def formatLiftMetricBand @@ -182,7 +175,7 @@ object LiftEvaluator { perBandCounts.get(bandString) match { case Some((numTotal, numYes)) => { val lift = numTotal match { - case 0.0 => Double.NaN + case 0L => Double.NaN case _ => numYes.toDouble / numTotal } LiftMetricBand( diff --git a/core/src/test/scala/com/salesforce/op/evaluators/LiftEvaluatorTest.scala b/core/src/test/scala/com/salesforce/op/evaluators/LiftEvaluatorTest.scala new file mode 100644 index 0000000000..6379737279 --- /dev/null +++ b/core/src/test/scala/com/salesforce/op/evaluators/LiftEvaluatorTest.scala @@ -0,0 +1,130 @@ +package com.salesforce.op.evaluators + +import com.salesforce.op.test.TestSparkContext +import org.junit.runner.RunWith +import org.scalatest.FlatSpec +import org.scalatest.junit.JUnitRunner + +/** + * @author sselix + * @since 218 + */ +@RunWith(classOf[JUnitRunner]) +class LiftEvaluatorTest extends FlatSpec with TestSparkContext { + + lazy val labelSeq = for { + i <- 0 until 10 + j <- 0 until 10 + } yield { + if (j < i) 1.0 + else 0.0 + } + lazy val scoreSeq = (0.01 to 1.0 by 0.01) + + lazy val scores = sc.parallelize(scoreSeq) + + lazy val scoresAndLabels = sc.parallelize(scoreSeq.zip(labelSeq)) + + "LiftEvaluator.getDefaultScoreBands" should "give proper default bands" in { + val bands = LiftEvaluator.getDefaultScoreBands(scores) + bands.head shouldBe (0.0, 0.1, "0-10") + bands.last shouldBe (0.9, 1.0, "90-100") + bands.size shouldBe 10 + } + + "LiftEvaluator.categorizeScoreIntoBand" should "categorize scores into correct bands" in { + val bands = Seq((0.0, 0.5, "A"), (0.5, 0.9, "B")) + LiftEvaluator.categorizeScoreIntoBand(0.3, bands) shouldBe Some("A") + LiftEvaluator.categorizeScoreIntoBand(0.95, bands) shouldBe None + a[MatchError] should be thrownBy { + LiftEvaluator.categorizeScoreIntoBand(-0.1, bands) + } + a[MatchError] should be thrownBy { + LiftEvaluator.categorizeScoreIntoBand(1.1, bands) + } + } + + "LiftEvaluator.aggregateBandedLabels" should "correctly count records within score bands" in { + val bandedLabels = sc.parallelize( + Seq(("A", 1.0), ("A", 0.0), ("B", 0.0), ("B", 0.0)) + ) + val perBandCounts = LiftEvaluator.aggregateBandedLabels(bandedLabels) + val (numTotalA, _) = perBandCounts("A") + val (_, numPositivesB) = perBandCounts("B") + + numTotalA shouldBe 2L + numPositivesB shouldBe 0L + } + + "LiftEvaluator.overallLiftRate" should "calculate an overall rate" in { + val perBandCountsFilled = Map("A" -> (4L, 2L), "B" -> (1L, 0L)) + val perBandCountsEmpty = Map[String, (Long, Long)]() + val overallRateFilled = LiftEvaluator.overallLiftRate(perBandCountsFilled) + val overallRateEmpty = LiftEvaluator.overallLiftRate(perBandCountsEmpty) + overallRateFilled shouldBe 0.4 + overallRateEmpty.isNaN shouldBe true + } + + "LiftEvaluator.formatLiftMetricBand" should "format a LiftMetricBand as required" in { + val perBandCounts = Map("A" -> (4L, 2L)) + val metricBandA = LiftEvaluator.formatLiftMetricBand(0.0, 0.1, "A", perBandCounts, 0.5) + metricBandA.group shouldBe "A" + metricBandA.lowerBound shouldBe 0.0 + metricBandA.upperBound shouldBe 0.1 + metricBandA.rate shouldBe 0.5 + metricBandA.average shouldBe 0.5 + metricBandA.totalCount shouldBe 4L + metricBandA.yesCount shouldBe 2L + metricBandA.noCount shouldBe 2L + + val metricBandB = LiftEvaluator.formatLiftMetricBand(0.1, 0.2, "B", perBandCounts, 0.5) + metricBandB.group shouldBe "B" + metricBandB.lowerBound shouldBe 0.1 + metricBandB.upperBound shouldBe 0.2 + metricBandB.rate.isNaN shouldBe true + metricBandB.average shouldBe 0.5 + metricBandB.totalCount shouldBe 0L + metricBandB.yesCount shouldBe 0L + metricBandB.noCount shouldBe 0L + } + + "LiftEvaluator.liftMetricBands" should "correctly calculate a Seq[LiftMetricBand]" in { + val liftSeq = LiftEvaluator.liftMetricBands(scoresAndLabels, LiftEvaluator.getDefaultScoreBands) + val band010 = liftSeq.find(_.group == "0-10").get + val band90100 = liftSeq.find(_.group == "90-100").get + + band010.rate shouldBe 0.0 + band010.lowerBound shouldBe 0.0 + band010.upperBound shouldBe 0.1 + band010.average shouldBe 0.45 + band90100.rate shouldBe 0.9 + } + + "LiftEvaluator.liftMetricBands" should "correctly give defaults with empty RDD" in { + val liftSeq = LiftEvaluator.liftMetricBands( + sc.parallelize(Seq[(Double, Double)]()), + LiftEvaluator.getDefaultScoreBands + ) + val band010 = liftSeq.find(_.group == "0-10").get + val band90100 = liftSeq.find(_.group == "90-100").get + + band010.rate.isNaN shouldBe true + band010.lowerBound shouldBe 0.0 + band010.upperBound shouldBe 0.1 + band010.average.isNaN shouldBe true + band90100.rate.isNaN shouldBe true + } + + "LiftEvaluator.apply" should "correctly calculate a Seq[LiftMetricBand]" in { + val liftSeq = LiftEvaluator.apply(scoresAndLabels) + val band010 = liftSeq.find(_.group == "0-10").get + val band90100 = liftSeq.find(_.group == "90-100").get + + band010.rate shouldBe 0.0 + band010.lowerBound shouldBe 0.0 + band010.upperBound shouldBe 0.1 + band010.average shouldBe 0.45 + band90100.rate shouldBe 0.9 + } + +} diff --git a/core/src/test/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluatorTest.scala b/core/src/test/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluatorTest.scala index 5fd345064d..3c46c77eb1 100644 --- a/core/src/test/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluatorTest.scala +++ b/core/src/test/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluatorTest.scala @@ -149,6 +149,7 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext val (tp, tn, fp, fn, precision, recall, f1) = getPosNegValues( flattenedData2.select(predValue.name, test_label.name).rdd ) + val overallLiftRate = (tp + fn) / (tp + tn + fp + fn) tp.toDouble shouldBe metrics.TP tn.toDouble shouldBe metrics.TN @@ -159,6 +160,10 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext recall shouldBe metrics.Recall f1 shouldBe metrics.F1 1.0 - sparkMulticlassEvaluator.setMetricName(Error.sparkEntryName).evaluate(flattenedData2) shouldBe metrics.Error + + LiftEvaluator.getDefaultScoreBands(sc.emptyRDD).size shouldBe metrics.LiftMetrics.size + overallLiftRate shouldBe metrics.LiftMetrics.head.average + overallLiftRate shouldBe metrics.LiftMetrics.last.average } it should "evaluate the metrics with one prediction input" in { @@ -169,6 +174,7 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext transformedData2.select(prediction.name, test_label.name).rdd .map( r => Row(r.getMap[String, Double](0).toMap.toPrediction.prediction, r.getDouble(1)) ) ) + val overallLiftRate = (tp + fn) / (tp + tn + fp + fn) tp.toDouble shouldBe metrics.TP tn.toDouble shouldBe metrics.TN @@ -178,6 +184,10 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext metrics.Precision shouldBe precision metrics.Recall shouldBe recall metrics.F1 shouldBe f1 + + LiftEvaluator.getDefaultScoreBands(sc.emptyRDD).size shouldBe metrics.LiftMetrics.size + overallLiftRate shouldBe metrics.LiftMetrics.head.average + overallLiftRate shouldBe metrics.LiftMetrics.last.average } it should "evaluate the metrics on dataset with only the label and prediction 0" in { @@ -194,6 +204,11 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext metricsZero.Precision shouldBe 0.0 metricsZero.Recall shouldBe 0.0 metricsZero.Error shouldBe 0.0 + + metricsZero.LiftMetrics.head.rate shouldBe 0.0 + metricsZero.LiftMetrics.head.yesCount shouldBe 0L + metricsZero.LiftMetrics.head.noCount shouldBe 1L + metricsZero.LiftMetrics.tail.head.rate.isNaN shouldBe true } @@ -210,6 +225,11 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext metricsOne.Precision shouldBe 1.0 metricsOne.Recall shouldBe 1.0 metricsOne.Error shouldBe 0.0 + + metricsOne.LiftMetrics.head.rate shouldBe 1.0 + metricsOne.LiftMetrics.head.yesCount shouldBe 1L + metricsOne.LiftMetrics.head.noCount shouldBe 0L + metricsOne.LiftMetrics.tail.head.rate.isNaN shouldBe true } private def getPosNegValues(rdd: RDD[Row]): (Double, Double, Double, Double, Double, Double, Double) = { diff --git a/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala b/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala index 56ef443a98..9c16645b6a 100644 --- a/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala +++ b/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala @@ -55,7 +55,7 @@ class ModelSelectorSummaryTest extends FlatSpec with TestSparkContext { trainEvaluation = BinaryClassificationMetrics(Precision = 0.1, Recall = 0.2, F1 = 0.3, AuROC = 0.4, AuPR = 0.5, Error = 0.6, TP = 0.7, TN = 0.8, FP = 0.9, FN = 1.0, thresholds = Seq(1.1), precisionByThreshold = Seq(1.2), recallByThreshold = Seq(1.3), falsePositiveRateByThreshold = Seq(1.4), - LiftMetrics = Seq()), + LiftMetrics = Seq(LiftEvaluator.LiftMetricBand("0-10", 0.0, 0.1, 0.1, 0.5, 10L, 1L, 9L))), holdoutEvaluation = Option(RegressionMetrics(RootMeanSquaredError = 1.3, MeanSquaredError = 1.4, R2 = 1.5, MeanAbsoluteError = 1.6)) ) From 3350d747f6f5afc80e761771dbd88e2eca92a6d1 Mon Sep 17 00:00:00 2001 From: shae-selix Date: Wed, 27 Feb 2019 17:28:28 -0800 Subject: [PATCH 04/14] LiftEvaluator cleaning up docs and scalastyle --- .../op/evaluators/EvaluationMetrics.scala | 1 - .../op/evaluators/LiftEvaluator.scala | 169 +++++++++++------- .../OpBinaryClassificationEvaluator.scala | 3 +- .../op/evaluators/LiftEvaluatorTest.scala | 39 +++- 4 files changed, 135 insertions(+), 77 deletions(-) diff --git a/core/src/main/scala/com/salesforce/op/evaluators/EvaluationMetrics.scala b/core/src/main/scala/com/salesforce/op/evaluators/EvaluationMetrics.scala index b25a8af29f..f227cd79b0 100644 --- a/core/src/main/scala/com/salesforce/op/evaluators/EvaluationMetrics.scala +++ b/core/src/main/scala/com/salesforce/op/evaluators/EvaluationMetrics.scala @@ -140,7 +140,6 @@ object BinaryClassEvalMetrics extends Enum[ClassificationEvalMetric] { case object FP extends ClassificationEvalMetric("FP", "false positive") case object FN extends ClassificationEvalMetric("FN", "false negative") case object BrierScore extends ClassificationEvalMetric("brierScore", "brier score") - case object LiftMetrics extends ClassificationEvalMetric("liftMetrics", "lift plot") } /** diff --git a/core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala b/core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala index 12c2c991d1..dfcb2ee9b0 100644 --- a/core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala +++ b/core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala @@ -1,28 +1,58 @@ +/* + * Copyright (c) 2017, Salesforce.com, Inc. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * * Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * * Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * * Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + */ + package com.salesforce.op.evaluators import org.apache.spark.rdd.RDD /** - * Object to calculate Lift metrics for BinaryClassification problems - * Intended for write-back to core for Scorecard or to Looker - * - * Algorithm for calculating a chart as seen here: - * https://www.kdnuggets.com/2016/03/lift-analysis-data-scientist-secret-weapon.html - */ + * Object to calculate Lift metrics for BinaryClassification problems + * Intended to build a Lift Plot. + * + * Algorithm for calculating a chart as seen here: + * https://www.kdnuggets.com/2016/03/lift-analysis-data-scientist-secret-weapon.html + */ object LiftEvaluator { /** - * Stores basic lift values for a specific band of scores - * - * @param group name / key for score band - * @param lowerBound minimum score represented in lift - * @param upperBound maximum score represented in lift - * @param rate calculated lift value, i.e. # yes / total count - * @param average lift rate across all score bands - * @param totalCount total number of records in score band - * @param yesCount number of yes records in score band - * @param noCount number of no records in score band - */ + * Stores basic lift values for a specific band of scores + * + * @param group name / key for score band + * @param lowerBound minimum score represented in lift + * @param upperBound maximum score represented in lift + * @param rate calculated lift value, i.e. # yes / total count + * @param average lift rate across all score bands + * @param totalCount total number of records in score band + * @param yesCount number of yes records in score band + * @param noCount number of no records in score band + */ case class LiftMetricBand ( group: String, @@ -36,11 +66,12 @@ object LiftEvaluator { ) /** - * Builds Lift Map for serialization, wrapper for liftMetricBands function - * - * @param scoreAndLabels RDD[(Double, Double)] of BinaryClassification (score, label) tuples - * @return Seq of LiftMetricBand containers of Lift calculations - */ + * Builds Seq[LiftMetricBand] for BinaryClassificationMetrics, calls liftMetricBands function + * with default score bands function + * + * @param scoreAndLabels RDD[(Double, Double)] of BinaryClassification (score, label) tuples + * @return Seq of LiftMetricBand containers of Lift calculations + */ def apply ( scoreAndLabels: RDD[(Double, Double)] @@ -52,18 +83,18 @@ object LiftEvaluator { } /** - * Builds Lift Map for serialization using RDD api - * - * @param scoreAndLabels RDD[(Double, Double)] of BinaryClassification (score, label) tuples - * @param getScoreBands function to calculate score bands, potentially using score distribution - * @return Seq of LiftMetricBand containers of Lift calculations - */ + * Builds Seq of LiftMetricBand using RDD api + * + * @param scoreAndLabels RDD[(Double, Double)] of BinaryClassification (score, label) tuples + * @param getScoreBands function to calculate score bands, potentially using score distribution + * @return Seq of LiftMetricBand containers of Lift calculations + */ private[op] def liftMetricBands ( scoreAndLabels: RDD[(Double, Double)], getScoreBands: RDD[Double] => Seq[(Double, Double, String)] ): Seq[LiftMetricBand] = { - val bands = getScoreBands(scoreAndLabels.map{case (score, _) => score}) + val bands = getScoreBands(scoreAndLabels.map { case (score, _) => score }) val bandedLabels = scoreAndLabels.map { case (score, label) => (categorizeScoreIntoBand((score, bands)), label) }.collect { case (Some(band), label) => (band, label) } @@ -75,14 +106,15 @@ object LiftEvaluator { } /** - * function to return score bands for calculating lift - * Default: 10 equidistant bands for all 0.1 increments - * from 0.0 to 1.0 - * - * @return sequence of (lowerBound, upperBound, bandString) tuples - */ + * function to return score bands for calculating lift + * Default: 10 equidistant bands for all 0.1 increments + * from 0.0 to 1.0 + * + * @param scores RDD of scores. unused in this function + * @return sequence of (lowerBound, upperBound, bandString) tuples + */ private[op] def getDefaultScoreBands(scores: RDD[Double]): - Seq[(Double, Double, String)] = + Seq[(Double, Double, String)] = Seq( (0.0, 0.1, "0-10"), (0.1, 0.2, "10-20"), @@ -97,16 +129,16 @@ object LiftEvaluator { ) /** - * PartialFunction. Defined when scores are [0.0, 1.0] - * Places a score Double into a score band based on - * lower and upper bounds - * - * @param score BinaryClassification score Double, [0.0, 1.0] - * @param bands sequence of upper/lower score bands - * @return optional key to describe categorized band, if found - */ + * PartialFunction. Defined when scores are [0.0, 1.0] + * Places a score Double into a score band based on + * lower and upper bounds + * + * @param score BinaryClassification score Double, [0.0, 1.0] + * @param bands sequence of upper/lower score bands + * @return optional key to describe categorized band, if found + */ private[op] def categorizeScoreIntoBand: - PartialFunction[(Double, Seq[(Double, Double, String)]), Option[String]] = { + PartialFunction[(Double, Seq[(Double, Double, String)]), Option[String]] = { case (score: Double, bands: Seq[(Double, Double, String)]) if (score >= 0.0) & (score <= 1.0) => bands.find { case (l, u, _) => @@ -118,11 +150,11 @@ object LiftEvaluator { } /** - * aggregates labels into counts by lift band - * - * @param bandedLabels PairRDD of (bandString, label) - * @return Map of bandString -> (total count, count of positive labels) - */ + * aggregates labels into counts by lift band + * + * @param bandedLabels PairRDD of (bandString, label) + * @return Map of bandString -> (total count, count of positive labels) + */ private[op] def aggregateBandedLabels ( bandedLabels: RDD[(String, Double)] @@ -138,14 +170,14 @@ object LiftEvaluator { } /** - * calculates a baseline "yes" rate across score bands - * - * @param perBandCounts - * @return overall # yes / total records across all bands - */ + * calculates a baseline "yes" rate across score bands + * + * @param perBandCounts + * @return overall # yes / total records across all bands + */ private[op] def overallLiftRate(perBandCounts: Map[String, (Long, Long)]): Double = { - val overallTotalCount = perBandCounts.values.map({case (totalCount, _) => totalCount}).sum - val overallYesCount = perBandCounts.values.map({case (_, yesCount) => yesCount}).sum + val overallTotalCount = perBandCounts.values.map({ case (totalCount, _) => totalCount }).sum + val overallYesCount = perBandCounts.values.map({ case (_, yesCount) => yesCount }).sum overallTotalCount match { case 0L => Double.NaN case _ => overallYesCount.toDouble / overallTotalCount @@ -153,17 +185,17 @@ object LiftEvaluator { } /** - * Formats lift data in one band into LiftMetricBand data, - * including lower bound of score band, upper bound, total record - * count per band, and lift (# trues / total) - * - * @param lower lower bound of band - * @param upper upper bound of band - * @param bandString String key of band e.g. "10-20" - * @param perBandCounts calculated total counts and counts of true labels - * @param overallRate overall Lift rate across all bands - * @return LiftMetricBand container of metrics - */ + * Formats lift data in one band into LiftMetricBand data, + * including lower bound of score band, upper bound, total record + * count per band, and lift (# trues / total) + * + * @param lower lower bound of band + * @param upper upper bound of band + * @param bandString String key of band e.g. "10-20" + * @param perBandCounts calculated total counts and counts of true labels + * @param overallRate overall Lift rate across all bands + * @return LiftMetricBand container of metrics + */ private[op] def formatLiftMetricBand ( lower: Double, @@ -202,5 +234,4 @@ object LiftEvaluator { } } - } diff --git a/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala b/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala index 609e84ad1a..3cd5f90ce4 100644 --- a/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala +++ b/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala @@ -115,7 +115,7 @@ private[op] class OpBinaryClassificationEvaluator val falsePositiveRateByThreshold = sparkMLMetrics.roc().collect().map(_._1).slice(1, thresholds.length + 1) val aUROC = sparkMLMetrics.areaUnderROC() val aUPR = sparkMLMetrics.areaUnderPR() - val liftMetrics = LiftEvaluator(scoreAndLabels) + val liftMetrics = LiftEvaluator(scoreAndLabels = scoreAndLabels) val metrics = BinaryClassificationMetrics( Precision = precision, Recall = recall, F1 = f1, AuROC = aUROC, AuPR = aUPR, Error = error, TP = tp, TN = tn, FP = fp, FN = fn, @@ -200,6 +200,7 @@ case class BinaryClassificationMetrics recallByThreshold: Seq[Double], @JsonDeserialize(contentAs = classOf[java.lang.Double]) falsePositiveRateByThreshold: Seq[Double], + @JsonDeserialize(contentAs = classOf[LiftMetricBand]) LiftMetrics: Seq[LiftMetricBand] ) extends EvaluationMetrics { def rocCurve: Seq[(Double, Double)] = recallByThreshold.zip(falsePositiveRateByThreshold) diff --git a/core/src/test/scala/com/salesforce/op/evaluators/LiftEvaluatorTest.scala b/core/src/test/scala/com/salesforce/op/evaluators/LiftEvaluatorTest.scala index 6379737279..931f7b7149 100644 --- a/core/src/test/scala/com/salesforce/op/evaluators/LiftEvaluatorTest.scala +++ b/core/src/test/scala/com/salesforce/op/evaluators/LiftEvaluatorTest.scala @@ -1,3 +1,33 @@ +/* + * Copyright (c) 2017, Salesforce.com, Inc. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * * Redistributions of source code must retain the above copyright notice, this + * list of conditions and the following disclaimer. + * + * * Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * * Neither the name of the copyright holder nor the names of its + * contributors may be used to endorse or promote products derived from + * this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + */ + package com.salesforce.op.evaluators import com.salesforce.op.test.TestSparkContext @@ -5,10 +35,7 @@ import org.junit.runner.RunWith import org.scalatest.FlatSpec import org.scalatest.junit.JUnitRunner -/** - * @author sselix - * @since 218 - */ + @RunWith(classOf[JUnitRunner]) class LiftEvaluatorTest extends FlatSpec with TestSparkContext { @@ -27,8 +54,8 @@ class LiftEvaluatorTest extends FlatSpec with TestSparkContext { "LiftEvaluator.getDefaultScoreBands" should "give proper default bands" in { val bands = LiftEvaluator.getDefaultScoreBands(scores) - bands.head shouldBe (0.0, 0.1, "0-10") - bands.last shouldBe (0.9, 1.0, "90-100") + bands.head shouldBe(0.0, 0.1, "0-10") + bands.last shouldBe(0.9, 1.0, "90-100") bands.size shouldBe 10 } From cc65e50a0c1e8185ad1318da452727b4ee518d9e Mon Sep 17 00:00:00 2001 From: shae-selix Date: Thu, 28 Feb 2019 10:24:27 -0800 Subject: [PATCH 05/14] Changing NaN to None for json serialization --- .../op/evaluators/EvaluationMetrics.scala | 1 + .../op/evaluators/LiftEvaluator.scala | 26 +++++++------- .../op/evaluators/LiftEvaluatorTest.scala | 34 +++++++++---------- .../OpBinaryClassificationEvaluatorTest.scala | 16 ++++----- .../selector/ModelSelectorSummaryTest.scala | 2 +- 5 files changed, 40 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/com/salesforce/op/evaluators/EvaluationMetrics.scala b/core/src/main/scala/com/salesforce/op/evaluators/EvaluationMetrics.scala index f227cd79b0..b25a8af29f 100644 --- a/core/src/main/scala/com/salesforce/op/evaluators/EvaluationMetrics.scala +++ b/core/src/main/scala/com/salesforce/op/evaluators/EvaluationMetrics.scala @@ -140,6 +140,7 @@ object BinaryClassEvalMetrics extends Enum[ClassificationEvalMetric] { case object FP extends ClassificationEvalMetric("FP", "false positive") case object FN extends ClassificationEvalMetric("FN", "false negative") case object BrierScore extends ClassificationEvalMetric("brierScore", "brier score") + case object LiftMetrics extends ClassificationEvalMetric("liftMetrics", "lift plot") } /** diff --git a/core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala b/core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala index dfcb2ee9b0..e240a17079 100644 --- a/core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala +++ b/core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala @@ -47,8 +47,8 @@ object LiftEvaluator { * @param group name / key for score band * @param lowerBound minimum score represented in lift * @param upperBound maximum score represented in lift - * @param rate calculated lift value, i.e. # yes / total count - * @param average lift rate across all score bands + * @param rate optional calculated lift value, i.e. # yes / total count + * @param average optional lift rate across all score bands * @param totalCount total number of records in score band * @param yesCount number of yes records in score band * @param noCount number of no records in score band @@ -58,12 +58,12 @@ object LiftEvaluator { group: String, lowerBound: Double, upperBound: Double, - rate: Double, - average: Double, + rate: Option[Double], + average: Option[Double], totalCount: Long, yesCount: Long, noCount: Long - ) + ) extends EvaluationMetrics /** * Builds Seq[LiftMetricBand] for BinaryClassificationMetrics, calls liftMetricBands function @@ -175,12 +175,12 @@ object LiftEvaluator { * @param perBandCounts * @return overall # yes / total records across all bands */ - private[op] def overallLiftRate(perBandCounts: Map[String, (Long, Long)]): Double = { + private[op] def overallLiftRate(perBandCounts: Map[String, (Long, Long)]): Option[Double] = { val overallTotalCount = perBandCounts.values.map({ case (totalCount, _) => totalCount }).sum val overallYesCount = perBandCounts.values.map({ case (_, yesCount) => yesCount }).sum overallTotalCount match { - case 0L => Double.NaN - case _ => overallYesCount.toDouble / overallTotalCount + case 0L => None + case _ => Some(overallYesCount.toDouble / overallTotalCount) } } @@ -193,7 +193,7 @@ object LiftEvaluator { * @param upper upper bound of band * @param bandString String key of band e.g. "10-20" * @param perBandCounts calculated total counts and counts of true labels - * @param overallRate overall Lift rate across all bands + * @param overallRate optional overall Lift rate across all bands * @return LiftMetricBand container of metrics */ private[op] def formatLiftMetricBand @@ -202,13 +202,13 @@ object LiftEvaluator { upper: Double, bandString: String, perBandCounts: Map[String, (Long, Long)], - overallRate: Double + overallRate: Option[Double] ): LiftMetricBand = { perBandCounts.get(bandString) match { case Some((numTotal, numYes)) => { val lift = numTotal match { - case 0L => Double.NaN - case _ => numYes.toDouble / numTotal + case 0L => None + case _ => Some(numYes.toDouble / numTotal) } LiftMetricBand( group = bandString, @@ -225,7 +225,7 @@ object LiftEvaluator { group = bandString, lowerBound = lower, upperBound = upper, - rate = Double.NaN, + rate = None, average = overallRate, totalCount = 0L, yesCount = 0L, diff --git a/core/src/test/scala/com/salesforce/op/evaluators/LiftEvaluatorTest.scala b/core/src/test/scala/com/salesforce/op/evaluators/LiftEvaluatorTest.scala index 931f7b7149..ab6c243c14 100644 --- a/core/src/test/scala/com/salesforce/op/evaluators/LiftEvaluatorTest.scala +++ b/core/src/test/scala/com/salesforce/op/evaluators/LiftEvaluatorTest.scala @@ -88,28 +88,28 @@ class LiftEvaluatorTest extends FlatSpec with TestSparkContext { val perBandCountsEmpty = Map[String, (Long, Long)]() val overallRateFilled = LiftEvaluator.overallLiftRate(perBandCountsFilled) val overallRateEmpty = LiftEvaluator.overallLiftRate(perBandCountsEmpty) - overallRateFilled shouldBe 0.4 - overallRateEmpty.isNaN shouldBe true + overallRateFilled shouldBe Some(0.4) + overallRateEmpty shouldBe None } "LiftEvaluator.formatLiftMetricBand" should "format a LiftMetricBand as required" in { val perBandCounts = Map("A" -> (4L, 2L)) - val metricBandA = LiftEvaluator.formatLiftMetricBand(0.0, 0.1, "A", perBandCounts, 0.5) + val metricBandA = LiftEvaluator.formatLiftMetricBand(0.0, 0.1, "A", perBandCounts, Some(0.5)) metricBandA.group shouldBe "A" metricBandA.lowerBound shouldBe 0.0 metricBandA.upperBound shouldBe 0.1 - metricBandA.rate shouldBe 0.5 - metricBandA.average shouldBe 0.5 + metricBandA.rate shouldBe Some(0.5) + metricBandA.average shouldBe Some(0.5) metricBandA.totalCount shouldBe 4L metricBandA.yesCount shouldBe 2L metricBandA.noCount shouldBe 2L - val metricBandB = LiftEvaluator.formatLiftMetricBand(0.1, 0.2, "B", perBandCounts, 0.5) + val metricBandB = LiftEvaluator.formatLiftMetricBand(0.1, 0.2, "B", perBandCounts, Some(0.5)) metricBandB.group shouldBe "B" metricBandB.lowerBound shouldBe 0.1 metricBandB.upperBound shouldBe 0.2 - metricBandB.rate.isNaN shouldBe true - metricBandB.average shouldBe 0.5 + metricBandB.rate shouldBe None + metricBandB.average shouldBe Some(0.5) metricBandB.totalCount shouldBe 0L metricBandB.yesCount shouldBe 0L metricBandB.noCount shouldBe 0L @@ -120,11 +120,11 @@ class LiftEvaluatorTest extends FlatSpec with TestSparkContext { val band010 = liftSeq.find(_.group == "0-10").get val band90100 = liftSeq.find(_.group == "90-100").get - band010.rate shouldBe 0.0 + band010.rate shouldBe Some(0.0) band010.lowerBound shouldBe 0.0 band010.upperBound shouldBe 0.1 - band010.average shouldBe 0.45 - band90100.rate shouldBe 0.9 + band010.average shouldBe Some(0.45) + band90100.rate shouldBe Some(0.9) } "LiftEvaluator.liftMetricBands" should "correctly give defaults with empty RDD" in { @@ -135,11 +135,11 @@ class LiftEvaluatorTest extends FlatSpec with TestSparkContext { val band010 = liftSeq.find(_.group == "0-10").get val band90100 = liftSeq.find(_.group == "90-100").get - band010.rate.isNaN shouldBe true + band010.rate shouldBe None band010.lowerBound shouldBe 0.0 band010.upperBound shouldBe 0.1 - band010.average.isNaN shouldBe true - band90100.rate.isNaN shouldBe true + band010.average shouldBe None + band90100.rate shouldBe None } "LiftEvaluator.apply" should "correctly calculate a Seq[LiftMetricBand]" in { @@ -147,11 +147,11 @@ class LiftEvaluatorTest extends FlatSpec with TestSparkContext { val band010 = liftSeq.find(_.group == "0-10").get val band90100 = liftSeq.find(_.group == "90-100").get - band010.rate shouldBe 0.0 + band010.rate shouldBe Some(0.0) band010.lowerBound shouldBe 0.0 band010.upperBound shouldBe 0.1 - band010.average shouldBe 0.45 - band90100.rate shouldBe 0.9 + band010.average shouldBe Some(0.45) + band90100.rate shouldBe Some(0.9) } } diff --git a/core/src/test/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluatorTest.scala b/core/src/test/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluatorTest.scala index 3c46c77eb1..46aaa3f5b8 100644 --- a/core/src/test/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluatorTest.scala +++ b/core/src/test/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluatorTest.scala @@ -162,8 +162,8 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext 1.0 - sparkMulticlassEvaluator.setMetricName(Error.sparkEntryName).evaluate(flattenedData2) shouldBe metrics.Error LiftEvaluator.getDefaultScoreBands(sc.emptyRDD).size shouldBe metrics.LiftMetrics.size - overallLiftRate shouldBe metrics.LiftMetrics.head.average - overallLiftRate shouldBe metrics.LiftMetrics.last.average + Some(overallLiftRate) shouldBe metrics.LiftMetrics.head.average + Some(overallLiftRate) shouldBe metrics.LiftMetrics.last.average } it should "evaluate the metrics with one prediction input" in { @@ -186,8 +186,8 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext metrics.F1 shouldBe f1 LiftEvaluator.getDefaultScoreBands(sc.emptyRDD).size shouldBe metrics.LiftMetrics.size - overallLiftRate shouldBe metrics.LiftMetrics.head.average - overallLiftRate shouldBe metrics.LiftMetrics.last.average + Some(overallLiftRate) shouldBe metrics.LiftMetrics.head.average + Some(overallLiftRate) shouldBe metrics.LiftMetrics.last.average } it should "evaluate the metrics on dataset with only the label and prediction 0" in { @@ -205,10 +205,10 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext metricsZero.Recall shouldBe 0.0 metricsZero.Error shouldBe 0.0 - metricsZero.LiftMetrics.head.rate shouldBe 0.0 + metricsZero.LiftMetrics.head.rate shouldBe Some(0.0) metricsZero.LiftMetrics.head.yesCount shouldBe 0L metricsZero.LiftMetrics.head.noCount shouldBe 1L - metricsZero.LiftMetrics.tail.head.rate.isNaN shouldBe true + metricsZero.LiftMetrics.tail.head.rate shouldBe None } @@ -226,10 +226,10 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext metricsOne.Recall shouldBe 1.0 metricsOne.Error shouldBe 0.0 - metricsOne.LiftMetrics.head.rate shouldBe 1.0 + metricsOne.LiftMetrics.head.rate shouldBe Some(1.0) metricsOne.LiftMetrics.head.yesCount shouldBe 1L metricsOne.LiftMetrics.head.noCount shouldBe 0L - metricsOne.LiftMetrics.tail.head.rate.isNaN shouldBe true + metricsOne.LiftMetrics.tail.head.rate shouldBe None } private def getPosNegValues(rdd: RDD[Row]): (Double, Double, Double, Double, Double, Double, Double) = { diff --git a/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala b/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala index 9c16645b6a..2debc209be 100644 --- a/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala +++ b/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala @@ -55,7 +55,7 @@ class ModelSelectorSummaryTest extends FlatSpec with TestSparkContext { trainEvaluation = BinaryClassificationMetrics(Precision = 0.1, Recall = 0.2, F1 = 0.3, AuROC = 0.4, AuPR = 0.5, Error = 0.6, TP = 0.7, TN = 0.8, FP = 0.9, FN = 1.0, thresholds = Seq(1.1), precisionByThreshold = Seq(1.2), recallByThreshold = Seq(1.3), falsePositiveRateByThreshold = Seq(1.4), - LiftMetrics = Seq(LiftEvaluator.LiftMetricBand("0-10", 0.0, 0.1, 0.1, 0.5, 10L, 1L, 9L))), + LiftMetrics = Seq(LiftEvaluator.LiftMetricBand("0-10", 0.0, 0.1, Some(0.1), Some(0.5), 10L, 1L, 9L))), holdoutEvaluation = Option(RegressionMetrics(RootMeanSquaredError = 1.3, MeanSquaredError = 1.4, R2 = 1.5, MeanAbsoluteError = 1.6)) ) From b6b9e0793b70917b5613fc797e481fa4637e21fe Mon Sep 17 00:00:00 2001 From: shae-selix Date: Thu, 21 Mar 2019 22:52:43 -0700 Subject: [PATCH 06/14] Refactoring LiftEvaluator to extend OpBinaryClassificationEvaluatorBase --- .../op/evaluators/LiftEvaluator.scala | 240 ++++++++++++++---- .../OpBinaryClassificationEvaluator.scala | 21 +- .../op/evaluators/LiftEvaluatorTest.scala | 72 +++--- .../OpBinaryClassificationEvaluatorTest.scala | 35 +-- .../selector/ModelSelectorSummaryTest.scala | 7 +- 5 files changed, 262 insertions(+), 113 deletions(-) diff --git a/core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala b/core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala index e240a17079..2223d6193b 100644 --- a/core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala +++ b/core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala @@ -30,58 +30,100 @@ package com.salesforce.op.evaluators +import com.fasterxml.jackson.databind.annotation.JsonDeserialize +import com.salesforce.op.UID import org.apache.spark.rdd.RDD +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.types.DoubleType +import org.slf4j.LoggerFactory /** - * Object to calculate Lift metrics for BinaryClassification problems - * Intended to build a Lift Plot. + * Evaluator Class to calculate Lift metrics for BinaryClassification problems + * Intended to build a Lift Plot, or with a threshold, evaluate to a numeric + * value, liftRatio, to determine model fit. See: + * https://en.wikipedia.org/wiki/Lift_(data_mining) * * Algorithm for calculating a chart as seen here: * https://www.kdnuggets.com/2016/03/lift-analysis-data-scientist-secret-weapon.html + * + * @param threshold decision value to categorize score probabilities into predicted labels + * @param bandFn function to convert score distribution into score bands + * @param uid UID for evaluator */ -object LiftEvaluator { +class LiftEvaluator +( + threshold: Double = LiftMetrics.defaultThreshold, + bandFn: RDD[Double] => Seq[(Double, Double, String)] = LiftEvaluator.getDefaultScoreBands, + override val uid: String = UID[OpBinaryClassificationEvaluator] +) extends OpBinaryClassificationEvaluatorBase[LiftMetrics](uid = uid) { + + @transient private lazy val log = LoggerFactory.getLogger(this.getClass) + + override val name: EvalMetric = BinaryClassEvalMetrics.LiftMetrics /** - * Stores basic lift values for a specific band of scores + * Default metrics is liftRatio, which is calculated as: + * (# yes records with score >= threshold / # total records with score >= threshold) + * / (# yes records overall / # total records overall) * - * @param group name / key for score band - * @param lowerBound minimum score represented in lift - * @param upperBound maximum score represented in lift - * @param rate optional calculated lift value, i.e. # yes / total count - * @param average optional lift rate across all score bands - * @param totalCount total number of records in score band - * @param yesCount number of yes records in score band - * @param noCount number of no records in score band + * @return double value used as spark eval number */ - case class LiftMetricBand - ( - group: String, - lowerBound: Double, - upperBound: Double, - rate: Option[Double], - average: Option[Double], - totalCount: Long, - yesCount: Long, - noCount: Long - ) extends EvaluationMetrics + override def getDefaultMetric: LiftMetrics => Double = _.liftRatio /** - * Builds Seq[LiftMetricBand] for BinaryClassificationMetrics, calls liftMetricBands function - * with default score bands function + * Evaluates entire dataset, pulling out score and label columns + * into an RDD and return an an instance of LiftMetrics * - * @param scoreAndLabels RDD[(Double, Double)] of BinaryClassification (score, label) tuples - * @return Seq of LiftMetricBand containers of Lift calculations + * @param dataset data to evaluate + * @return metrics */ - def apply - ( - scoreAndLabels: RDD[(Double, Double)] - ): Seq[LiftMetricBand] = { - liftMetricBands( - scoreAndLabels, - getDefaultScoreBands - ) + override def evaluateAll(dataset: Dataset[_]): LiftMetrics = { + val labelColumnName = getLabelCol + val dataToUse = makeDataToUse(dataset, labelColumnName) + .select(col(getProbabilityCol), col(labelColumnName).cast(DoubleType)).rdd + if (dataToUse.isEmpty()) { + log.warn("The dataset is empty. Returning empty metrics.") + LiftMetrics.empty + } else { + val scoreAndLabels = dataToUse.map { + case Row(prob: Vector, label: Double) => (prob(1), label) + case Row(prob: Double, label: Double) => (prob, label) + } + evaluateScoreAndLabels(scoreAndLabels) + } } + /** + * Calculates a Seq of lift metrics per score band and + * an overall lift ratio value, returned as LiftMetrics, + * from an RDD of scores and labels + * + * @param scoreAndLabels RDD of score and label doubles + * @return an instance of LiftMetrics + */ + def evaluateScoreAndLabels(scoreAndLabels: RDD[(Double, Double)]): LiftMetrics = { + val liftMetricBands = LiftEvaluator.liftMetricBands(scoreAndLabels, bandFn) + val overallRate = LiftEvaluator.thresholdLiftRate(scoreAndLabels, 0.0) + val liftRatio = LiftEvaluator.liftRatio(overallRate, scoreAndLabels, threshold) + LiftMetrics( + liftMetricBands = liftMetricBands, + threshold = threshold, + liftRatio = liftRatio, + overallRate = overallRate) + } +} + +/** + * Object to calculate Lift metrics for BinaryClassification problems + * Intended to build a Lift Plot. + * + * Algorithm for calculating a chart as seen here: + * https://www.kdnuggets.com/2016/03/lift-analysis-data-scientist-secret-weapon.html + */ +object LiftEvaluator { + /** * Builds Seq of LiftMetricBand using RDD api * @@ -92,17 +134,17 @@ object LiftEvaluator { private[op] def liftMetricBands ( scoreAndLabels: RDD[(Double, Double)], - getScoreBands: RDD[Double] => Seq[(Double, Double, String)] + bandFn: RDD[Double] => Seq[(Double, Double, String)] ): Seq[LiftMetricBand] = { - val bands = getScoreBands(scoreAndLabels.map { case (score, _) => score }) + val scores = scoreAndLabels.map { case (score, _) => score } + val bands = bandFn(scores) val bandedLabels = scoreAndLabels.map { case (score, label) => (categorizeScoreIntoBand((score, bands)), label) }.collect { case (Some(band), label) => (band, label) } val perBandCounts = aggregateBandedLabels(bandedLabels) - val overallRate = overallLiftRate(perBandCounts) - bands.map({ case (lower, upper, band) => - formatLiftMetricBand(lower, upper, band, perBandCounts, overallRate) - }).sortBy(band => band.lowerBound) + bands.map { case (lower, upper, band) => + formatLiftMetricBand(lower, upper, band, perBandCounts) + }.sortBy(band => band.lowerBound) } /** @@ -113,8 +155,7 @@ object LiftEvaluator { * @param scores RDD of scores. unused in this function * @return sequence of (lowerBound, upperBound, bandString) tuples */ - private[op] def getDefaultScoreBands(scores: RDD[Double]): - Seq[(Double, Double, String)] = + private[op] def getDefaultScoreBands(scores: RDD[Double]): Seq[(Double, Double, String)] = Seq( (0.0, 0.1, "0-10"), (0.1, 0.2, "10-20"), @@ -170,17 +211,57 @@ object LiftEvaluator { } /** - * calculates a baseline "yes" rate across score bands + * Subsets scoresAndLabels to only records with scores greater than + * a threshold, categorizing them as predicted "yes" labels, then + * returns (# of true "yes" labels / # predicted yes) * - * @param perBandCounts - * @return overall # yes / total records across all bands + * @param scoreAndLabels RDD of labels and scores + * @param threshold decision value, where scores >= thres are predicted "yes" + * @return Optional lift rate, None if denominator is 0 */ - private[op] def overallLiftRate(perBandCounts: Map[String, (Long, Long)]): Option[Double] = { - val overallTotalCount = perBandCounts.values.map({ case (totalCount, _) => totalCount }).sum - val overallYesCount = perBandCounts.values.map({ case (_, yesCount) => yesCount }).sum - overallTotalCount match { + private[op] def thresholdLiftRate + ( + scoreAndLabels: RDD[(Double, Double)], + threshold: Double + ): Option[Double] = { + val (yesCount, totalCount) = scoreAndLabels.aggregate(zeroValue = (0L, 0L))({ + case ((yesCount, totalCount), (score, label)) => { + if (score < threshold) (yesCount, totalCount) + else (yesCount + label.toLong, totalCount + 1L) + } + }, { case ((yesCountX, totalCountX), (yesCountY, totalCountY)) => + (yesCountX + yesCountY, totalCountX + totalCountY) + }) + totalCount match { case 0L => None - case _ => Some(overallYesCount.toDouble / overallTotalCount) + case _ => Some(yesCount.toDouble / totalCount.toDouble) + } + } + + /** + * Given a threshold decision value, calculates the lift + * in label prediction accuracy over random, as described here: + * https://en.wikipedia.org/wiki/Lift_(data_mining) + * + * @param overallRate # yes / total count across all data + * @param scoreAndLabels RDD of scores and labels + * @param threshold decision boundary for categorizing scores + * @return lift ratio, given threshold + */ + private[op] def liftRatio + ( + overallRate: Option[Double], + scoreAndLabels: RDD[(Double, Double)], + threshold: Double + ): Double = overallRate match { + case None => LiftMetrics.defaultLiftRatio + case Some(0.0) => LiftMetrics.defaultLiftRatio + case Some(rate) => { + val thresholdLift = thresholdLiftRate(scoreAndLabels, threshold) + thresholdLift match { + case None => LiftMetrics.defaultLiftRatio + case Some(thresholdRate) => thresholdRate / rate + } } } @@ -193,7 +274,6 @@ object LiftEvaluator { * @param upper upper bound of band * @param bandString String key of band e.g. "10-20" * @param perBandCounts calculated total counts and counts of true labels - * @param overallRate optional overall Lift rate across all bands * @return LiftMetricBand container of metrics */ private[op] def formatLiftMetricBand @@ -201,8 +281,7 @@ object LiftEvaluator { lower: Double, upper: Double, bandString: String, - perBandCounts: Map[String, (Long, Long)], - overallRate: Option[Double] + perBandCounts: Map[String, (Long, Long)] ): LiftMetricBand = { perBandCounts.get(bandString) match { case Some((numTotal, numYes)) => { @@ -215,7 +294,6 @@ object LiftEvaluator { lowerBound = lower, upperBound = upper, rate = lift, - average = overallRate, totalCount = numTotal, yesCount = numYes, noCount = numTotal - numYes @@ -226,7 +304,6 @@ object LiftEvaluator { lowerBound = lower, upperBound = upper, rate = None, - average = overallRate, totalCount = 0L, yesCount = 0L, noCount = 0L @@ -235,3 +312,54 @@ object LiftEvaluator { } } + +/** + * Stores basic lift values for a specific band of scores + * + * @param group name / key for score band + * @param lowerBound minimum score represented in lift + * @param upperBound maximum score represented in lift + * @param rate optional calculated lift value, i.e. # yes / total count + * @param totalCount total number of records in score band + * @param yesCount number of yes records in score band + * @param noCount number of no records in score band + */ +case class LiftMetricBand +( + group: String, + lowerBound: Double, + upperBound: Double, + rate: Option[Double], + totalCount: Long, + yesCount: Long, + noCount: Long +) extends EvaluationMetrics + +/** + * Stores sequence of lift score band metrics + * as well as overall lift values + * + * @param liftMetricBands Seq of LiftMetricBand, calculated by LiftEvaluator + * @param threshold threshold used to categorize scores + * @param liftRatio overall lift ratio, given a specified threshold + * @param overallRate # yes records / # total records + */ +case class LiftMetrics +( + @JsonDeserialize(contentAs = classOf[LiftMetricBand]) + liftMetricBands: Seq[LiftMetricBand], + threshold: Double, + liftRatio: Double, + overallRate: Option[Double] +) extends EvaluationMetrics + +/** + * Companion object to LiftMetrics case class + * for storing default values + */ +object LiftMetrics { + val defaultThreshold = 0.5 + val defaultLiftRatio = 1.0 + + def empty: LiftMetrics = LiftMetrics(Seq(), defaultThreshold, defaultLiftRatio, None) +} diff --git a/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala b/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala index 3cd5f90ce4..45f87594af 100644 --- a/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala +++ b/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala @@ -34,7 +34,6 @@ import com.fasterxml.jackson.databind.annotation.JsonDeserialize import com.salesforce.op.UID import com.salesforce.op.utils.spark.RichEvaluator._ import com.salesforce.op.evaluators.BinaryClassEvalMetrics._ -import com.salesforce.op.evaluators.LiftEvaluator.LiftMetricBand import org.apache.spark.ml.evaluation.{BinaryClassificationEvaluator, MulticlassClassificationEvaluator} import org.apache.spark.ml.linalg.Vector import org.apache.spark.mllib.evaluation.{MulticlassMetrics, BinaryClassificationMetrics => SparkMLBinaryClassificationMetrics} @@ -49,9 +48,9 @@ import org.slf4j.LoggerFactory * The metrics are AUROC, AUPR, Precision, Recall, F1 and Error Rate * Default evaluation returns AUROC * - * @param name name of default metric + * @param name name of default metric * @param isLargerBetter is metric better if larger - * @param uid uid for instance + * @param uid uid for instance */ private[op] class OpBinaryClassificationEvaluator @@ -84,7 +83,7 @@ private[op] class OpBinaryClassificationEvaluator if (rdd.isEmpty()) { log.warn("The dataset is empty. Returning empty metrics.") BinaryClassificationMetrics(0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, - Seq(), Seq(), Seq(), Seq(), Seq()) + Seq(), Seq(), Seq(), Seq(), LiftMetrics.empty) } else { val multiclassMetrics = new MulticlassMetrics(rdd) val labels = multiclassMetrics.labels @@ -115,7 +114,7 @@ private[op] class OpBinaryClassificationEvaluator val falsePositiveRateByThreshold = sparkMLMetrics.roc().collect().map(_._1).slice(1, thresholds.length + 1) val aUROC = sparkMLMetrics.areaUnderROC() val aUPR = sparkMLMetrics.areaUnderPR() - val liftMetrics = LiftEvaluator(scoreAndLabels = scoreAndLabels) + val liftMetrics = new LiftEvaluator().evaluateScoreAndLabels(scoreAndLabels = scoreAndLabels) val metrics = BinaryClassificationMetrics( Precision = precision, Recall = recall, F1 = f1, AuROC = aUROC, AuPR = aUPR, Error = error, TP = tp, TN = tn, FP = fp, FN = fn, @@ -128,10 +127,10 @@ private[op] class OpBinaryClassificationEvaluator } final protected def getBinaryEvaluatorMetric( - metricName: ClassificationEvalMetric, - dataset: Dataset[_], - default: => Double - ): Double = { + metricName: ClassificationEvalMetric, + dataset: Dataset[_], + default: => Double + ): Double = { import dataset.sparkSession.implicits._ val labelColName = getLabelCol val dataUse = makeDataToUse(dataset, labelColName) @@ -200,9 +199,9 @@ case class BinaryClassificationMetrics recallByThreshold: Seq[Double], @JsonDeserialize(contentAs = classOf[java.lang.Double]) falsePositiveRateByThreshold: Seq[Double], - @JsonDeserialize(contentAs = classOf[LiftMetricBand]) - LiftMetrics: Seq[LiftMetricBand] + LiftMetrics: LiftMetrics ) extends EvaluationMetrics { def rocCurve: Seq[(Double, Double)] = recallByThreshold.zip(falsePositiveRateByThreshold) + def prCurve: Seq[(Double, Double)] = precisionByThreshold.zip(recallByThreshold) } diff --git a/core/src/test/scala/com/salesforce/op/evaluators/LiftEvaluatorTest.scala b/core/src/test/scala/com/salesforce/op/evaluators/LiftEvaluatorTest.scala index ab6c243c14..c920b8c345 100644 --- a/core/src/test/scala/com/salesforce/op/evaluators/LiftEvaluatorTest.scala +++ b/core/src/test/scala/com/salesforce/op/evaluators/LiftEvaluatorTest.scala @@ -39,6 +39,8 @@ import org.scalatest.junit.JUnitRunner @RunWith(classOf[JUnitRunner]) class LiftEvaluatorTest extends FlatSpec with TestSparkContext { + import spark.implicits._ + lazy val labelSeq = for { i <- 0 until 10 j <- 0 until 10 @@ -47,10 +49,11 @@ class LiftEvaluatorTest extends FlatSpec with TestSparkContext { else 0.0 } lazy val scoreSeq = (0.01 to 1.0 by 0.01) - lazy val scores = sc.parallelize(scoreSeq) - - lazy val scoresAndLabels = sc.parallelize(scoreSeq.zip(labelSeq)) + lazy val scoreAndLabels = sc.parallelize(scoreSeq.zip(labelSeq)) + lazy val emptyScoreAndLabels = sc.parallelize(Seq[(Double, Double)]()) + lazy val dataset = scoreAndLabels.toDF("probability", "label") + lazy val emptyDataset = emptyScoreAndLabels.toDF("probability", "label") "LiftEvaluator.getDefaultScoreBands" should "give proper default bands" in { val bands = LiftEvaluator.getDefaultScoreBands(scores) @@ -83,53 +86,41 @@ class LiftEvaluatorTest extends FlatSpec with TestSparkContext { numPositivesB shouldBe 0L } - "LiftEvaluator.overallLiftRate" should "calculate an overall rate" in { - val perBandCountsFilled = Map("A" -> (4L, 2L), "B" -> (1L, 0L)) - val perBandCountsEmpty = Map[String, (Long, Long)]() - val overallRateFilled = LiftEvaluator.overallLiftRate(perBandCountsFilled) - val overallRateEmpty = LiftEvaluator.overallLiftRate(perBandCountsEmpty) - overallRateFilled shouldBe Some(0.4) - overallRateEmpty shouldBe None - } - "LiftEvaluator.formatLiftMetricBand" should "format a LiftMetricBand as required" in { val perBandCounts = Map("A" -> (4L, 2L)) - val metricBandA = LiftEvaluator.formatLiftMetricBand(0.0, 0.1, "A", perBandCounts, Some(0.5)) + val metricBandA = LiftEvaluator.formatLiftMetricBand(0.0, 0.1, "A", perBandCounts) metricBandA.group shouldBe "A" metricBandA.lowerBound shouldBe 0.0 metricBandA.upperBound shouldBe 0.1 metricBandA.rate shouldBe Some(0.5) - metricBandA.average shouldBe Some(0.5) metricBandA.totalCount shouldBe 4L metricBandA.yesCount shouldBe 2L metricBandA.noCount shouldBe 2L - val metricBandB = LiftEvaluator.formatLiftMetricBand(0.1, 0.2, "B", perBandCounts, Some(0.5)) + val metricBandB = LiftEvaluator.formatLiftMetricBand(0.1, 0.2, "B", perBandCounts) metricBandB.group shouldBe "B" metricBandB.lowerBound shouldBe 0.1 metricBandB.upperBound shouldBe 0.2 metricBandB.rate shouldBe None - metricBandB.average shouldBe Some(0.5) metricBandB.totalCount shouldBe 0L metricBandB.yesCount shouldBe 0L metricBandB.noCount shouldBe 0L } "LiftEvaluator.liftMetricBands" should "correctly calculate a Seq[LiftMetricBand]" in { - val liftSeq = LiftEvaluator.liftMetricBands(scoresAndLabels, LiftEvaluator.getDefaultScoreBands) + val liftSeq = LiftEvaluator.liftMetricBands(scoreAndLabels, LiftEvaluator.getDefaultScoreBands) val band010 = liftSeq.find(_.group == "0-10").get val band90100 = liftSeq.find(_.group == "90-100").get band010.rate shouldBe Some(0.0) band010.lowerBound shouldBe 0.0 band010.upperBound shouldBe 0.1 - band010.average shouldBe Some(0.45) band90100.rate shouldBe Some(0.9) } "LiftEvaluator.liftMetricBands" should "correctly give defaults with empty RDD" in { val liftSeq = LiftEvaluator.liftMetricBands( - sc.parallelize(Seq[(Double, Double)]()), + emptyScoreAndLabels, LiftEvaluator.getDefaultScoreBands ) val band010 = liftSeq.find(_.group == "0-10").get @@ -138,20 +129,43 @@ class LiftEvaluatorTest extends FlatSpec with TestSparkContext { band010.rate shouldBe None band010.lowerBound shouldBe 0.0 band010.upperBound shouldBe 0.1 - band010.average shouldBe None band90100.rate shouldBe None } - "LiftEvaluator.apply" should "correctly calculate a Seq[LiftMetricBand]" in { - val liftSeq = LiftEvaluator.apply(scoresAndLabels) - val band010 = liftSeq.find(_.group == "0-10").get - val band90100 = liftSeq.find(_.group == "90-100").get + "LiftEvaluator.thresholdLiftRate" should "correctly calculate lift with a threshold" in { + val thresholdLift = LiftEvaluator.thresholdLiftRate(scoreAndLabels, 0.5) + val overallRate = LiftEvaluator.thresholdLiftRate(scoreAndLabels, 0.0) + val thresholdLiftZeros = LiftEvaluator.thresholdLiftRate( + sc.parallelize(Seq[(Double, Double)]((0.6, 0.0), (0.7, 0.0), (0.8, 0.0))), 0.5) + val thresholdLiftEmpty = LiftEvaluator.thresholdLiftRate(emptyScoreAndLabels, 0.5) + thresholdLift shouldBe Some(0.6862745098039216) + overallRate shouldBe Some(0.45) + thresholdLiftZeros shouldBe Some(0.0) + thresholdLiftEmpty shouldBe None + } - band010.rate shouldBe Some(0.0) - band010.lowerBound shouldBe 0.0 - band010.upperBound shouldBe 0.1 - band010.average shouldBe Some(0.45) - band90100.rate shouldBe Some(0.9) + "LiftEvaluator.evaluateScoreAndLabels" should "correctly produce LiftMetrics" in { + val evaluator = new LiftEvaluator() + val liftMetrics = evaluator.evaluateScoreAndLabels(scoreAndLabels) + val liftMetricsEmpty = evaluator.evaluateScoreAndLabels(emptyScoreAndLabels) + liftMetrics.liftMetricBands.size shouldBe LiftEvaluator.getDefaultScoreBands(scores).size + liftMetrics.liftRatio shouldBe 1.5250544662309369 + liftMetrics.overallRate shouldBe Some(0.45) + liftMetricsEmpty.liftMetricBands.size shouldBe LiftEvaluator.getDefaultScoreBands(scores).size + liftMetricsEmpty.liftRatio shouldBe LiftMetrics.defaultLiftRatio + liftMetricsEmpty.overallRate shouldBe None + } + + "LiftEvaluator.evaluateAll" should "correctly produce LiftMetrics" in { + val evaluator = new LiftEvaluator() + val liftMetrics = evaluator.evaluateAll(dataset) + val liftMetricsEmpty = evaluator.evaluateAll(emptyDataset) + liftMetrics.liftMetricBands.size shouldBe LiftEvaluator.getDefaultScoreBands(scores).size + liftMetrics.liftRatio shouldBe 1.5250544662309369 + liftMetrics.overallRate shouldBe Some(0.45) + liftMetricsEmpty.liftMetricBands.size shouldBe 0 + liftMetricsEmpty.liftRatio shouldBe LiftMetrics.defaultLiftRatio + liftMetricsEmpty.overallRate shouldBe None } } diff --git a/core/src/test/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluatorTest.scala b/core/src/test/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluatorTest.scala index 46aaa3f5b8..d520e296d2 100644 --- a/core/src/test/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluatorTest.scala +++ b/core/src/test/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluatorTest.scala @@ -161,9 +161,8 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext f1 shouldBe metrics.F1 1.0 - sparkMulticlassEvaluator.setMetricName(Error.sparkEntryName).evaluate(flattenedData2) shouldBe metrics.Error - LiftEvaluator.getDefaultScoreBands(sc.emptyRDD).size shouldBe metrics.LiftMetrics.size - Some(overallLiftRate) shouldBe metrics.LiftMetrics.head.average - Some(overallLiftRate) shouldBe metrics.LiftMetrics.last.average + LiftEvaluator.getDefaultScoreBands(sc.emptyRDD).size shouldBe metrics.LiftMetrics.liftMetricBands.size + Some(overallLiftRate) shouldBe metrics.LiftMetrics.overallRate } it should "evaluate the metrics with one prediction input" in { @@ -172,7 +171,7 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext val (tp, tn, fp, fn, precision, recall, f1) = getPosNegValues( transformedData2.select(prediction.name, test_label.name).rdd - .map( r => Row(r.getMap[String, Double](0).toMap.toPrediction.prediction, r.getDouble(1)) ) + .map(r => Row(r.getMap[String, Double](0).toMap.toPrediction.prediction, r.getDouble(1))) ) val overallLiftRate = (tp + fn) / (tp + tn + fp + fn) @@ -185,9 +184,8 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext metrics.Recall shouldBe recall metrics.F1 shouldBe f1 - LiftEvaluator.getDefaultScoreBands(sc.emptyRDD).size shouldBe metrics.LiftMetrics.size - Some(overallLiftRate) shouldBe metrics.LiftMetrics.head.average - Some(overallLiftRate) shouldBe metrics.LiftMetrics.last.average + LiftEvaluator.getDefaultScoreBands(sc.emptyRDD).size shouldBe metrics.LiftMetrics.liftMetricBands.size + Some(overallLiftRate) shouldBe metrics.LiftMetrics.overallRate } it should "evaluate the metrics on dataset with only the label and prediction 0" in { @@ -205,14 +203,16 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext metricsZero.Recall shouldBe 0.0 metricsZero.Error shouldBe 0.0 - metricsZero.LiftMetrics.head.rate shouldBe Some(0.0) - metricsZero.LiftMetrics.head.yesCount shouldBe 0L - metricsZero.LiftMetrics.head.noCount shouldBe 1L - metricsZero.LiftMetrics.tail.head.rate shouldBe None + metricsZero.LiftMetrics.liftMetricBands.head.rate shouldBe Some(0.0) + metricsZero.LiftMetrics.liftMetricBands.head.yesCount shouldBe 0L + metricsZero.LiftMetrics.liftMetricBands.head.noCount shouldBe 1L + metricsZero.LiftMetrics.liftMetricBands.tail.head.rate shouldBe None + metricsZero.LiftMetrics.threshold shouldBe LiftMetrics.defaultThreshold + metricsZero.LiftMetrics.overallRate shouldBe Some(0.0) + metricsZero.LiftMetrics.liftRatio shouldBe LiftMetrics.defaultLiftRatio } - it should "evaluate the metrics on dataset with only the label and prediction 1" in { val transformedDataOne = model.setInput(one_label, one_features).transform(one_ds) @@ -226,10 +226,13 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext metricsOne.Recall shouldBe 1.0 metricsOne.Error shouldBe 0.0 - metricsOne.LiftMetrics.head.rate shouldBe Some(1.0) - metricsOne.LiftMetrics.head.yesCount shouldBe 1L - metricsOne.LiftMetrics.head.noCount shouldBe 0L - metricsOne.LiftMetrics.tail.head.rate shouldBe None + metricsOne.LiftMetrics.liftMetricBands.head.rate shouldBe Some(1.0) + metricsOne.LiftMetrics.liftMetricBands.head.yesCount shouldBe 1L + metricsOne.LiftMetrics.liftMetricBands.head.noCount shouldBe 0L + metricsOne.LiftMetrics.liftMetricBands.tail.head.rate shouldBe None + metricsOne.LiftMetrics.threshold shouldBe LiftMetrics.defaultThreshold + metricsOne.LiftMetrics.overallRate shouldBe Some(1.0) + metricsOne.LiftMetrics.liftRatio shouldBe 1.0 } private def getPosNegValues(rdd: RDD[Row]): (Double, Double, Double, Double, Double, Double, Double) = { diff --git a/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala b/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala index 2debc209be..90fa800416 100644 --- a/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala +++ b/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala @@ -55,7 +55,12 @@ class ModelSelectorSummaryTest extends FlatSpec with TestSparkContext { trainEvaluation = BinaryClassificationMetrics(Precision = 0.1, Recall = 0.2, F1 = 0.3, AuROC = 0.4, AuPR = 0.5, Error = 0.6, TP = 0.7, TN = 0.8, FP = 0.9, FN = 1.0, thresholds = Seq(1.1), precisionByThreshold = Seq(1.2), recallByThreshold = Seq(1.3), falsePositiveRateByThreshold = Seq(1.4), - LiftMetrics = Seq(LiftEvaluator.LiftMetricBand("0-10", 0.0, 0.1, Some(0.1), Some(0.5), 10L, 1L, 9L))), + LiftMetrics = LiftMetrics( + liftMetricBands = Seq(LiftMetricBand("0-10", 0.0, 0.1, Some(0.1), 10L, 1L, 9L)), + threshold = LiftMetrics.defaultThreshold, + liftRatio = LiftMetrics.defaultLiftRatio, + overallRate = Some(0.1) + )), holdoutEvaluation = Option(RegressionMetrics(RootMeanSquaredError = 1.3, MeanSquaredError = 1.4, R2 = 1.5, MeanAbsoluteError = 1.6)) ) From 7102c4bab9e4d329fd1ceeee94e2ae1201a1e754 Mon Sep 17 00:00:00 2001 From: shae-selix Date: Wed, 27 Mar 2019 18:05:37 -0700 Subject: [PATCH 07/14] Adapting OpBinScoreEvaluator for LiftPlot needs --- .../op/evaluators/LiftEvaluator.scala | 365 ------------------ .../op/evaluators/OpBinScoreEvaluator.scala | 127 +++--- .../OpBinaryClassificationEvaluator.scala | 19 +- .../op/evaluators/LiftEvaluatorTest.scala | 171 -------- .../evaluators/OpBinScoreEvaluatorTest.scala | 8 +- .../OpBinaryClassificationEvaluatorTest.scala | 37 +- .../selector/ModelSelectorSummaryTest.scala | 13 +- 7 files changed, 115 insertions(+), 625 deletions(-) delete mode 100644 core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala delete mode 100644 core/src/test/scala/com/salesforce/op/evaluators/LiftEvaluatorTest.scala diff --git a/core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala b/core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala deleted file mode 100644 index 2223d6193b..0000000000 --- a/core/src/main/scala/com/salesforce/op/evaluators/LiftEvaluator.scala +++ /dev/null @@ -1,365 +0,0 @@ -/* - * Copyright (c) 2017, Salesforce.com, Inc. - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * * Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * * Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * * Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - */ - -package com.salesforce.op.evaluators - -import com.fasterxml.jackson.databind.annotation.JsonDeserialize -import com.salesforce.op.UID -import org.apache.spark.rdd.RDD -import org.apache.spark.ml.linalg.Vector -import org.apache.spark.sql.{Dataset, Row} -import org.apache.spark.sql.functions.col -import org.apache.spark.sql.types.DoubleType -import org.slf4j.LoggerFactory - -/** - * Evaluator Class to calculate Lift metrics for BinaryClassification problems - * Intended to build a Lift Plot, or with a threshold, evaluate to a numeric - * value, liftRatio, to determine model fit. See: - * https://en.wikipedia.org/wiki/Lift_(data_mining) - * - * Algorithm for calculating a chart as seen here: - * https://www.kdnuggets.com/2016/03/lift-analysis-data-scientist-secret-weapon.html - * - * @param threshold decision value to categorize score probabilities into predicted labels - * @param bandFn function to convert score distribution into score bands - * @param uid UID for evaluator - */ -class LiftEvaluator -( - threshold: Double = LiftMetrics.defaultThreshold, - bandFn: RDD[Double] => Seq[(Double, Double, String)] = LiftEvaluator.getDefaultScoreBands, - override val uid: String = UID[OpBinaryClassificationEvaluator] -) extends OpBinaryClassificationEvaluatorBase[LiftMetrics](uid = uid) { - - @transient private lazy val log = LoggerFactory.getLogger(this.getClass) - - override val name: EvalMetric = BinaryClassEvalMetrics.LiftMetrics - - /** - * Default metrics is liftRatio, which is calculated as: - * (# yes records with score >= threshold / # total records with score >= threshold) - * / (# yes records overall / # total records overall) - * - * @return double value used as spark eval number - */ - override def getDefaultMetric: LiftMetrics => Double = _.liftRatio - - /** - * Evaluates entire dataset, pulling out score and label columns - * into an RDD and return an an instance of LiftMetrics - * - * @param dataset data to evaluate - * @return metrics - */ - override def evaluateAll(dataset: Dataset[_]): LiftMetrics = { - val labelColumnName = getLabelCol - val dataToUse = makeDataToUse(dataset, labelColumnName) - .select(col(getProbabilityCol), col(labelColumnName).cast(DoubleType)).rdd - if (dataToUse.isEmpty()) { - log.warn("The dataset is empty. Returning empty metrics.") - LiftMetrics.empty - } else { - val scoreAndLabels = dataToUse.map { - case Row(prob: Vector, label: Double) => (prob(1), label) - case Row(prob: Double, label: Double) => (prob, label) - } - evaluateScoreAndLabels(scoreAndLabels) - } - } - - /** - * Calculates a Seq of lift metrics per score band and - * an overall lift ratio value, returned as LiftMetrics, - * from an RDD of scores and labels - * - * @param scoreAndLabels RDD of score and label doubles - * @return an instance of LiftMetrics - */ - def evaluateScoreAndLabels(scoreAndLabels: RDD[(Double, Double)]): LiftMetrics = { - val liftMetricBands = LiftEvaluator.liftMetricBands(scoreAndLabels, bandFn) - val overallRate = LiftEvaluator.thresholdLiftRate(scoreAndLabels, 0.0) - val liftRatio = LiftEvaluator.liftRatio(overallRate, scoreAndLabels, threshold) - LiftMetrics( - liftMetricBands = liftMetricBands, - threshold = threshold, - liftRatio = liftRatio, - overallRate = overallRate) - } -} - -/** - * Object to calculate Lift metrics for BinaryClassification problems - * Intended to build a Lift Plot. - * - * Algorithm for calculating a chart as seen here: - * https://www.kdnuggets.com/2016/03/lift-analysis-data-scientist-secret-weapon.html - */ -object LiftEvaluator { - - /** - * Builds Seq of LiftMetricBand using RDD api - * - * @param scoreAndLabels RDD[(Double, Double)] of BinaryClassification (score, label) tuples - * @param getScoreBands function to calculate score bands, potentially using score distribution - * @return Seq of LiftMetricBand containers of Lift calculations - */ - private[op] def liftMetricBands - ( - scoreAndLabels: RDD[(Double, Double)], - bandFn: RDD[Double] => Seq[(Double, Double, String)] - ): Seq[LiftMetricBand] = { - val scores = scoreAndLabels.map { case (score, _) => score } - val bands = bandFn(scores) - val bandedLabels = scoreAndLabels.map { case (score, label) => - (categorizeScoreIntoBand((score, bands)), label) - }.collect { case (Some(band), label) => (band, label) } - val perBandCounts = aggregateBandedLabels(bandedLabels) - bands.map { case (lower, upper, band) => - formatLiftMetricBand(lower, upper, band, perBandCounts) - }.sortBy(band => band.lowerBound) - } - - /** - * function to return score bands for calculating lift - * Default: 10 equidistant bands for all 0.1 increments - * from 0.0 to 1.0 - * - * @param scores RDD of scores. unused in this function - * @return sequence of (lowerBound, upperBound, bandString) tuples - */ - private[op] def getDefaultScoreBands(scores: RDD[Double]): Seq[(Double, Double, String)] = - Seq( - (0.0, 0.1, "0-10"), - (0.1, 0.2, "10-20"), - (0.2, 0.3, "20-30"), - (0.3, 0.4, "30-40"), - (0.4, 0.5, "40-50"), - (0.5, 0.6, "50-60"), - (0.6, 0.7, "60-70"), - (0.7, 0.8, "70-80"), - (0.8, 0.9, "80-90"), - (0.9, 1.0, "90-100") - ) - - /** - * PartialFunction. Defined when scores are [0.0, 1.0] - * Places a score Double into a score band based on - * lower and upper bounds - * - * @param score BinaryClassification score Double, [0.0, 1.0] - * @param bands sequence of upper/lower score bands - * @return optional key to describe categorized band, if found - */ - private[op] def categorizeScoreIntoBand: - PartialFunction[(Double, Seq[(Double, Double, String)]), Option[String]] = { - case (score: Double, bands: Seq[(Double, Double, String)]) - if (score >= 0.0) & (score <= 1.0) => - bands.find { case (l, u, _) => - (score >= l) & (score <= u) - } match { - case Some((_, _, bandString)) => Some(bandString) - case None => None - } - } - - /** - * aggregates labels into counts by lift band - * - * @param bandedLabels PairRDD of (bandString, label) - * @return Map of bandString -> (total count, count of positive labels) - */ - private[op] def aggregateBandedLabels - ( - bandedLabels: RDD[(String, Double)] - ): Map[String, (Long, Long)] = { - val countsPerBand = bandedLabels.countByKey() - val truesPerBand = bandedLabels.aggregateByKey(zeroValue = 0.0)( - { case (sum, label) => sum + label }, - { case (sumX, sumY) => sumX + sumY }) - .collectAsMap() - countsPerBand.map { case (band, count) => - band -> (count, truesPerBand.getOrElse(band, 0.0).toLong) - }.toMap - } - - /** - * Subsets scoresAndLabels to only records with scores greater than - * a threshold, categorizing them as predicted "yes" labels, then - * returns (# of true "yes" labels / # predicted yes) - * - * @param scoreAndLabels RDD of labels and scores - * @param threshold decision value, where scores >= thres are predicted "yes" - * @return Optional lift rate, None if denominator is 0 - */ - private[op] def thresholdLiftRate - ( - scoreAndLabels: RDD[(Double, Double)], - threshold: Double - ): Option[Double] = { - val (yesCount, totalCount) = scoreAndLabels.aggregate(zeroValue = (0L, 0L))({ - case ((yesCount, totalCount), (score, label)) => { - if (score < threshold) (yesCount, totalCount) - else (yesCount + label.toLong, totalCount + 1L) - } - }, { case ((yesCountX, totalCountX), (yesCountY, totalCountY)) => - (yesCountX + yesCountY, totalCountX + totalCountY) - }) - totalCount match { - case 0L => None - case _ => Some(yesCount.toDouble / totalCount.toDouble) - } - } - - /** - * Given a threshold decision value, calculates the lift - * in label prediction accuracy over random, as described here: - * https://en.wikipedia.org/wiki/Lift_(data_mining) - * - * @param overallRate # yes / total count across all data - * @param scoreAndLabels RDD of scores and labels - * @param threshold decision boundary for categorizing scores - * @return lift ratio, given threshold - */ - private[op] def liftRatio - ( - overallRate: Option[Double], - scoreAndLabels: RDD[(Double, Double)], - threshold: Double - ): Double = overallRate match { - case None => LiftMetrics.defaultLiftRatio - case Some(0.0) => LiftMetrics.defaultLiftRatio - case Some(rate) => { - val thresholdLift = thresholdLiftRate(scoreAndLabels, threshold) - thresholdLift match { - case None => LiftMetrics.defaultLiftRatio - case Some(thresholdRate) => thresholdRate / rate - } - } - } - - /** - * Formats lift data in one band into LiftMetricBand data, - * including lower bound of score band, upper bound, total record - * count per band, and lift (# trues / total) - * - * @param lower lower bound of band - * @param upper upper bound of band - * @param bandString String key of band e.g. "10-20" - * @param perBandCounts calculated total counts and counts of true labels - * @return LiftMetricBand container of metrics - */ - private[op] def formatLiftMetricBand - ( - lower: Double, - upper: Double, - bandString: String, - perBandCounts: Map[String, (Long, Long)] - ): LiftMetricBand = { - perBandCounts.get(bandString) match { - case Some((numTotal, numYes)) => { - val lift = numTotal match { - case 0L => None - case _ => Some(numYes.toDouble / numTotal) - } - LiftMetricBand( - group = bandString, - lowerBound = lower, - upperBound = upper, - rate = lift, - totalCount = numTotal, - yesCount = numYes, - noCount = numTotal - numYes - ) - } - case None => LiftMetricBand( - group = bandString, - lowerBound = lower, - upperBound = upper, - rate = None, - totalCount = 0L, - yesCount = 0L, - noCount = 0L - ) - } - } - -} - -/** - * Stores basic lift values for a specific band of scores - * - * @param group name / key for score band - * @param lowerBound minimum score represented in lift - * @param upperBound maximum score represented in lift - * @param rate optional calculated lift value, i.e. # yes / total count - * @param totalCount total number of records in score band - * @param yesCount number of yes records in score band - * @param noCount number of no records in score band - */ -case class LiftMetricBand -( - group: String, - lowerBound: Double, - upperBound: Double, - rate: Option[Double], - totalCount: Long, - yesCount: Long, - noCount: Long -) extends EvaluationMetrics - -/** - * Stores sequence of lift score band metrics - * as well as overall lift values - * - * @param liftMetricBands Seq of LiftMetricBand, calculated by LiftEvaluator - * @param threshold threshold used to categorize scores - * @param liftRatio overall lift ratio, given a specified threshold - * @param overallRate # yes records / # total records - */ -case class LiftMetrics -( - @JsonDeserialize(contentAs = classOf[LiftMetricBand]) - liftMetricBands: Seq[LiftMetricBand], - threshold: Double, - liftRatio: Double, - overallRate: Option[Double] -) extends EvaluationMetrics - -/** - * Companion object to LiftMetrics case class - * for storing default values - */ -object LiftMetrics { - val defaultThreshold = 0.5 - val defaultLiftRatio = 1.0 - - def empty: LiftMetrics = LiftMetrics(Seq(), defaultThreshold, defaultLiftRatio, None) -} diff --git a/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala b/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala index bc5595b797..3d4cc5c5af 100644 --- a/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala +++ b/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala @@ -34,6 +34,7 @@ import com.fasterxml.jackson.databind.annotation.JsonDeserialize import com.salesforce.op.UID import com.twitter.algebird.Operators._ import com.twitter.algebird.Tuple4Semigroup +import org.apache.spark.rdd.RDD import org.apache.spark.ml.linalg.Vector import org.apache.spark.sql.functions.col import org.apache.spark.sql.types.DoubleType @@ -46,12 +47,14 @@ import org.slf4j.LoggerFactory * This evaluator creates the specified number of bins and computes the statistics for each bin * and returns [[BinaryClassificationBinMetrics]]. * - * @param numOfBins number of bins to produce - * @param uid uid for instance + * @param numOfBins number of bins to produce + * @param isLargerBetter false, i.e. larger BrierScore values are not better + * @param uid uid for instance */ private[op] class OpBinScoreEvaluator ( - val numOfBins: Int = 100, + val numOfBins: Int = 10, + override val isLargerBetter: Boolean = false, uid: String = UID[OpBinScoreEvaluator] ) extends OpBinaryClassificationEvaluatorBase[BinaryClassificationBinMetrics](uid = uid) { @@ -66,60 +69,67 @@ private[op] class OpBinScoreEvaluator val labelColumnName = getLabelCol val dataToUse = makeDataToUse(data, labelColumnName) .select(col(getProbabilityCol), col(labelColumnName).cast(DoubleType)).rdd + val scoreAndLabels = dataToUse.map { + case Row(prob: Vector, label: Double) => (prob(1), label) + case Row(prob: Double, label: Double) => (prob, label) + } + evaluateScoreAndLabels(scoreAndLabels) + } - if (dataToUse.isEmpty()) { - log.warn("The dataset is empty. Returning empty metrics.") - BinaryClassificationBinMetrics(0.0, Seq(), Seq(), Seq(), Seq()) - } else { - val scoreAndLabels = dataToUse.map { - case Row(prob: Vector, label: Double) => (prob(1), label) - case Row(prob: Double, label: Double) => (prob, label) - } + def evaluateScoreAndLabels(scoreAndLabels: RDD[(Double, Double)]): BinaryClassificationBinMetrics = { - val (maxScore, minScore) = scoreAndLabels.map { - case (score, _) => (score, score) - }.fold(1.0, 0.0) { - case ((maxVal, minVal), (scoreMax, scoreMin)) => - (math.max(maxVal, scoreMax), math.min(minVal, scoreMin)) - } + val (maxScore, minScore) = scoreAndLabels.map { + case (score, _) => (score, score) + }.fold(1.0, 0.0) { + case ((maxVal, minVal), (scoreMax, scoreMin)) => + (math.max(maxVal, scoreMax), math.min(minVal, scoreMin)) + } - // Finding stats per bin -> avg score, avg conv rate, - // total num of data points and overall brier score. - implicit val sg = new Tuple4Semigroup[Double, Double, Long, Double]() - val stats = scoreAndLabels.map { - case (score, label) => - (getBinIndex(score, minScore, maxScore), (score, label, 1L, math.pow(score - label, 2))) - }.reduceByKey(_ + _).map { - case (bin, (scoreSum, labelSum, count, squaredError)) => - (bin, scoreSum / count, labelSum / count, count, squaredError) - }.collect() - - val zero = (new Array[Double](numOfBins), new Array[Double](numOfBins), new Array[Long](numOfBins), 0.0, 0L) - val (averageScore, averageConversionRate, numberOfDataPoints, brierScoreSum, numberOfPoints) = - stats.foldLeft(zero) { - case ((score, convRate, dataPoints, brierScoreSum, totalPoints), - (binIndex, avgScore, avgConvRate, counts, squaredError)) => - score(binIndex) = avgScore - convRate(binIndex) = avgConvRate - dataPoints(binIndex) = counts - (score, convRate, dataPoints, brierScoreSum + squaredError, totalPoints + counts) - } - - // binCenters is the center point in each bin. - // e.g., for bins [(0.0 - 0.5), (0.5 - 1.0)], bin centers are [0.25, 0.75]. - val diff = maxScore - minScore - val binCenters = for {i <- 0 until numOfBins} yield minScore + ((diff * i) / numOfBins) + (diff / (2 * numOfBins)) - - val metrics = BinaryClassificationBinMetrics( - BrierScore = brierScoreSum / numberOfPoints, - binCenters = binCenters, - numberOfDataPoints = numberOfDataPoints, - averageScore = averageScore, - averageConversionRate = averageConversionRate - ) - - log.info("Evaluated metrics: {}", metrics.toString) - metrics + // Finding stats per bin -> avg score, avg conv rate, + // total num of data points and overall brier score. + implicit val sg = new Tuple4Semigroup[Double, Double, Long, Double]() + val stats = scoreAndLabels.map { + case (score, label) => + (getBinIndex(score, minScore, maxScore), (score, label, 1L, math.pow(score - label, 2))) + }.reduceByKey(_ + _).map { + case (bin, (scoreSum, labelSum, count, squaredError)) => + (bin, scoreSum, labelSum, count, squaredError) + }.collect() + + stats.toList match { + case Nil => BinaryClassificationBinMetrics.empty + case _ => { + val zero = (new Array[Double](numOfBins), new Array[Double](numOfBins), + new Array[Long](numOfBins), new Array[Double](numOfBins), 0.0, 0L) + val (averageScore, averageConversionRate, numberOfDataPoints, sumOfLabels, brierScoreSum, numberOfPoints) = + stats.foldLeft(zero) { + case ((score, convRate, dataPoints, labelSums, brierScoreSum, totalPoints), + (binIndex, scoreSum, labelSum, counts, squaredError)) => + score(binIndex) = scoreSum / counts + convRate(binIndex) = labelSum / counts + dataPoints(binIndex) = counts + labelSums(binIndex) = labelSum + (score, convRate, dataPoints, labelSums, brierScoreSum + squaredError, totalPoints + counts) + } + + // binCenters is the center point in each bin. + // e.g., for bins [(0.0 - 0.5), (0.5 - 1.0)], bin centers are [0.25, 0.75]. + val diff = maxScore - minScore + val binCenters = for {i <- 0 until numOfBins} yield minScore + ((diff * i) / numOfBins) + (diff / (2 * numOfBins)) + + val metrics = BinaryClassificationBinMetrics( + BrierScore = brierScoreSum / numberOfPoints, + binSize = diff / numOfBins, + binCenters = binCenters, + numberOfDataPoints = numberOfDataPoints, + sumOfLabels = sumOfLabels, + averageScore = averageScore, + averageConversionRate = averageConversionRate + ) + + log.info("Evaluated metrics: {}", metrics.toString) + metrics + } } } @@ -134,20 +144,29 @@ private[op] class OpBinScoreEvaluator * Metrics of BinaryClassificationBinMetrics * * @param BrierScore brier score for overall dataset + * @param binSize size of each bin * @param binCenters center of each bin * @param numberOfDataPoints total number of data points in each bin + * @param * @param averageScore average score in each bin * @param averageConversionRate average conversion rate in each bin */ case class BinaryClassificationBinMetrics ( BrierScore: Double, + binSize: Double, @JsonDeserialize(contentAs = classOf[java.lang.Double]) binCenters: Seq[Double], @JsonDeserialize(contentAs = classOf[java.lang.Long]) numberOfDataPoints: Seq[Long], @JsonDeserialize(contentAs = classOf[java.lang.Double]) + sumOfLabels: Seq[Double], + @JsonDeserialize(contentAs = classOf[java.lang.Double]) averageScore: Seq[Double], @JsonDeserialize(contentAs = classOf[java.lang.Double]) averageConversionRate: Seq[Double] ) extends EvaluationMetrics + +object BinaryClassificationBinMetrics { + def empty: BinaryClassificationBinMetrics = BinaryClassificationBinMetrics(0.0, 0.0, Seq(), Seq(), Seq(), Seq(), Seq()) +} diff --git a/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala b/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala index 45f87594af..6ee00d73a6 100644 --- a/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala +++ b/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala @@ -83,7 +83,7 @@ private[op] class OpBinaryClassificationEvaluator if (rdd.isEmpty()) { log.warn("The dataset is empty. Returning empty metrics.") BinaryClassificationMetrics(0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, - Seq(), Seq(), Seq(), Seq(), LiftMetrics.empty) + Seq(), Seq(), Seq(), Seq(), BinaryClassificationBinMetrics.empty) } else { val multiclassMetrics = new MulticlassMetrics(rdd) val labels = multiclassMetrics.labels @@ -114,23 +114,24 @@ private[op] class OpBinaryClassificationEvaluator val falsePositiveRateByThreshold = sparkMLMetrics.roc().collect().map(_._1).slice(1, thresholds.length + 1) val aUROC = sparkMLMetrics.areaUnderROC() val aUPR = sparkMLMetrics.areaUnderPR() - val liftMetrics = new LiftEvaluator().evaluateScoreAndLabels(scoreAndLabels = scoreAndLabels) + val binMetrics = new OpBinScoreEvaluator().evaluateScoreAndLabels(scoreAndLabels = scoreAndLabels) val metrics = BinaryClassificationMetrics( Precision = precision, Recall = recall, F1 = f1, AuROC = aUROC, AuPR = aUPR, Error = error, TP = tp, TN = tn, FP = fp, FN = fn, thresholds, precisionByThreshold, recallByThreshold, falsePositiveRateByThreshold, - LiftMetrics = liftMetrics + BinaryClassificationBinMetrics = binMetrics ) log.info("Evaluated metrics: {}", metrics.toString) metrics } } - final protected def getBinaryEvaluatorMetric( - metricName: ClassificationEvalMetric, - dataset: Dataset[_], - default: => Double - ): Double = { + final protected def getBinaryEvaluatorMetric + ( + metricName: ClassificationEvalMetric, + dataset: Dataset[_], + default: => Double + ): Double = { import dataset.sparkSession.implicits._ val labelColName = getLabelCol val dataUse = makeDataToUse(dataset, labelColName) @@ -199,7 +200,7 @@ case class BinaryClassificationMetrics recallByThreshold: Seq[Double], @JsonDeserialize(contentAs = classOf[java.lang.Double]) falsePositiveRateByThreshold: Seq[Double], - LiftMetrics: LiftMetrics + BinaryClassificationBinMetrics: BinaryClassificationBinMetrics ) extends EvaluationMetrics { def rocCurve: Seq[(Double, Double)] = recallByThreshold.zip(falsePositiveRateByThreshold) diff --git a/core/src/test/scala/com/salesforce/op/evaluators/LiftEvaluatorTest.scala b/core/src/test/scala/com/salesforce/op/evaluators/LiftEvaluatorTest.scala deleted file mode 100644 index c920b8c345..0000000000 --- a/core/src/test/scala/com/salesforce/op/evaluators/LiftEvaluatorTest.scala +++ /dev/null @@ -1,171 +0,0 @@ -/* - * Copyright (c) 2017, Salesforce.com, Inc. - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * * Redistributions of source code must retain the above copyright notice, this - * list of conditions and the following disclaimer. - * - * * Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * * Neither the name of the copyright holder nor the names of its - * contributors may be used to endorse or promote products derived from - * this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE - * DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, - * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE - * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - */ - -package com.salesforce.op.evaluators - -import com.salesforce.op.test.TestSparkContext -import org.junit.runner.RunWith -import org.scalatest.FlatSpec -import org.scalatest.junit.JUnitRunner - - -@RunWith(classOf[JUnitRunner]) -class LiftEvaluatorTest extends FlatSpec with TestSparkContext { - - import spark.implicits._ - - lazy val labelSeq = for { - i <- 0 until 10 - j <- 0 until 10 - } yield { - if (j < i) 1.0 - else 0.0 - } - lazy val scoreSeq = (0.01 to 1.0 by 0.01) - lazy val scores = sc.parallelize(scoreSeq) - lazy val scoreAndLabels = sc.parallelize(scoreSeq.zip(labelSeq)) - lazy val emptyScoreAndLabels = sc.parallelize(Seq[(Double, Double)]()) - lazy val dataset = scoreAndLabels.toDF("probability", "label") - lazy val emptyDataset = emptyScoreAndLabels.toDF("probability", "label") - - "LiftEvaluator.getDefaultScoreBands" should "give proper default bands" in { - val bands = LiftEvaluator.getDefaultScoreBands(scores) - bands.head shouldBe(0.0, 0.1, "0-10") - bands.last shouldBe(0.9, 1.0, "90-100") - bands.size shouldBe 10 - } - - "LiftEvaluator.categorizeScoreIntoBand" should "categorize scores into correct bands" in { - val bands = Seq((0.0, 0.5, "A"), (0.5, 0.9, "B")) - LiftEvaluator.categorizeScoreIntoBand(0.3, bands) shouldBe Some("A") - LiftEvaluator.categorizeScoreIntoBand(0.95, bands) shouldBe None - a[MatchError] should be thrownBy { - LiftEvaluator.categorizeScoreIntoBand(-0.1, bands) - } - a[MatchError] should be thrownBy { - LiftEvaluator.categorizeScoreIntoBand(1.1, bands) - } - } - - "LiftEvaluator.aggregateBandedLabels" should "correctly count records within score bands" in { - val bandedLabels = sc.parallelize( - Seq(("A", 1.0), ("A", 0.0), ("B", 0.0), ("B", 0.0)) - ) - val perBandCounts = LiftEvaluator.aggregateBandedLabels(bandedLabels) - val (numTotalA, _) = perBandCounts("A") - val (_, numPositivesB) = perBandCounts("B") - - numTotalA shouldBe 2L - numPositivesB shouldBe 0L - } - - "LiftEvaluator.formatLiftMetricBand" should "format a LiftMetricBand as required" in { - val perBandCounts = Map("A" -> (4L, 2L)) - val metricBandA = LiftEvaluator.formatLiftMetricBand(0.0, 0.1, "A", perBandCounts) - metricBandA.group shouldBe "A" - metricBandA.lowerBound shouldBe 0.0 - metricBandA.upperBound shouldBe 0.1 - metricBandA.rate shouldBe Some(0.5) - metricBandA.totalCount shouldBe 4L - metricBandA.yesCount shouldBe 2L - metricBandA.noCount shouldBe 2L - - val metricBandB = LiftEvaluator.formatLiftMetricBand(0.1, 0.2, "B", perBandCounts) - metricBandB.group shouldBe "B" - metricBandB.lowerBound shouldBe 0.1 - metricBandB.upperBound shouldBe 0.2 - metricBandB.rate shouldBe None - metricBandB.totalCount shouldBe 0L - metricBandB.yesCount shouldBe 0L - metricBandB.noCount shouldBe 0L - } - - "LiftEvaluator.liftMetricBands" should "correctly calculate a Seq[LiftMetricBand]" in { - val liftSeq = LiftEvaluator.liftMetricBands(scoreAndLabels, LiftEvaluator.getDefaultScoreBands) - val band010 = liftSeq.find(_.group == "0-10").get - val band90100 = liftSeq.find(_.group == "90-100").get - - band010.rate shouldBe Some(0.0) - band010.lowerBound shouldBe 0.0 - band010.upperBound shouldBe 0.1 - band90100.rate shouldBe Some(0.9) - } - - "LiftEvaluator.liftMetricBands" should "correctly give defaults with empty RDD" in { - val liftSeq = LiftEvaluator.liftMetricBands( - emptyScoreAndLabels, - LiftEvaluator.getDefaultScoreBands - ) - val band010 = liftSeq.find(_.group == "0-10").get - val band90100 = liftSeq.find(_.group == "90-100").get - - band010.rate shouldBe None - band010.lowerBound shouldBe 0.0 - band010.upperBound shouldBe 0.1 - band90100.rate shouldBe None - } - - "LiftEvaluator.thresholdLiftRate" should "correctly calculate lift with a threshold" in { - val thresholdLift = LiftEvaluator.thresholdLiftRate(scoreAndLabels, 0.5) - val overallRate = LiftEvaluator.thresholdLiftRate(scoreAndLabels, 0.0) - val thresholdLiftZeros = LiftEvaluator.thresholdLiftRate( - sc.parallelize(Seq[(Double, Double)]((0.6, 0.0), (0.7, 0.0), (0.8, 0.0))), 0.5) - val thresholdLiftEmpty = LiftEvaluator.thresholdLiftRate(emptyScoreAndLabels, 0.5) - thresholdLift shouldBe Some(0.6862745098039216) - overallRate shouldBe Some(0.45) - thresholdLiftZeros shouldBe Some(0.0) - thresholdLiftEmpty shouldBe None - } - - "LiftEvaluator.evaluateScoreAndLabels" should "correctly produce LiftMetrics" in { - val evaluator = new LiftEvaluator() - val liftMetrics = evaluator.evaluateScoreAndLabels(scoreAndLabels) - val liftMetricsEmpty = evaluator.evaluateScoreAndLabels(emptyScoreAndLabels) - liftMetrics.liftMetricBands.size shouldBe LiftEvaluator.getDefaultScoreBands(scores).size - liftMetrics.liftRatio shouldBe 1.5250544662309369 - liftMetrics.overallRate shouldBe Some(0.45) - liftMetricsEmpty.liftMetricBands.size shouldBe LiftEvaluator.getDefaultScoreBands(scores).size - liftMetricsEmpty.liftRatio shouldBe LiftMetrics.defaultLiftRatio - liftMetricsEmpty.overallRate shouldBe None - } - - "LiftEvaluator.evaluateAll" should "correctly produce LiftMetrics" in { - val evaluator = new LiftEvaluator() - val liftMetrics = evaluator.evaluateAll(dataset) - val liftMetricsEmpty = evaluator.evaluateAll(emptyDataset) - liftMetrics.liftMetricBands.size shouldBe LiftEvaluator.getDefaultScoreBands(scores).size - liftMetrics.liftRatio shouldBe 1.5250544662309369 - liftMetrics.overallRate shouldBe Some(0.45) - liftMetricsEmpty.liftMetricBands.size shouldBe 0 - liftMetricsEmpty.liftRatio shouldBe LiftMetrics.defaultLiftRatio - liftMetricsEmpty.overallRate shouldBe None - } - -} diff --git a/core/src/test/scala/com/salesforce/op/evaluators/OpBinScoreEvaluatorTest.scala b/core/src/test/scala/com/salesforce/op/evaluators/OpBinScoreEvaluatorTest.scala index ac2b51a1ce..bc39a6d2a7 100644 --- a/core/src/test/scala/com/salesforce/op/evaluators/OpBinScoreEvaluatorTest.scala +++ b/core/src/test/scala/com/salesforce/op/evaluators/OpBinScoreEvaluatorTest.scala @@ -75,8 +75,10 @@ class OpBinScoreEvaluatorTest extends FlatSpec with TestSparkContext { metrics shouldBe BinaryClassificationBinMetrics( 0.09800605366, + 0.25, Seq(0.125, 0.375, 0.625, 0.875), Seq(2, 0, 1, 2), + Seq(0.0, 0.0, 0.0, 2.0), Seq(0.003205, 0.0, 0.7, 0.99999), Seq(0.0, 0.0, 0.0, 1.0) ) @@ -89,8 +91,10 @@ class OpBinScoreEvaluatorTest extends FlatSpec with TestSparkContext { metrics shouldBe BinaryClassificationBinMetrics( 40.999986666733335, + 3.2499975, Seq(0.62500875, 3.87500625, 7.125003749999999, 10.37500125), Seq(2, 0, 0, 1), + Seq(1.0, 0.0, 0.0, 1.0), Seq(0.49999999999999994, 0.0, 0.0, 12.0), Seq(0.5, 0.0, 0.0, 1.0) ) @@ -107,7 +111,7 @@ class OpBinScoreEvaluatorTest extends FlatSpec with TestSparkContext { val metrics = new OpBinScoreEvaluator(numOfBins = 10) .setLabelCol(labelEmptyData.name).setPredictionCol(predictionEmptyData.name).evaluateAll(emptyData) - metrics shouldBe BinaryClassificationBinMetrics(0.0, Seq(), Seq(), Seq(), Seq()) + metrics shouldBe BinaryClassificationBinMetrics(0.0, 0.0, Seq(), Seq(), Seq(), Seq(), Seq()) } it should "evaluate bin metrics for skewed data" in { @@ -116,8 +120,10 @@ class OpBinScoreEvaluatorTest extends FlatSpec with TestSparkContext { metrics shouldBe BinaryClassificationBinMetrics( 7.294225500000013E-4, + 0.2, Seq(0.1, 0.30000000000000004, 0.5, 0.7, 0.9), Seq(0, 0, 0, 0, 4), + Seq(0.0, 0.0, 0.0, 0.0, 4.0), Seq(0.0, 0.0, 0.0, 0.0, 0.98617), Seq(0.0, 0.0, 0.0, 0.0, 1.0) ) diff --git a/core/src/test/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluatorTest.scala b/core/src/test/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluatorTest.scala index d520e296d2..1b9d9cc353 100644 --- a/core/src/test/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluatorTest.scala +++ b/core/src/test/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluatorTest.scala @@ -43,6 +43,7 @@ import org.apache.spark.ml.tuning.ParamGridBuilder import org.apache.spark.mllib.evaluation.MulticlassMetrics import org.apache.spark.rdd.RDD import org.apache.spark.sql.Row +import org.apache.spark.sql.functions.sum import org.junit.runner.RunWith import org.scalatest.FlatSpec import org.scalatest.junit.JUnitRunner @@ -149,7 +150,8 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext val (tp, tn, fp, fn, precision, recall, f1) = getPosNegValues( flattenedData2.select(predValue.name, test_label.name).rdd ) - val overallLiftRate = (tp + fn) / (tp + tn + fp + fn) + val numRows = transformedData.count() + val sumOfLabels = transformedData.select(sum(test_label.name)).collect()(0)(0) tp.toDouble shouldBe metrics.TP tn.toDouble shouldBe metrics.TN @@ -161,8 +163,8 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext f1 shouldBe metrics.F1 1.0 - sparkMulticlassEvaluator.setMetricName(Error.sparkEntryName).evaluate(flattenedData2) shouldBe metrics.Error - LiftEvaluator.getDefaultScoreBands(sc.emptyRDD).size shouldBe metrics.LiftMetrics.liftMetricBands.size - Some(overallLiftRate) shouldBe metrics.LiftMetrics.overallRate + numRows shouldBe metrics.BinaryClassificationBinMetrics.numberOfDataPoints.sum + sumOfLabels shouldBe metrics.BinaryClassificationBinMetrics.sumOfLabels.sum } it should "evaluate the metrics with one prediction input" in { @@ -173,7 +175,8 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext transformedData2.select(prediction.name, test_label.name).rdd .map(r => Row(r.getMap[String, Double](0).toMap.toPrediction.prediction, r.getDouble(1))) ) - val overallLiftRate = (tp + fn) / (tp + tn + fp + fn) + val numRows = transformedData2.count() + val sumOfLabels = transformedData2.select(sum(test_label.name)).collect()(0)(0) tp.toDouble shouldBe metrics.TP tn.toDouble shouldBe metrics.TN @@ -184,8 +187,8 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext metrics.Recall shouldBe recall metrics.F1 shouldBe f1 - LiftEvaluator.getDefaultScoreBands(sc.emptyRDD).size shouldBe metrics.LiftMetrics.liftMetricBands.size - Some(overallLiftRate) shouldBe metrics.LiftMetrics.overallRate + numRows shouldBe metrics.BinaryClassificationBinMetrics.numberOfDataPoints.sum + sumOfLabels shouldBe metrics.BinaryClassificationBinMetrics.sumOfLabels.sum } it should "evaluate the metrics on dataset with only the label and prediction 0" in { @@ -203,13 +206,10 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext metricsZero.Recall shouldBe 0.0 metricsZero.Error shouldBe 0.0 - metricsZero.LiftMetrics.liftMetricBands.head.rate shouldBe Some(0.0) - metricsZero.LiftMetrics.liftMetricBands.head.yesCount shouldBe 0L - metricsZero.LiftMetrics.liftMetricBands.head.noCount shouldBe 1L - metricsZero.LiftMetrics.liftMetricBands.tail.head.rate shouldBe None - metricsZero.LiftMetrics.threshold shouldBe LiftMetrics.defaultThreshold - metricsZero.LiftMetrics.overallRate shouldBe Some(0.0) - metricsZero.LiftMetrics.liftRatio shouldBe LiftMetrics.defaultLiftRatio + metricsZero.BinaryClassificationBinMetrics.BrierScore shouldBe 0.0 + metricsZero.BinaryClassificationBinMetrics.numberOfDataPoints.sum shouldBe 1L + metricsZero.BinaryClassificationBinMetrics.sumOfLabels.sum shouldBe 0.0 + metricsZero.BinaryClassificationBinMetrics.averageConversionRate(0) shouldBe 0.0 } @@ -226,13 +226,10 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext metricsOne.Recall shouldBe 1.0 metricsOne.Error shouldBe 0.0 - metricsOne.LiftMetrics.liftMetricBands.head.rate shouldBe Some(1.0) - metricsOne.LiftMetrics.liftMetricBands.head.yesCount shouldBe 1L - metricsOne.LiftMetrics.liftMetricBands.head.noCount shouldBe 0L - metricsOne.LiftMetrics.liftMetricBands.tail.head.rate shouldBe None - metricsOne.LiftMetrics.threshold shouldBe LiftMetrics.defaultThreshold - metricsOne.LiftMetrics.overallRate shouldBe Some(1.0) - metricsOne.LiftMetrics.liftRatio shouldBe 1.0 + metricsOne.BinaryClassificationBinMetrics.BrierScore shouldBe 1.0 + metricsOne.BinaryClassificationBinMetrics.numberOfDataPoints.sum shouldBe 1L + metricsOne.BinaryClassificationBinMetrics.sumOfLabels.sum shouldBe 1.0 + metricsOne.BinaryClassificationBinMetrics.averageConversionRate(0) shouldBe 1.0 } private def getPosNegValues(rdd: RDD[Row]): (Double, Double, Double, Double, Double, Double, Double) = { diff --git a/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala b/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala index 90fa800416..dcc80235e8 100644 --- a/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala +++ b/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala @@ -55,11 +55,14 @@ class ModelSelectorSummaryTest extends FlatSpec with TestSparkContext { trainEvaluation = BinaryClassificationMetrics(Precision = 0.1, Recall = 0.2, F1 = 0.3, AuROC = 0.4, AuPR = 0.5, Error = 0.6, TP = 0.7, TN = 0.8, FP = 0.9, FN = 1.0, thresholds = Seq(1.1), precisionByThreshold = Seq(1.2), recallByThreshold = Seq(1.3), falsePositiveRateByThreshold = Seq(1.4), - LiftMetrics = LiftMetrics( - liftMetricBands = Seq(LiftMetricBand("0-10", 0.0, 0.1, Some(0.1), 10L, 1L, 9L)), - threshold = LiftMetrics.defaultThreshold, - liftRatio = LiftMetrics.defaultLiftRatio, - overallRate = Some(0.1) + BinaryClassificationBinMetrics = BinaryClassificationBinMetrics( + 1.0, + 0.1, + Seq(0.2, 0.3), + Seq(3L, 0L, 5L), + Seq(6.0, 7.0), + Seq(0.4, 0.5), + Seq(0.8, 0.9) )), holdoutEvaluation = Option(RegressionMetrics(RootMeanSquaredError = 1.3, MeanSquaredError = 1.4, R2 = 1.5, MeanAbsoluteError = 1.6)) From 92809a00ac42d90c16d09fb44968bc416f5a88a3 Mon Sep 17 00:00:00 2001 From: shae-selix Date: Wed, 27 Mar 2019 18:11:31 -0700 Subject: [PATCH 08/14] Fixing ModelInsightsTest --- core/src/test/scala/com/salesforce/op/ModelInsightsTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/com/salesforce/op/ModelInsightsTest.scala b/core/src/test/scala/com/salesforce/op/ModelInsightsTest.scala index 007cc80ccf..84e0ad2c29 100644 --- a/core/src/test/scala/com/salesforce/op/ModelInsightsTest.scala +++ b/core/src/test/scala/com/salesforce/op/ModelInsightsTest.scala @@ -328,7 +328,7 @@ class ModelInsightsTest extends FlatSpec with PassengerSparkFixtureTest { pretty should not include m.modelName } } - pretty should include("area under precision-recall | 1.0") + pretty should include("area under precision-recall | 1.0") pretty should include("Model Evaluation Metrics") pretty should include("Top Model Insights") pretty should include("Top Positive Correlations") From f3eba657c5c83fb5c0e40e3970a700f6dcd403d2 Mon Sep 17 00:00:00 2001 From: shae-selix Date: Wed, 27 Mar 2019 23:29:18 -0700 Subject: [PATCH 09/14] Fixing scalastyle --- .../com/salesforce/op/evaluators/OpBinScoreEvaluator.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala b/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala index 3d4cc5c5af..88b7783311 100644 --- a/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala +++ b/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala @@ -115,7 +115,8 @@ private[op] class OpBinScoreEvaluator // binCenters is the center point in each bin. // e.g., for bins [(0.0 - 0.5), (0.5 - 1.0)], bin centers are [0.25, 0.75]. val diff = maxScore - minScore - val binCenters = for {i <- 0 until numOfBins} yield minScore + ((diff * i) / numOfBins) + (diff / (2 * numOfBins)) + val binCenters = for {i <- 0 until numOfBins} + yield minScore + ((diff * i) / numOfBins) + (diff / (2 * numOfBins)) val metrics = BinaryClassificationBinMetrics( BrierScore = brierScoreSum / numberOfPoints, @@ -168,5 +169,6 @@ case class BinaryClassificationBinMetrics ) extends EvaluationMetrics object BinaryClassificationBinMetrics { - def empty: BinaryClassificationBinMetrics = BinaryClassificationBinMetrics(0.0, 0.0, Seq(), Seq(), Seq(), Seq(), Seq()) + def empty: BinaryClassificationBinMetrics = + BinaryClassificationBinMetrics(0.0, 0.0, Seq(), Seq(), Seq(), Seq(), Seq()) } From 2d704012fb0ab27b3bc78598f6014720c272ab15 Mon Sep 17 00:00:00 2001 From: shae-selix Date: Thu, 28 Mar 2019 12:57:47 -0700 Subject: [PATCH 10/14] Removing binmetrics from BinaryClassificationMetrics --- .../op/evaluators/EvaluationMetrics.scala | 1 - .../OpBinaryClassificationEvaluator.scala | 10 +++------ .../com/salesforce/op/ModelInsightsTest.scala | 2 +- .../com/salesforce/op/OpWorkflowTest.scala | 2 +- .../OpBinaryClassificationEvaluatorTest.scala | 22 ------------------- .../selector/ModelSelectorSummaryTest.scala | 11 +--------- 6 files changed, 6 insertions(+), 42 deletions(-) diff --git a/core/src/main/scala/com/salesforce/op/evaluators/EvaluationMetrics.scala b/core/src/main/scala/com/salesforce/op/evaluators/EvaluationMetrics.scala index b25a8af29f..f227cd79b0 100644 --- a/core/src/main/scala/com/salesforce/op/evaluators/EvaluationMetrics.scala +++ b/core/src/main/scala/com/salesforce/op/evaluators/EvaluationMetrics.scala @@ -140,7 +140,6 @@ object BinaryClassEvalMetrics extends Enum[ClassificationEvalMetric] { case object FP extends ClassificationEvalMetric("FP", "false positive") case object FN extends ClassificationEvalMetric("FN", "false negative") case object BrierScore extends ClassificationEvalMetric("brierScore", "brier score") - case object LiftMetrics extends ClassificationEvalMetric("liftMetrics", "lift plot") } /** diff --git a/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala b/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala index 6ee00d73a6..1caa46f3d6 100644 --- a/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala +++ b/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala @@ -82,8 +82,7 @@ private[op] class OpBinaryClassificationEvaluator if (rdd.isEmpty()) { log.warn("The dataset is empty. Returning empty metrics.") - BinaryClassificationMetrics(0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, - Seq(), Seq(), Seq(), Seq(), BinaryClassificationBinMetrics.empty) + BinaryClassificationMetrics(0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, Seq(), Seq(), Seq(), Seq()) } else { val multiclassMetrics = new MulticlassMetrics(rdd) val labels = multiclassMetrics.labels @@ -114,12 +113,10 @@ private[op] class OpBinaryClassificationEvaluator val falsePositiveRateByThreshold = sparkMLMetrics.roc().collect().map(_._1).slice(1, thresholds.length + 1) val aUROC = sparkMLMetrics.areaUnderROC() val aUPR = sparkMLMetrics.areaUnderPR() - val binMetrics = new OpBinScoreEvaluator().evaluateScoreAndLabels(scoreAndLabels = scoreAndLabels) val metrics = BinaryClassificationMetrics( Precision = precision, Recall = recall, F1 = f1, AuROC = aUROC, AuPR = aUPR, Error = error, TP = tp, TN = tn, FP = fp, FN = fn, - thresholds, precisionByThreshold, recallByThreshold, falsePositiveRateByThreshold, - BinaryClassificationBinMetrics = binMetrics + thresholds, precisionByThreshold, recallByThreshold, falsePositiveRateByThreshold ) log.info("Evaluated metrics: {}", metrics.toString) metrics @@ -199,8 +196,7 @@ case class BinaryClassificationMetrics @JsonDeserialize(contentAs = classOf[java.lang.Double]) recallByThreshold: Seq[Double], @JsonDeserialize(contentAs = classOf[java.lang.Double]) - falsePositiveRateByThreshold: Seq[Double], - BinaryClassificationBinMetrics: BinaryClassificationBinMetrics + falsePositiveRateByThreshold: Seq[Double] ) extends EvaluationMetrics { def rocCurve: Seq[(Double, Double)] = recallByThreshold.zip(falsePositiveRateByThreshold) diff --git a/core/src/test/scala/com/salesforce/op/ModelInsightsTest.scala b/core/src/test/scala/com/salesforce/op/ModelInsightsTest.scala index 84e0ad2c29..cf26ec6e53 100644 --- a/core/src/test/scala/com/salesforce/op/ModelInsightsTest.scala +++ b/core/src/test/scala/com/salesforce/op/ModelInsightsTest.scala @@ -328,7 +328,7 @@ class ModelInsightsTest extends FlatSpec with PassengerSparkFixtureTest { pretty should not include m.modelName } } - pretty should include("area under precision-recall | 1.0") + pretty should include regex raw"area under precision-recall\s+|\s+1.0" pretty should include("Model Evaluation Metrics") pretty should include("Top Model Insights") pretty should include("Top Positive Correlations") diff --git a/core/src/test/scala/com/salesforce/op/OpWorkflowTest.scala b/core/src/test/scala/com/salesforce/op/OpWorkflowTest.scala index 42c671831c..cb99bee317 100644 --- a/core/src/test/scala/com/salesforce/op/OpWorkflowTest.scala +++ b/core/src/test/scala/com/salesforce/op/OpWorkflowTest.scala @@ -377,8 +377,8 @@ class OpWorkflowTest extends FlatSpec with PassengerSparkFixtureTest { val prettySummary = fittedWorkflow.summaryPretty() log.info(prettySummary) + prettySummary should include regex raw"area under precision-recall\s+|\s+1.0\s+|\s+0.0" prettySummary should include("Selected Model - OpLogisticRegression") - prettySummary should include("area under precision-recall | 1.0 | 0.0") prettySummary should include("Model Evaluation Metrics") prettySummary should include("Top Model Insights") prettySummary should include("Top Positive Correlations") diff --git a/core/src/test/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluatorTest.scala b/core/src/test/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluatorTest.scala index 1b9d9cc353..e163847162 100644 --- a/core/src/test/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluatorTest.scala +++ b/core/src/test/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluatorTest.scala @@ -33,7 +33,6 @@ package com.salesforce.op.evaluators import com.salesforce.op.evaluators.BinaryClassEvalMetrics._ import com.salesforce.op.features.types._ import com.salesforce.op.stages.impl.classification.{BinaryClassificationModelSelector, OpLogisticRegression} -import com.salesforce.op.stages.impl.selector.ModelSelectorNames.EstimatorType import com.salesforce.op.test.{TestFeatureBuilder, TestSparkContext} import org.apache.spark.ml.Transformer import org.apache.spark.ml.evaluation._ @@ -43,7 +42,6 @@ import org.apache.spark.ml.tuning.ParamGridBuilder import org.apache.spark.mllib.evaluation.MulticlassMetrics import org.apache.spark.rdd.RDD import org.apache.spark.sql.Row -import org.apache.spark.sql.functions.sum import org.junit.runner.RunWith import org.scalatest.FlatSpec import org.scalatest.junit.JUnitRunner @@ -150,8 +148,6 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext val (tp, tn, fp, fn, precision, recall, f1) = getPosNegValues( flattenedData2.select(predValue.name, test_label.name).rdd ) - val numRows = transformedData.count() - val sumOfLabels = transformedData.select(sum(test_label.name)).collect()(0)(0) tp.toDouble shouldBe metrics.TP tn.toDouble shouldBe metrics.TN @@ -162,9 +158,6 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext recall shouldBe metrics.Recall f1 shouldBe metrics.F1 1.0 - sparkMulticlassEvaluator.setMetricName(Error.sparkEntryName).evaluate(flattenedData2) shouldBe metrics.Error - - numRows shouldBe metrics.BinaryClassificationBinMetrics.numberOfDataPoints.sum - sumOfLabels shouldBe metrics.BinaryClassificationBinMetrics.sumOfLabels.sum } it should "evaluate the metrics with one prediction input" in { @@ -175,8 +168,6 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext transformedData2.select(prediction.name, test_label.name).rdd .map(r => Row(r.getMap[String, Double](0).toMap.toPrediction.prediction, r.getDouble(1))) ) - val numRows = transformedData2.count() - val sumOfLabels = transformedData2.select(sum(test_label.name)).collect()(0)(0) tp.toDouble shouldBe metrics.TP tn.toDouble shouldBe metrics.TN @@ -186,9 +177,6 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext metrics.Precision shouldBe precision metrics.Recall shouldBe recall metrics.F1 shouldBe f1 - - numRows shouldBe metrics.BinaryClassificationBinMetrics.numberOfDataPoints.sum - sumOfLabels shouldBe metrics.BinaryClassificationBinMetrics.sumOfLabels.sum } it should "evaluate the metrics on dataset with only the label and prediction 0" in { @@ -205,11 +193,6 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext metricsZero.Precision shouldBe 0.0 metricsZero.Recall shouldBe 0.0 metricsZero.Error shouldBe 0.0 - - metricsZero.BinaryClassificationBinMetrics.BrierScore shouldBe 0.0 - metricsZero.BinaryClassificationBinMetrics.numberOfDataPoints.sum shouldBe 1L - metricsZero.BinaryClassificationBinMetrics.sumOfLabels.sum shouldBe 0.0 - metricsZero.BinaryClassificationBinMetrics.averageConversionRate(0) shouldBe 0.0 } @@ -225,11 +208,6 @@ class OpBinaryClassificationEvaluatorTest extends FlatSpec with TestSparkContext metricsOne.Precision shouldBe 1.0 metricsOne.Recall shouldBe 1.0 metricsOne.Error shouldBe 0.0 - - metricsOne.BinaryClassificationBinMetrics.BrierScore shouldBe 1.0 - metricsOne.BinaryClassificationBinMetrics.numberOfDataPoints.sum shouldBe 1L - metricsOne.BinaryClassificationBinMetrics.sumOfLabels.sum shouldBe 1.0 - metricsOne.BinaryClassificationBinMetrics.averageConversionRate(0) shouldBe 1.0 } private def getPosNegValues(rdd: RDD[Row]): (Double, Double, Double, Double, Double, Double, Double) = { diff --git a/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala b/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala index dcc80235e8..1bd9595542 100644 --- a/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala +++ b/core/src/test/scala/com/salesforce/op/stages/impl/selector/ModelSelectorSummaryTest.scala @@ -54,16 +54,7 @@ class ModelSelectorSummaryTest extends FlatSpec with TestSparkContext { validationResults = Seq(ModelEvaluation("test4", "test5", "test6", SingleMetric("test7", 0.1), Map.empty)), trainEvaluation = BinaryClassificationMetrics(Precision = 0.1, Recall = 0.2, F1 = 0.3, AuROC = 0.4, AuPR = 0.5, Error = 0.6, TP = 0.7, TN = 0.8, FP = 0.9, FN = 1.0, thresholds = Seq(1.1), - precisionByThreshold = Seq(1.2), recallByThreshold = Seq(1.3), falsePositiveRateByThreshold = Seq(1.4), - BinaryClassificationBinMetrics = BinaryClassificationBinMetrics( - 1.0, - 0.1, - Seq(0.2, 0.3), - Seq(3L, 0L, 5L), - Seq(6.0, 7.0), - Seq(0.4, 0.5), - Seq(0.8, 0.9) - )), + precisionByThreshold = Seq(1.2), recallByThreshold = Seq(1.3), falsePositiveRateByThreshold = Seq(1.4)), holdoutEvaluation = Option(RegressionMetrics(RootMeanSquaredError = 1.3, MeanSquaredError = 1.4, R2 = 1.5, MeanAbsoluteError = 1.6)) ) From 0d22997a77e949f584b88a237fc9919feeb9b6c5 Mon Sep 17 00:00:00 2001 From: shae-selix Date: Thu, 28 Mar 2019 14:34:33 -0700 Subject: [PATCH 11/14] Cleaning up some comments --- .../com/salesforce/op/evaluators/OpBinScoreEvaluator.scala | 5 ++--- .../op/evaluators/OpBinaryClassificationEvaluator.scala | 1 - 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala b/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala index 88b7783311..1d474d1d16 100644 --- a/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala +++ b/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala @@ -53,8 +53,7 @@ import org.slf4j.LoggerFactory */ private[op] class OpBinScoreEvaluator ( - val numOfBins: Int = 10, - override val isLargerBetter: Boolean = false, + val numOfBins: Int = 100, uid: String = UID[OpBinScoreEvaluator] ) extends OpBinaryClassificationEvaluatorBase[BinaryClassificationBinMetrics](uid = uid) { @@ -148,7 +147,7 @@ private[op] class OpBinScoreEvaluator * @param binSize size of each bin * @param binCenters center of each bin * @param numberOfDataPoints total number of data points in each bin - * @param + * @param sumOfLabels sum of the label in each bin * @param averageScore average score in each bin * @param averageConversionRate average conversion rate in each bin */ diff --git a/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala b/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala index 1caa46f3d6..693e725455 100644 --- a/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala +++ b/core/src/main/scala/com/salesforce/op/evaluators/OpBinaryClassificationEvaluator.scala @@ -199,6 +199,5 @@ case class BinaryClassificationMetrics falsePositiveRateByThreshold: Seq[Double] ) extends EvaluationMetrics { def rocCurve: Seq[(Double, Double)] = recallByThreshold.zip(falsePositiveRateByThreshold) - def prCurve: Seq[(Double, Double)] = precisionByThreshold.zip(recallByThreshold) } From 0fa01787853c7f4f9330ab5fbd9a7fac3a0760f1 Mon Sep 17 00:00:00 2001 From: shae-selix Date: Thu, 28 Mar 2019 20:59:07 -0700 Subject: [PATCH 12/14] Cleaning up some comments --- .../scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala b/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala index 1d474d1d16..eb7a510795 100644 --- a/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala +++ b/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala @@ -48,7 +48,6 @@ import org.slf4j.LoggerFactory * and returns [[BinaryClassificationBinMetrics]]. * * @param numOfBins number of bins to produce - * @param isLargerBetter false, i.e. larger BrierScore values are not better * @param uid uid for instance */ private[op] class OpBinScoreEvaluator From 75dcd59957579ef2417243d5fcb6e38380b20c72 Mon Sep 17 00:00:00 2001 From: shae-selix Date: Mon, 1 Apr 2019 16:14:47 -0700 Subject: [PATCH 13/14] `sumOfLabels` -> `numberOfPositiveLabels` --- .../op/evaluators/OpBinScoreEvaluator.scala | 41 ++++++++++--------- .../evaluators/OpBinScoreEvaluatorTest.scala | 6 +-- 2 files changed, 24 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala b/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala index eb7a510795..9e05cce88e 100644 --- a/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala +++ b/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala @@ -85,29 +85,30 @@ private[op] class OpBinScoreEvaluator // Finding stats per bin -> avg score, avg conv rate, // total num of data points and overall brier score. - implicit val sg = new Tuple4Semigroup[Double, Double, Long, Double]() + implicit val sg = new Tuple4Semigroup[Double, Long, Long, Double]() val stats = scoreAndLabels.map { case (score, label) => - (getBinIndex(score, minScore, maxScore), (score, label, 1L, math.pow(score - label, 2))) + (getBinIndex(score, minScore, maxScore), + (score, if (label > 0.0) 1L else 0L, 1L, math.pow(score - label, 2))) }.reduceByKey(_ + _).map { - case (bin, (scoreSum, labelSum, count, squaredError)) => - (bin, scoreSum, labelSum, count, squaredError) + case (bin, (scoreSum, positiveCount, count, squaredError)) => + (bin, scoreSum, positiveCount, count, squaredError) }.collect() stats.toList match { case Nil => BinaryClassificationBinMetrics.empty case _ => { val zero = (new Array[Double](numOfBins), new Array[Double](numOfBins), - new Array[Long](numOfBins), new Array[Double](numOfBins), 0.0, 0L) - val (averageScore, averageConversionRate, numberOfDataPoints, sumOfLabels, brierScoreSum, numberOfPoints) = + new Array[Long](numOfBins), new Array[Long](numOfBins), 0.0, 0L) + val (averageScore, averageConversionRate, numberOfDataPoints, positiveLabels, brierScoreSum, numberOfPoints) = stats.foldLeft(zero) { - case ((score, convRate, dataPoints, labelSums, brierScoreSum, totalPoints), - (binIndex, scoreSum, labelSum, counts, squaredError)) => + case ((score, convRate, dataPoints, positiveLabels, brierScoreSum, totalPoints), + (binIndex, scoreSum, positiveCount, counts, squaredError)) => score(binIndex) = scoreSum / counts - convRate(binIndex) = labelSum / counts + convRate(binIndex) = positiveCount.toDouble / counts dataPoints(binIndex) = counts - labelSums(binIndex) = labelSum - (score, convRate, dataPoints, labelSums, brierScoreSum + squaredError, totalPoints + counts) + positiveLabels(binIndex) = positiveCount + (score, convRate, dataPoints, positiveLabels, brierScoreSum + squaredError, totalPoints + counts) } // binCenters is the center point in each bin. @@ -121,7 +122,7 @@ private[op] class OpBinScoreEvaluator binSize = diff / numOfBins, binCenters = binCenters, numberOfDataPoints = numberOfDataPoints, - sumOfLabels = sumOfLabels, + numberOfPositiveLabels = positiveLabels, averageScore = averageScore, averageConversionRate = averageConversionRate ) @@ -142,13 +143,13 @@ private[op] class OpBinScoreEvaluator /** * Metrics of BinaryClassificationBinMetrics * - * @param BrierScore brier score for overall dataset - * @param binSize size of each bin - * @param binCenters center of each bin - * @param numberOfDataPoints total number of data points in each bin - * @param sumOfLabels sum of the label in each bin - * @param averageScore average score in each bin - * @param averageConversionRate average conversion rate in each bin + * @param BrierScore brier score for overall dataset + * @param binSize size of each bin + * @param binCenters center of each bin + * @param numberOfDataPoints total number of data points in each bin + * @param numberOfPositiveLabels sum of the label in each bin + * @param averageScore average score in each bin + * @param averageConversionRate average conversion rate in each bin */ case class BinaryClassificationBinMetrics ( @@ -159,7 +160,7 @@ case class BinaryClassificationBinMetrics @JsonDeserialize(contentAs = classOf[java.lang.Long]) numberOfDataPoints: Seq[Long], @JsonDeserialize(contentAs = classOf[java.lang.Double]) - sumOfLabels: Seq[Double], + numberOfPositiveLabels: Seq[Long], @JsonDeserialize(contentAs = classOf[java.lang.Double]) averageScore: Seq[Double], @JsonDeserialize(contentAs = classOf[java.lang.Double]) diff --git a/core/src/test/scala/com/salesforce/op/evaluators/OpBinScoreEvaluatorTest.scala b/core/src/test/scala/com/salesforce/op/evaluators/OpBinScoreEvaluatorTest.scala index bc39a6d2a7..f7f77985a9 100644 --- a/core/src/test/scala/com/salesforce/op/evaluators/OpBinScoreEvaluatorTest.scala +++ b/core/src/test/scala/com/salesforce/op/evaluators/OpBinScoreEvaluatorTest.scala @@ -78,7 +78,7 @@ class OpBinScoreEvaluatorTest extends FlatSpec with TestSparkContext { 0.25, Seq(0.125, 0.375, 0.625, 0.875), Seq(2, 0, 1, 2), - Seq(0.0, 0.0, 0.0, 2.0), + Seq(0, 0, 0, 2), Seq(0.003205, 0.0, 0.7, 0.99999), Seq(0.0, 0.0, 0.0, 1.0) ) @@ -94,7 +94,7 @@ class OpBinScoreEvaluatorTest extends FlatSpec with TestSparkContext { 3.2499975, Seq(0.62500875, 3.87500625, 7.125003749999999, 10.37500125), Seq(2, 0, 0, 1), - Seq(1.0, 0.0, 0.0, 1.0), + Seq(1, 0, 0, 1), Seq(0.49999999999999994, 0.0, 0.0, 12.0), Seq(0.5, 0.0, 0.0, 1.0) ) @@ -123,7 +123,7 @@ class OpBinScoreEvaluatorTest extends FlatSpec with TestSparkContext { 0.2, Seq(0.1, 0.30000000000000004, 0.5, 0.7, 0.9), Seq(0, 0, 0, 0, 4), - Seq(0.0, 0.0, 0.0, 0.0, 4.0), + Seq(0, 0, 0, 0, 4), Seq(0.0, 0.0, 0.0, 0.0, 0.98617), Seq(0.0, 0.0, 0.0, 0.0, 1.0) ) From fe94120a66d74565f0b675af1f1524b2f7723b3d Mon Sep 17 00:00:00 2001 From: shae-selix Date: Mon, 1 Apr 2019 16:16:50 -0700 Subject: [PATCH 14/14] small comment fix --- .../com/salesforce/op/evaluators/OpBinScoreEvaluator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala b/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala index 9e05cce88e..8e22c61375 100644 --- a/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala +++ b/core/src/main/scala/com/salesforce/op/evaluators/OpBinScoreEvaluator.scala @@ -147,7 +147,7 @@ private[op] class OpBinScoreEvaluator * @param binSize size of each bin * @param binCenters center of each bin * @param numberOfDataPoints total number of data points in each bin - * @param numberOfPositiveLabels sum of the label in each bin + * @param numberOfPositiveLabels count of labels > 0 in each bin * @param averageScore average score in each bin * @param averageConversionRate average conversion rate in each bin */